From 3eaa96e666e20165997d8264cee94214433638bc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 23 Oct 2019 16:46:38 +0300 Subject: [PATCH 001/743] add UUID to ASTCreateQuery --- dbms/src/Core/Settings.h | 1 + dbms/src/Core/SettingsCommon.cpp | 6 ++++++ dbms/src/Core/SettingsCommon.h | 7 +++++++ dbms/src/Databases/DatabaseAtomic.cpp | 15 +++++++++++++ dbms/src/Databases/DatabaseAtomic.h | 21 +++++++++++++++++++ dbms/src/Databases/DatabaseFactory.cpp | 3 +++ dbms/src/Databases/IDatabase.h | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 10 ++++++++- dbms/src/Parsers/ASTCreateQuery.h | 1 + dbms/src/Parsers/ParserCreateQuery.cpp | 17 +++++++++++++-- dbms/src/Parsers/ParserCreateQuery.h | 6 +++--- 11 files changed, 82 insertions(+), 7 deletions(-) create mode 100644 dbms/src/Databases/DatabaseAtomic.cpp create mode 100644 dbms/src/Databases/DatabaseAtomic.h diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 30752113a6b..60f6a656534 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -379,6 +379,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.") \ 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.") \ \ + M(SettingDefaultDatabaseEngine, default_database_engine, /*DefaultDatabaseEngine::Ordinary*/ DefaultDatabaseEngine::Atomic, "Default database engine.") \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ diff --git a/dbms/src/Core/SettingsCommon.cpp b/dbms/src/Core/SettingsCommon.cpp index 2de2202eae3..92d4b7166f9 100644 --- a/dbms/src/Core/SettingsCommon.cpp +++ b/dbms/src/Core/SettingsCommon.cpp @@ -460,6 +460,12 @@ IMPLEMENT_SETTING_ENUM(FormatSettings::DateTimeInputFormat, DATE_TIME_INPUT_FORM IMPLEMENT_SETTING_ENUM(LogsLevel, LOGS_LEVEL_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS) +#define DEFAULT_DATABASE_ENGINE_LIST_OF_NAMES(M) \ + M(Ordinary, "Ordinary") \ + M(Atomic, "Atomic") +IMPLEMENT_SETTING_ENUM(DefaultDatabaseEngine , DEFAULT_DATABASE_ENGINE_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS) + + namespace details { String SettingsCollectionUtils::deserializeName(ReadBuffer & buf) diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index d99b8f8a357..fde576bd131 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -268,6 +268,13 @@ enum class LogsLevel }; using SettingLogsLevel = SettingEnum; +enum class DefaultDatabaseEngine +{ + Ordinary, + Atomic, +}; +using SettingDefaultDatabaseEngine = SettingEnum; + namespace details { diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp new file mode 100644 index 00000000000..cc67b45f8a1 --- /dev/null +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -0,0 +1,15 @@ +#include + + +namespace DB +{ + +DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Context & context_) + : DatabaseOrdinary(name_, metadata_path_, context_) +{ +} + + + +} + diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h new file mode 100644 index 00000000000..fa1e3573a8a --- /dev/null +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +class DatabaseAtomic : /* public DatabaseWithOwnTablesBase */ public DatabaseOrdinary +{ +public: + + DatabaseAtomic(String name_, String metadata_path_, const Context & context_); + + String getEngineName() const override { return "Atomic"; } + + +}; + +} diff --git a/dbms/src/Databases/DatabaseFactory.cpp b/dbms/src/Databases/DatabaseFactory.cpp index fe81bd3d495..fe90f2cea26 100644 --- a/dbms/src/Databases/DatabaseFactory.cpp +++ b/dbms/src/Databases/DatabaseFactory.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -43,6 +44,8 @@ DatabasePtr DatabaseFactory::get( if (engine_name == "Ordinary") return std::make_shared(database_name, metadata_path, context); + else if (engine_name == "Atomic") + return std::make_shared(database_name, metadata_path, context); else if (engine_name == "Memory") return std::make_shared(database_name); else if (engine_name == "Dictionary") diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 8fce43c4b2d..4ec56f6f3d4 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -124,7 +124,7 @@ public: const ConstraintsDescription & /*constraints*/, const ASTModifier & /*engine_modifier*/) { - throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(getEngineName() + ": alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 8913f6a4b12..7f9c86f4038 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -99,7 +99,12 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) String database_engine_name; if (!create.storage) { - database_engine_name = "Ordinary"; /// Default database engine. + /// For new-style databases engine is explicitly specified in .sql + /// When attaching old-style database during server startup, we must always use Ordinary engine + // FIXME maybe throw an exception if it's an ATTACH DATABASE query from user (it's not server startup) and engine is not specified + bool old_style_database = create.attach || + context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; + database_engine_name = old_style_database ? "Ordinary" : "Atomic"; auto engine = std::make_shared(); engine->name = database_engine_name; auto storage = std::make_shared(); @@ -620,6 +625,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.temporary || create.is_live_view) { database = context.getDatabase(database_name); + if (!create.uuid.empty() && database->getEngineName() != "Atomic") + throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", + ErrorCodes::INCORRECT_QUERY); data_path = database->getDataPath(); /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 41847c28b1e..9e88e51bdd6 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -73,6 +73,7 @@ public: ASTPtr as_table_function; ASTSelectWithUnionQuery * select = nullptr; ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.) + String uuid; /// For ATTACH TABLE query when db engine is Atomic /** Get the text that identifies this element. */ String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + database) + delim + table; } diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index ce1490d18bd..9a54d843e55 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -336,10 +336,12 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create("CREATE"); - ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_attach("ATTACH"); + ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_table("TABLE"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserKeyword s_uuid("UUID"); + ParserKeyword s_on("ON"); ParserKeyword s_as("AS"); ParserToken s_dot(TokenType::Dot); ParserToken s_lparen(TokenType::OpeningRoundBracket); @@ -350,9 +352,11 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserSelectWithUnionQuery select_p; ParserFunction table_function_p; ParserNameList names_p; + ParserStringLiteral uuid_p; ASTPtr database; ASTPtr table; + ASTPtr uuid; ASTPtr columns_list; ASTPtr to_database; ASTPtr to_table; @@ -395,7 +399,14 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; } - if (ParserKeyword{"ON"}.ignore(pos, expected)) + if (attach && s_uuid.ignore(pos, expected)) + { + /// For CREATE query uuid will be generated + if (!uuid_p.parse(pos, uuid, expected)) + return false; + } + + if (s_on.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) return false; @@ -471,6 +482,8 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe tryGetIdentifierNameInto(database, query->database); tryGetIdentifierNameInto(table, query->table); + if (uuid) + query->uuid = uuid->as()->value.get(); query->cluster = cluster_str; tryGetIdentifierNameInto(to_database, query->to_database); diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 7b31a43af30..99e63058fdd 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -289,7 +289,7 @@ protected: }; /** Query like this: - * CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name + * CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] * ( * name1 type1, * name2 type2, @@ -299,10 +299,10 @@ protected: * ) ENGINE = engine * * Or: - * CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine] + * CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] AS [db2.]name2 [ENGINE = engine] * * Or: - * CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name AS ENGINE = engine SELECT ... + * CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] AS ENGINE = engine SELECT ... * */ class ParserCreateTableQuery : public IParserBase From 9c461ff1432b53910cffacf87b012e6187573493 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 23 Oct 2019 21:39:07 +0300 Subject: [PATCH 002/743] refactor InterpreterCreateQuery::createTable(...) --- dbms/src/Databases/DatabaseOrdinary.cpp | 12 +- .../Interpreters/InterpreterCreateQuery.cpp | 276 +++++++++--------- .../src/Interpreters/InterpreterCreateQuery.h | 17 +- dbms/src/Parsers/IAST.h | 9 + dbms/src/Storages/StorageFactory.cpp | 2 +- dbms/src/Storages/StorageFactory.h | 2 +- 6 files changed, 160 insertions(+), 158 deletions(-) diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 5987597e3e9..d8f30c1113a 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -269,16 +269,8 @@ void DatabaseOrdinary::alterTable( ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(constraints); ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); - - if (ast_create_query.columns_list->indices) - ast_create_query.columns_list->replace(ast_create_query.columns_list->indices, new_indices); - else - ast_create_query.columns_list->set(ast_create_query.columns_list->indices, new_indices); - - if (ast_create_query.columns_list->constraints) - ast_create_query.columns_list->replace(ast_create_query.columns_list->constraints, new_constraints); - else - ast_create_query.columns_list->set(ast_create_query.columns_list->constraints, new_constraints); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints); if (storage_modifier) storage_modifier(*ast_create_query.storage); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 7f9c86f4038..4abf60f39b7 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -402,84 +402,97 @@ ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const A } -ColumnsDescription InterpreterCreateQuery::setProperties( - ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const +InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(ASTCreateQuery & create) const { - ColumnsDescription columns; - IndicesDescription indices; - ConstraintsDescription constraints; + TableProperties properties; + TableStructureReadLockHolder as_storage_lock; if (create.columns_list) { if (create.columns_list->columns) - columns = getColumnsDescription(*create.columns_list->columns, context); + properties.columns = getColumnsDescription(*create.columns_list->columns, context); if (create.columns_list->indices) for (const auto & index : create.columns_list->indices->children) - indices.indices.push_back( + properties.indices.indices.push_back( std::dynamic_pointer_cast(index->clone())); - if (create.columns_list->constraints) - for (const auto & constraint : create.columns_list->constraints->children) - constraints.constraints.push_back( - std::dynamic_pointer_cast(constraint->clone())); + properties.constraints = getConstraintsDescription(create.columns_list->constraints); } else if (!create.as_table.empty()) { - columns = as_storage->getColumns(); + String as_database_name = create.as_database.empty() ? context.getCurrentDatabase() : create.as_database; + StoragePtr as_storage = context.getTable(as_database_name, create.as_table); + + /// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table. + as_storage_lock = as_storage->lockStructureForShare(false, context.getCurrentQueryId()); + properties.columns = as_storage->getColumns(); /// Secondary indices make sense only for MergeTree family of storage engines. /// We should not copy them for other storages. if (create.storage && endsWith(create.storage->engine->name, "MergeTree")) - indices = as_storage->getIndices(); + properties.indices = as_storage->getIndices(); - constraints = as_storage->getConstraints(); + properties.constraints = as_storage->getConstraints(); } else if (create.select) { - columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); + Block as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), context); + properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); } + else if (create.as_table_function) + return {}; else throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); + /// Even if query has list of columns, canonicalize it (unfold Nested columns). - ASTPtr new_columns = formatColumns(columns); - ASTPtr new_indices = formatIndices(indices); - ASTPtr new_constraints = formatConstraints(constraints); - if (!create.columns_list) - { - auto new_columns_list = std::make_shared(); - create.set(create.columns_list, new_columns_list); - } + create.set(create.columns_list, std::make_shared()); - if (create.columns_list->columns) - create.columns_list->replace(create.columns_list->columns, new_columns); - else - create.columns_list->set(create.columns_list->columns, new_columns); + ASTPtr new_columns = formatColumns(properties.columns); + ASTPtr new_indices = formatIndices(properties.indices); + ASTPtr new_constraints = formatConstraints(properties.constraints); - if (new_indices && create.columns_list->indices) - create.columns_list->replace(create.columns_list->indices, new_indices); - else if (new_indices) - create.columns_list->set(create.columns_list->indices, new_indices); + create.columns_list->setOrReplace(create.columns_list->columns, new_columns); + create.columns_list->setOrReplace(create.columns_list->indices, new_indices); + create.columns_list->setOrReplace(create.columns_list->constraints, new_constraints); - if (new_constraints && create.columns_list->constraints) - create.columns_list->replace(create.columns_list->constraints, new_constraints); - else if (new_constraints) - create.columns_list->set(create.columns_list->constraints, new_constraints); + validateTableStructure(create, properties); + /// Set the table engine if it was not specified explicitly. + setEngine(create); + return properties; +} + +void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & create, + const InterpreterCreateQuery::TableProperties & properties) const +{ /// Check for duplicates std::set all_columns; - for (const auto & column : columns) + for (const auto & column : properties.columns) { if (!all_columns.emplace(column.name).second) throw Exception("Column " + backQuoteIfNeed(column.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN); } - return columns; + /// Check low cardinality types in creating table if it was not allowed in setting + if (!create.attach && !context.getSettingsRef().allow_suspicious_low_cardinality_types && !create.is_materialized_view) + { + for (const auto & name_and_type_pair : properties.columns.getAllPhysical()) + { + if (const auto * current_type_ptr = typeid_cast(name_and_type_pair.type.get())) + { + if (!isStringOrFixedString(*removeNullable(current_type_ptr->getDictionaryType()))) + throw Exception("Creating columns of type " + current_type_ptr->getName() + " is prohibited by default " + "due to expected negative impact on performance. " + "It can be enabled with the \"allow_suspicious_low_cardinality_types\" setting.", + ErrorCodes::SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY); + } + } + } } - void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.storage) @@ -541,12 +554,10 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) 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); - String path = context.getPath(); String current_database = context.getCurrentDatabase(); String database_name = create.database.empty() ? current_database : create.database; String table_name = create.table; - String table_name_escaped = escapeForFileName(table_name); // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) @@ -566,26 +577,66 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) visitor.visit(*create.select); } - Block as_select_sample; - if (create.select && (!create.attach || !create.columns_list)) - as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), context); + /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. + TableProperties properties = setProperties(create); - String as_database_name = create.as_database.empty() ? current_database : create.as_database; - String as_table_name = create.as_table; + /// Actually creates table + bool created = doCreateTable(create, properties, database_name); + if (!created) + return {}; - StoragePtr as_storage; - TableStructureReadLockHolder as_storage_lock; + return fillTableIfNeeded(create, database_name); +} - if (!as_table_name.empty()) +bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, + const InterpreterCreateQuery::TableProperties & properties, + const String & database_name) +{ + std::unique_ptr guard; + + String data_path; + DatabasePtr database; + + const String & table_name = create.table; + bool need_add_to_database = !create.temporary || create.is_live_view; + if (need_add_to_database) { - as_storage = context.getTable(as_database_name, as_table_name); - as_storage_lock = as_storage->lockStructureForShare(false, context.getCurrentQueryId()); + database = context.getDatabase(database_name); + if (!create.uuid.empty() && database->getEngineName() != "Atomic") + throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", + ErrorCodes::INCORRECT_QUERY); + + data_path = database->getDataPath(); + + /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). + * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. + */ + guard = context.getDDLGuard(database_name, table_name); + + /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. + if (database->isTableExist(context, table_name)) + { + if (create.if_not_exists) + return false; + else if (create.replace_view) + { + /// when executing CREATE OR REPLACE VIEW, drop current existing view + auto drop_ast = std::make_shared(); + drop_ast->database = database_name; + drop_ast->table = table_name; + drop_ast->no_ddl_lock = true; + + InterpreterDropQuery interpreter(drop_ast, context); + interpreter.execute(); + } + else + throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + } } + else if (context.tryGetExternalTable(table_name) && create.if_not_exists) + return false; - ColumnsDescription columns; - ConstraintsDescription constraints; StoragePtr res; - if (create.as_table_function) { const auto & table_function = create.as_table_function->as(); @@ -594,101 +645,38 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } else { - /// Set and retrieve list of columns. - columns = setProperties(create, as_select_sample, as_storage); - constraints = getConstraintsDescription(create.columns_list->constraints); - - /// Check low cardinality types in creating table if it was not allowed in setting - if (!create.attach && !context.getSettingsRef().allow_suspicious_low_cardinality_types && !create.is_materialized_view) - { - for (const auto & name_and_type_pair : columns.getAllPhysical()) - { - if (const auto * current_type_ptr = typeid_cast(name_and_type_pair.type.get())) - { - if (!isStringOrFixedString(*removeNullable(current_type_ptr->getDictionaryType()))) - throw Exception("Creating columns of type " + current_type_ptr->getName() + " is prohibited by default due to expected negative impact on performance. It can be enabled with the \"allow_suspicious_low_cardinality_types\" setting.", - ErrorCodes::SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY); - } - } - } - - /// Set the table engine if it was not specified explicitly. - setEngine(create); + res = StorageFactory::instance().get(create, + data_path, + table_name, + database_name, + context, + context.getGlobalContext(), + properties.columns, + properties.constraints, + create.attach, + false); } - { - std::unique_ptr guard; + if (need_add_to_database) + database->createTable(context, table_name, res, query_ptr); + else + context.getSessionContext().addExternalTable(table_name, res, query_ptr); - String data_path; - DatabasePtr database; + /// We must call "startup" and "shutdown" while holding DDLGuard. + /// Because otherwise method "shutdown" (from InterpreterDropQuery) can be called before startup + /// (in case when table was created and instantly dropped before started up) + /// + /// Method "startup" may create background tasks and method "shutdown" will wait for them. + /// But if "shutdown" is called before "startup", it will exit early, because there are no background tasks to wait. + /// Then background task is created by "startup" method. And when destructor of a table object is called, background task is still active, + /// and the task will use references to freed data. - if (!create.temporary || create.is_live_view) - { - database = context.getDatabase(database_name); - if (!create.uuid.empty() && database->getEngineName() != "Atomic") - throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", - ErrorCodes::INCORRECT_QUERY); - data_path = database->getDataPath(); - - /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). - * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. - */ - guard = context.getDDLGuard(database_name, table_name); - - /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. - if (database->isTableExist(context, table_name)) - { - if (create.if_not_exists) - return {}; - else if (create.replace_view) - { - /// when executing CREATE OR REPLACE VIEW, drop current existing view - auto drop_ast = std::make_shared(); - drop_ast->database = database_name; - drop_ast->table = table_name; - drop_ast->no_ddl_lock = true; - - InterpreterDropQuery interpreter(drop_ast, context); - interpreter.execute(); - } - else - throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); - } - } - else if (context.tryGetExternalTable(table_name) && create.if_not_exists) - return {}; - - if (!create.as_table_function) - { - res = StorageFactory::instance().get(create, - data_path, - table_name, - database_name, - context, - context.getGlobalContext(), - columns, - constraints, - create.attach, - false); - } - - if (create.temporary && !create.is_live_view) - context.getSessionContext().addExternalTable(table_name, res, query_ptr); - else - database->createTable(context, table_name, res, query_ptr); - - /// We must call "startup" and "shutdown" while holding DDLGuard. - /// Because otherwise method "shutdown" (from InterpreterDropQuery) can be called before startup - /// (in case when table was created and instantly dropped before started up) - /// - /// Method "startup" may create background tasks and method "shutdown" will wait for them. - /// But if "shutdown" is called before "startup", it will exit early, because there are no background tasks to wait. - /// Then background task is created by "startup" method. And when destructor of a table object is called, background task is still active, - /// and the task will use references to freed data. - - res->startup(); - } + res->startup(); + return true; +} +BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name) +{ /// 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)) @@ -698,7 +686,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.temporary) insert->database = database_name; - insert->table = table_name; + insert->table = create.table; insert->select = create.select->clone(); if (create.temporary && !context.getSessionContext().hasQueryContext()) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index 2472f0321e1..be269496ab4 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -49,14 +49,27 @@ public: static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints); private: + struct TableProperties + { + ColumnsDescription columns; + IndicesDescription indices; + ConstraintsDescription constraints; + }; + BlockIO createDatabase(ASTCreateQuery & create); BlockIO createTable(ASTCreateQuery & create); - /// Calculate list of columns, constraints, indices, etc... of table and return columns. - ColumnsDescription setProperties(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const; + /// Calculate list of columns, constraints, indices, etc... of table. Rewrite query in canonical way. + TableProperties setProperties(ASTCreateQuery & create) const; + void validateTableStructure(const ASTCreateQuery & create, const TableProperties & properties) const; void setEngine(ASTCreateQuery & create) const; void checkAccess(const ASTCreateQuery & create); + /// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false. + bool doCreateTable(const ASTCreateQuery & create, const TableProperties & properties, const String & database_name); + /// Inserts data in created table if it's CREATE ... SELECT + BlockIO fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name); + ASTPtr query_ptr; Context & context; diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index d7c56d80a21..246b88f8c24 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -146,6 +146,15 @@ public: throw Exception("AST subtree not found in children", ErrorCodes::LOGICAL_ERROR); } + template + void setOrReplace(T * & field, const ASTPtr & child) + { + if (field) + replace(field, child); + else + set(field, child); + } + /// Convert to a string. /// Format settings. diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index d3796735a07..6e0e630efea 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -39,7 +39,7 @@ void StorageFactory::registerStorage(const std::string & name, Creator creator) StoragePtr StorageFactory::get( - ASTCreateQuery & query, + const ASTCreateQuery & query, const String & data_path, const String & table_name, const String & database_name, diff --git a/dbms/src/Storages/StorageFactory.h b/dbms/src/Storages/StorageFactory.h index ca19c55522d..d95c7a0c87b 100644 --- a/dbms/src/Storages/StorageFactory.h +++ b/dbms/src/Storages/StorageFactory.h @@ -46,7 +46,7 @@ public: using Creator = std::function; StoragePtr get( - ASTCreateQuery & query, + const ASTCreateQuery & query, const String & data_path, const String & table_name, const String & database_name, From 80c9b2ec1ca620165721bc6c7a4b7fd3b5328fc5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 25 Oct 2019 22:07:47 +0300 Subject: [PATCH 003/743] make data path relative --- dbms/src/Databases/DatabaseLazy.cpp | 14 +++--- dbms/src/Databases/DatabaseLazy.h | 4 +- dbms/src/Databases/DatabaseOnDisk.cpp | 8 ++-- dbms/src/Databases/DatabaseOnDisk.h | 4 +- dbms/src/Databases/DatabaseOrdinary.cpp | 12 ++--- dbms/src/Databases/DatabaseOrdinary.h | 2 +- dbms/src/Databases/DatabasesCommon.cpp | 6 ++- dbms/src/Databases/DatabasesCommon.h | 2 +- dbms/src/Databases/IDatabase.h | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../src/Interpreters/InterpreterDropQuery.cpp | 4 +- .../Interpreters/InterpreterSystemQuery.cpp | 3 +- dbms/src/Storages/IStorage.h | 11 +++++ dbms/src/Storages/StorageDistributed.cpp | 10 ++--- dbms/src/Storages/StorageDistributed.h | 4 +- dbms/src/Storages/StorageFactory.cpp | 8 ++-- dbms/src/Storages/StorageFactory.h | 8 ++-- dbms/src/Storages/StorageFile.cpp | 23 +++++----- dbms/src/Storages/StorageFile.h | 2 +- dbms/src/Storages/StorageJoin.cpp | 12 ++--- dbms/src/Storages/StorageJoin.h | 5 ++- dbms/src/Storages/StorageLog.cpp | 36 ++++++++------- dbms/src/Storages/StorageLog.h | 9 ++-- dbms/src/Storages/StorageSet.cpp | 22 ++++----- dbms/src/Storages/StorageSet.h | 10 +++-- dbms/src/Storages/StorageStripeLog.cpp | 45 ++++++++++--------- dbms/src/Storages/StorageStripeLog.h | 9 ++-- dbms/src/Storages/StorageTinyLog.cpp | 35 ++++++++------- dbms/src/Storages/StorageTinyLog.h | 7 +-- .../System/StorageSystemDatabases.cpp | 1 + 30 files changed, 176 insertions(+), 144 deletions(-) diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 35a28e539bc..15f0df0fe6d 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -32,22 +32,22 @@ namespace ErrorCodes static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; -DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context) +DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_) : name(name_) , metadata_path(metadata_path_) - , data_path(context.getPath() + "data/" + escapeForFileName(name) + "/") + , data_path("data/" + escapeForFileName(name) + "/") , expiration_time(expiration_time_) , log(&Logger::get("DatabaseLazy (" + name + ")")) { - Poco::File(getDataPath()).createDirectories(); + Poco::File(context_.getPath() + getDataPath()).createDirectories(); } void DatabaseLazy::loadTables( - Context & /* context */, + Context & context, bool /* has_force_restore_data_flag */) { - DatabaseOnDisk::iterateTableFiles(*this, log, [this](const String & file_name) + DatabaseOnDisk::iterateTableFiles(*this, log, context, [this](const String & file_name) { const std::string table_name = file_name.substr(0, file_name.size() - 4); attachTable(table_name, nullptr); @@ -134,9 +134,9 @@ void DatabaseLazy::alterTable( } -void DatabaseLazy::drop() +void DatabaseLazy::drop(const Context & context) { - DatabaseOnDisk::drop(*this); + DatabaseOnDisk::drop(*this, context); } bool DatabaseLazy::isTableExist( diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index 649116508b3..b1fe6b558be 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -16,7 +16,7 @@ class DatabaseLazyIterator; class DatabaseLazy : public IDatabase { public: - DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context); + DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_); String getEngineName() const override { return "Lazy"; } @@ -68,7 +68,7 @@ public: String getMetadataPath() const override; String getTableMetadataPath(const String & table_name) const override; - void drop() override; + void drop(const Context & context) override; bool isTableExist( const Context & context, diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 941935f37f4..9e48829dd5d 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -220,9 +220,9 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const IDatabase & database, const return ast; } -void DatabaseOnDisk::drop(const IDatabase & database) +void DatabaseOnDisk::drop(const IDatabase & database, const Context & context) { - Poco::File(database.getDataPath()).remove(false); + Poco::File(context.getPath() + database.getDataPath()).remove(false); Poco::File(database.getMetadataPath()).remove(false); } @@ -248,7 +248,7 @@ time_t DatabaseOnDisk::getTableMetadataModificationTime( } } -void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function) +void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function) { Poco::DirectoryIterator dir_end; for (Poco::DirectoryIterator dir_it(database.getMetadataPath()); dir_it != dir_end; ++dir_it) @@ -266,7 +266,7 @@ void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger if (endsWith(dir_it.name(), tmp_drop_ext)) { const std::string table_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext)); - if (Poco::File(database.getDataPath() + '/' + table_name).exists()) + if (Poco::File(context.getPath() + database.getDataPath() + '/' + table_name).exists()) { Poco::File(dir_it->path()).renameTo(table_name + ".sql"); LOG_WARNING(log, "Table " << backQuote(table_name) << " was not dropped previously"); diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 231db6fdccb..2529912ae67 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -60,7 +60,7 @@ public: const IDatabase & database, const Context & context); - static void drop(const IDatabase & database); + static void drop(const IDatabase & database, const Context & context); static String getTableMetadataPath( const IDatabase & database, @@ -72,7 +72,7 @@ public: using IteratingFunction = std::function; - static void iterateTableFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function); + static void iterateTableFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function); private: static ASTPtr getCreateTableQueryImpl( diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index d8f30c1113a..a5690c50fce 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -87,13 +87,13 @@ static void loadTable( } -DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context) +DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context_) : DatabaseWithOwnTablesBase(std::move(name_)) , metadata_path(metadata_path_) - , data_path(context.getPath() + "data/" + escapeForFileName(name) + "/") + , data_path("data/" + escapeForFileName(name) + "/") , log(&Logger::get("DatabaseOrdinary (" + name + ")")) { - Poco::File(getDataPath()).createDirectories(); + Poco::File(context_.getPath() + getDataPath()).createDirectories(); } @@ -104,7 +104,7 @@ void DatabaseOrdinary::loadTables( using FileNames = std::vector; FileNames file_names; - DatabaseOnDisk::iterateTableFiles(*this, log, [&file_names](const String & file_name) + DatabaseOnDisk::iterateTableFiles(*this, log, context, [&file_names](const String & file_name) { file_names.push_back(file_name); }); @@ -299,9 +299,9 @@ void DatabaseOrdinary::alterTable( } -void DatabaseOrdinary::drop() +void DatabaseOrdinary::drop(const Context & context) { - DatabaseOnDisk::drop(*this); + DatabaseOnDisk::drop(*this, context); } diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index d8f7e1b3612..5d0ddf7c337 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -66,7 +66,7 @@ public: String getMetadataPath() const override; String getTableMetadataPath(const String & table_name) const override; - void drop() override; + void drop(const Context & context) override; private: const String metadata_path; diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 1a278fbb03d..ff2628ddb96 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -57,7 +58,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query) std::pair createTableFromDefinition( const String & definition, const String & database_name, - const String & database_data_path, + const String & relative_db_path, Context & context, bool has_force_restore_data_flag, const String & description_for_error_message) @@ -90,7 +91,8 @@ std::pair createTableFromDefinition( ast_create_query.table, StorageFactory::instance().get( ast_create_query, - database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(), + relative_db_path + escapeForFileName(ast_create_query.table) + '/', + ast_create_query.table, database_name, context, context.getGlobalContext(), columns, constraints, true, has_force_restore_data_flag) }; diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 9eba0fc0596..76049f4dfcb 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -31,7 +31,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query); std::pair createTableFromDefinition( const String & definition, const String & database_name, - const String & database_data_path, + const String & relative_db_path, Context & context, bool has_force_restore_data_flag, const String & description_for_error_message); diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 4ec56f6f3d4..14d03bcd5dc 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -156,7 +156,7 @@ public: virtual void shutdown() = 0; /// Delete data and metadata stored inside the database, if exists. - virtual void drop() {} + virtual void drop(const Context & /*context*/) {} virtual ~IDatabase() {} }; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 4abf60f39b7..2d272c28f90 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -646,7 +646,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, else { res = StorageFactory::instance().get(create, - data_path, + data_path + escapeForFileName(table_name) + "/", table_name, database_name, context, diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 565863d139a..df5b572d633 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -119,7 +119,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t database_and_table.first->removeTable(context, database_and_table.second->getTableName()); database_and_table.second->is_dropped = true; - String database_data_path = database_and_table.first->getDataPath(); + String database_data_path = context.getPath() + database_and_table.first->getDataPath(); /// If it is not virtual database like Dictionary then drop remaining data dir if (!database_data_path.empty()) @@ -206,7 +206,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQ database->shutdown(); /// Delete the database. - database->drop(); + database->drop(context); /// Old ClickHouse versions did not store database.sql files Poco::File database_metadata_file(context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"); diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 6da0b9333ac..1fd6d49fe72 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -289,7 +290,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints); StoragePtr table = StorageFactory::instance().get(create, - data_path, + data_path + escapeForFileName(table_name) + "/", table_name, database_name, system_context, diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 8d30f31bde3..b067bb9318a 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -285,6 +285,17 @@ public: throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + // TODO refactor rename() and renameAtomic() + + /** + * Just updates names of database and table without moving any data on disk + * Can be called only from DatabaseAtomic. + */ + virtual void renameAtomic(const String & /*new_database_name*/, const String & /*new_table_name*/) + { + throw Exception("Method renameAtomic is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + /** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters. * This method must fully execute the ALTER query, taking care of the locks itself. * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata. diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 2c289dd714e..3cf0e1889f8 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -215,12 +215,12 @@ StorageDistributed::StorageDistributed( const String & cluster_name_, const Context & context_, const ASTPtr & sharding_key_, - const String & data_path_, + const String & relative_data_path_, bool attach_) : table_name(table_name_), database_name(database_name_), remote_database(remote_database_), remote_table(remote_table_), global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_), - path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(table_name) + '/')) + path(relative_data_path_.empty() ? "" : (context_.getPath() + relative_data_path_)) { setColumns(columns_); setConstraints(constraints_); @@ -250,9 +250,9 @@ StorageDistributed::StorageDistributed( const String & cluster_name_, const Context & context_, const ASTPtr & sharding_key_, - const String & data_path_, + const String & relative_data_path_, bool attach) - : StorageDistributed(database_name_, table_name_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, data_path_, attach) + : StorageDistributed(database_name_, table_name_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, relative_data_path_, attach) { remote_table_function_ptr = remote_table_function_ptr_; } @@ -641,7 +641,7 @@ void registerStorageDistributed(StorageFactory & factory) return StorageDistributed::create( args.database_name, args.table_name, args.columns, args.constraints, remote_database, remote_table, cluster_name, - args.context, sharding_key, args.data_path, + args.context, sharding_key, args.relative_data_path, args.attach); }); } diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index e2409fe136e..b6cf349c770 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -172,7 +172,7 @@ protected: const String & cluster_name_, const Context & context_, const ASTPtr & sharding_key_, - const String & data_path_, + const String & relative_data_path_, bool attach_); StorageDistributed( @@ -184,7 +184,7 @@ protected: const String & cluster_name_, const Context & context_, const ASTPtr & sharding_key_, - const String & data_path_, + const String & relative_data_path_, bool attach); ClusterPtr skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info); diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 6e0e630efea..59828932eb2 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -40,9 +40,9 @@ void StorageFactory::registerStorage(const std::string & name, Creator creator) StoragePtr StorageFactory::get( const ASTCreateQuery & query, - const String & data_path, + const String & relative_data_path, const String & table_name, - const String & database_name, + const String & database_name, // TODO remove Context & local_context, Context & context, const ColumnsDescription & columns, @@ -150,9 +150,9 @@ StoragePtr StorageFactory::get( .engine_args = args, .storage_def = storage_def, .query = query, - .data_path = data_path, + .relative_data_path = relative_data_path, .table_name = table_name, - .database_name = database_name, + .database_name = database_name, //TODO remove .local_context = local_context, .context = context, .columns = columns, diff --git a/dbms/src/Storages/StorageFactory.h b/dbms/src/Storages/StorageFactory.h index d95c7a0c87b..945bfaee0a4 100644 --- a/dbms/src/Storages/StorageFactory.h +++ b/dbms/src/Storages/StorageFactory.h @@ -32,9 +32,11 @@ public: ASTs & engine_args; ASTStorage * storage_def; const ASTCreateQuery & query; - const String & data_path; + /// Path to table data. + /// Relative to from server config (possibly of some of some for *MergeTree) + const String & relative_data_path; const String & table_name; - const String & database_name; + const String & database_name; //TODO remove Context & local_context; Context & context; const ColumnsDescription & columns; @@ -47,7 +49,7 @@ public: StoragePtr get( const ASTCreateQuery & query, - const String & data_path, + const String & relative_data_path, const String & table_name, const String & database_name, Context & local_context, diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index d38f1f76ccd..c27fb3b9b9a 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -52,7 +52,7 @@ namespace /* Recursive directory listing with matched paths as a result. * Have the same method in StorageHDFS. */ -std::vector listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match) +static std::vector listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match) { const size_t first_glob = for_match.find_first_of("*?{"); @@ -96,13 +96,13 @@ std::vector listFilesWithRegexpMatching(const std::string & path_fo return result; } -std::string getTablePath(const std::string & db_dir_path, const std::string & table_name, const std::string & format_name) +static std::string getTablePath(const std::string & table_dir_path, const std::string & format_name) { - return db_dir_path + escapeForFileName(table_name) + "/data." + escapeForFileName(format_name); + return table_dir_path + "/data." + escapeForFileName(format_name); } /// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..'). -void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path) +static void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path) { if (context_global.getApplicationType() != Context::ApplicationType::SERVER) return; @@ -121,7 +121,7 @@ void checkCreationIsAllowed(Context & context_global, const std::string & db_dir StorageFile::StorageFile( const std::string & table_path_, int table_fd_, - const std::string & db_dir_path, + const std::string & relative_table_dir_path, const std::string & database_name_, const std::string & table_name_, const std::string & format_name_, @@ -136,26 +136,27 @@ StorageFile::StorageFile( if (table_fd < 0) /// Will use file { + String table_dir_path = context_global.getPath() + relative_table_dir_path + "/"; use_table_fd = false; if (!table_path_.empty()) /// Is user's file { Poco::Path poco_path = Poco::Path(table_path_); if (poco_path.isRelative()) - poco_path = Poco::Path(db_dir_path, poco_path); + poco_path = Poco::Path(table_dir_path, poco_path); const std::string path = poco_path.absolute().toString(); paths = listFilesWithRegexpMatching("/", path); for (const auto & cur_path : paths) - checkCreationIsAllowed(context_global, db_dir_path, cur_path); + checkCreationIsAllowed(context_global, table_dir_path, cur_path); is_db_table = false; } else /// Is DB's file { - if (db_dir_path.empty()) + if (relative_table_dir_path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - paths = {getTablePath(db_dir_path, table_name, format_name)}; + paths = {getTablePath(table_dir_path, format_name)}; is_db_table = true; Poco::File(Poco::Path(paths.back()).parent()).createDirectories(); } @@ -344,7 +345,7 @@ void StorageFile::rename(const String & new_path_to_db, const String & new_datab std::unique_lock lock(rwlock); - std::string path_new = getTablePath(new_path_to_db, new_table_name, format_name); + std::string path_new = getTablePath(new_path_to_db + escapeForFileName(new_table_name), format_name); Poco::File(Poco::Path(path_new).parent()).createDirectories(); Poco::File(paths[0]).renameTo(path_new); @@ -400,7 +401,7 @@ void registerStorageFile(StorageFactory & factory) return StorageFile::create( source_path, source_fd, - args.data_path, + args.relative_data_path, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context); }); diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 0d9854f75cf..019fb96c75b 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -54,7 +54,7 @@ protected: StorageFile( const std::string & table_path_, int table_fd_, - const std::string & db_dir_path, + const std::string & relative_table_dir_path, const std::string & database_name_, const std::string & table_name_, const std::string & format_name_, diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index d3f41f3b0b5..408e0c8f775 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes } StorageJoin::StorageJoin( - const String & path_, + const String & relative_path_, const String & database_name_, const String & table_name_, const Names & key_names_, @@ -38,8 +38,9 @@ StorageJoin::StorageJoin( ASTTableJoin::Strictness strictness_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - bool overwrite) - : StorageSetOrJoinBase{path_, database_name_, table_name_, columns_, constraints_} + bool overwrite, + const Context & context_) + : StorageSetOrJoinBase{relative_path_, database_name_, table_name_, columns_, constraints_, context_} , key_names(key_names_) , use_nulls(use_nulls_) , limits(limits_) @@ -162,7 +163,7 @@ void registerStorageJoin(StorageFactory & factory) } return StorageJoin::create( - args.data_path, + args.relative_data_path, args.database_name, args.table_name, key_names, @@ -172,7 +173,8 @@ void registerStorageJoin(StorageFactory & factory) strictness, args.columns, args.constraints, - join_any_take_last_row); + join_any_take_last_row, + args.context); }); } diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index d770078ac39..5398bf762e9 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -59,7 +59,7 @@ private: protected: StorageJoin( - const String & path_, + const String & relative_path_, const String & database_name_, const String & table_name_, const Names & key_names_, @@ -68,7 +68,8 @@ protected: ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - bool overwrite); + bool overwrite, + const Context & context_); }; } diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 3811b226357..1fe41634239 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -211,7 +211,7 @@ Block LogBlockInputStream::readImpl() } catch (Exception & e) { - e.addMessage("while reading column " + name_type.name + " at " + storage.path + escapeForFileName(storage.table_name)); + e.addMessage("while reading column " + name_type.name + " at " + storage.path); throw; } @@ -418,29 +418,30 @@ void LogBlockOutputStream::writeMarks(MarksForColumns && marks) } StorageLog::StorageLog( - const std::string & path_, + const std::string & relative_path_, const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - size_t max_compress_block_size_) - : path(path_), table_name(table_name_), database_name(database_name_), + size_t max_compress_block_size_, + const Context & context_) + : path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), - file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json") + file_checker(path + "sizes.json") { setColumns(columns_); setConstraints(constraints_); - if (path.empty()) + if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); /// create files if they do not exist - Poco::File(path + escapeForFileName(table_name) + '/').createDirectories(); + Poco::File(path).createDirectories(); for (const auto & column : getColumns().getAllPhysical()) addFiles(column.name, *column.type); - marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME); + marks_file = Poco::File(path + DBMS_STORAGE_LOG_MARKS_FILE_NAME); } @@ -459,7 +460,7 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type) ColumnData & column_data = files[stream_name]; column_data.column_index = file_count; column_data.data_file = Poco::File{ - path + escapeForFileName(table_name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION}; + path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION}; column_names_by_idx.push_back(stream_name); ++file_count; @@ -517,24 +518,25 @@ void StorageLog::rename(const String & new_path_to_db, const String & new_databa std::unique_lock lock(rwlock); /// Rename directory with data. - Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name)); + String new_path = new_path_to_db + escapeForFileName(new_table_name) + '/'; + Poco::File(path).renameTo(new_path); - path = new_path_to_db; + path = new_path; table_name = new_table_name; database_name = new_database_name; - file_checker.setPath(path + escapeForFileName(table_name) + '/' + "sizes.json"); + file_checker.setPath(path + "sizes.json"); for (auto & file : files) - file.second.data_file = Poco::File(path + escapeForFileName(table_name) + '/' + Poco::Path(file.second.data_file.path()).getFileName()); + file.second.data_file = Poco::File(path + Poco::Path(file.second.data_file.path()).getFileName()); - marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME); + marks_file = Poco::File(path + DBMS_STORAGE_LOG_MARKS_FILE_NAME); } void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { std::shared_lock lock(rwlock); - String table_dir = path + escapeForFileName(table_name); + String table_dir = path; files.clear(); file_count = 0; @@ -647,8 +649,8 @@ void registerStorageLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageLog::create( - args.data_path, args.database_name, args.table_name, args.columns, args.constraints, - args.context.getSettings().max_compress_block_size); + args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, + args.context.getSettings().max_compress_block_size, args.context); }); } diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index df50b1132a2..b911c5ab4fe 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -44,7 +44,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - std::string fullPath() const { return path + escapeForFileName(table_name) + '/';} + std::string fullPath() const { return path; } Strings getDataPaths() const override { return {fullPath()}; } @@ -54,12 +54,13 @@ protected: * consisting of the specified columns; Create files if they do not exist. */ StorageLog( - const std::string & path_, + const std::string & relative_path_, const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - size_t max_compress_block_size_); + size_t max_compress_block_size_, + const Context & context_); private: String path; @@ -124,7 +125,7 @@ private: */ const Marks & getMarksWithRealRowCount() const; - std::string getFullPath() const { return path + escapeForFileName(table_name) + '/'; } + std::string getFullPath() const { return path; } }; } diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index c76857bf610..50f8773df90 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -87,31 +87,33 @@ BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageSetOrJoinBase::StorageSetOrJoinBase( - const String & path_, + const String & relative_path_, const String & database_name_, const String & table_name_, const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_) + const ConstraintsDescription & constraints_, + const Context & context_) : table_name(table_name_), database_name(database_name_) { setColumns(columns_); setConstraints(constraints_); - if (path_.empty()) + if (relative_path_.empty()) throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME); - path = path_ + escapeForFileName(table_name_) + '/'; + path = context_.getPath() + relative_path_; } StorageSet::StorageSet( - const String & path_, + const String & relative_path_, const String & database_name_, const String & table_name_, const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_) - : StorageSetOrJoinBase{path_, database_name_, table_name_, columns_, constraints_}, + const ConstraintsDescription & constraints_, + const Context & context_) + : StorageSetOrJoinBase{relative_path_, database_name_, table_name_, columns_, constraints_, context_}, set(std::make_shared(SizeLimits(), false)) { Block header = getSampleBlock(); @@ -197,10 +199,10 @@ void StorageSetOrJoinBase::rename( const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { /// Rename directory with data. - String new_path = new_path_to_db + escapeForFileName(new_table_name); + String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/"; Poco::File(path).renameTo(new_path); - path = new_path + "/"; + path = new_path; table_name = new_table_name; database_name = new_database_name; } @@ -215,7 +217,7 @@ void registerStorageSet(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageSet::create(args.data_path, args.database_name, args.table_name, args.columns, args.constraints); + return StorageSet::create(args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, args.context); }); } diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index 954e58d3929..e993039c304 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -30,11 +30,12 @@ public: protected: StorageSetOrJoinBase( - const String & path_, + const String & relative_path_, const String & database_name_, const String & table_name_, const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_); + const ConstraintsDescription & constraints_, + const Context & context_); String path; String table_name; @@ -79,11 +80,12 @@ private: protected: StorageSet( - const String & path_, + const String & relative_path_, const String & database_name_, const String & table_name_, const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_); + const ConstraintsDescription & constraints_, + const Context & context_); }; } diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 00c359f9797..b7ae8de92f4 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -121,8 +121,8 @@ private: started = true; data_in.emplace( - storage.full_path() + "data.bin", 0, 0, - std::min(static_cast(max_read_buffer_size), Poco::File(storage.full_path() + "data.bin").getSize())); + storage.fullPath() + "data.bin", 0, 0, + std::min(static_cast(max_read_buffer_size), Poco::File(storage.fullPath() + "data.bin").getSize())); block_in.emplace(*data_in, 0, index_begin, index_end); } @@ -135,11 +135,11 @@ class StripeLogBlockOutputStream final : public IBlockOutputStream public: explicit StripeLogBlockOutputStream(StorageStripeLog & storage_) : storage(storage_), lock(storage.rwlock), - data_out_compressed(storage.full_path() + "data.bin", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), + data_out_compressed(storage.fullPath() + "data.bin", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), data_out(data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size), - index_out_compressed(storage.full_path() + "index.mrk", INDEX_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), + index_out_compressed(storage.fullPath() + "index.mrk", INDEX_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), index_out(index_out_compressed), - block_out(data_out, 0, storage.getSampleBlock(), false, &index_out, Poco::File(storage.full_path() + "data.bin").getSize()) + block_out(data_out, 0, storage.getSampleBlock(), false, &index_out, Poco::File(storage.fullPath() + "data.bin").getSize()) { } @@ -194,30 +194,30 @@ private: StorageStripeLog::StorageStripeLog( - const std::string & path_, + const std::string & relative_path_, const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool attach, - size_t max_compress_block_size_) - : path(path_), table_name(table_name_), database_name(database_name_), + size_t max_compress_block_size_, + const Context & context_) + : path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), - file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json"), + file_checker(path + "sizes.json"), log(&Logger::get("StorageStripeLog")) { setColumns(columns_); setConstraints(constraints_); - if (path.empty()) + if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - String full_path = path + escapeForFileName(table_name) + '/'; if (!attach) { /// create files if they do not exist - if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST) - throwFromErrnoWithPath("Cannot create directory " + full_path, full_path, + if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST) + throwFromErrnoWithPath("Cannot create directory " + path, path, ErrorCodes::CANNOT_CREATE_DIRECTORY); } } @@ -228,12 +228,13 @@ void StorageStripeLog::rename(const String & new_path_to_db, const String & new_ std::unique_lock lock(rwlock); /// Rename directory with data. - Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name)); + String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/"; + Poco::File(path).renameTo(new_path_to_db + escapeForFileName(new_table_name)); - path = new_path_to_db; + path = new_path; table_name = new_table_name; database_name = new_database_name; - file_checker.setPath(path + escapeForFileName(table_name) + "/" + "sizes.json"); + file_checker.setPath(path + "sizes.json"); } @@ -251,10 +252,10 @@ BlockInputStreams StorageStripeLog::read( NameSet column_names_set(column_names.begin(), column_names.end()); - if (!Poco::File(full_path() + "index.mrk").exists()) + if (!Poco::File(fullPath() + "index.mrk").exists()) return { std::make_shared(getSampleBlockForColumns(column_names)) }; - CompressedReadBufferFromFile index_in(full_path() + "index.mrk", 0, 0, INDEX_BUFFER_SIZE); + CompressedReadBufferFromFile index_in(fullPath() + "index.mrk", 0, 0, INDEX_BUFFER_SIZE); std::shared_ptr index{std::make_shared(index_in, column_names_set)}; BlockInputStreams res; @@ -301,11 +302,11 @@ void StorageStripeLog::truncate(const ASTPtr &, const Context &, TableStructureW std::shared_lock lock(rwlock); - auto file = Poco::File(path + escapeForFileName(table_name)); + auto file = Poco::File(path); file.remove(true); file.createDirectories(); - file_checker = FileChecker{path + escapeForFileName(table_name) + '/' + "sizes.json"}; + file_checker = FileChecker{path + "sizes.json"}; } @@ -319,8 +320,8 @@ void registerStorageStripeLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageStripeLog::create( - args.data_path, args.database_name, args.table_name, args.columns, args.constraints, - args.attach, args.context.getSettings().max_compress_block_size); + args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, + args.attach, args.context.getSettings().max_compress_block_size, args.context); }); } diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 80d7195e054..4263ab14946 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -51,9 +51,9 @@ public: }; using Files_t = std::map; - std::string full_path() const { return path + escapeForFileName(table_name) + '/';} + std::string fullPath() const { return path; } - Strings getDataPaths() const override { return {full_path()}; } + Strings getDataPaths() const override { return {fullPath()}; } void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; @@ -71,13 +71,14 @@ private: protected: StorageStripeLog( - const std::string & path_, + const std::string & relative_path_, const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool attach, - size_t max_compress_block_size_); + size_t max_compress_block_size_, + const Context & context_); }; } diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 8ac3dfbff76..e62fab8c2b6 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -323,30 +323,30 @@ void TinyLogBlockOutputStream::write(const Block & block) StorageTinyLog::StorageTinyLog( - const std::string & path_, + const std::string & relative_path_, const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool attach, - size_t max_compress_block_size_) - : path(path_), table_name(table_name_), database_name(database_name_), + size_t max_compress_block_size_, + const Context & context_) + : path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), - file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json"), + file_checker(path + "sizes.json"), log(&Logger::get("StorageTinyLog")) { setColumns(columns_); setConstraints(constraints_); - if (path.empty()) + if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - String full_path = path + escapeForFileName(table_name) + '/'; if (!attach) { /// create files if they do not exist - if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST) - throwFromErrnoWithPath("Cannot create directory " + full_path, full_path, + if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST) + throwFromErrnoWithPath("Cannot create directory " + path, path, ErrorCodes::CANNOT_CREATE_DIRECTORY); } @@ -369,7 +369,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type ColumnData column_data; files.insert(std::make_pair(stream_name, column_data)); files[stream_name].data_file = Poco::File( - path + escapeForFileName(table_name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION); + path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION); } }; @@ -383,15 +383,16 @@ void StorageTinyLog::rename(const String & new_path_to_db, const String & new_da std::unique_lock lock(rwlock); /// Rename directory with data. - Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name)); + String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/"; + Poco::File(path).renameTo(new_path); - path = new_path_to_db; + path = new_path; table_name = new_table_name; database_name = new_database_name; - file_checker.setPath(path + escapeForFileName(table_name) + "/" + "sizes.json"); + file_checker.setPath(path + "sizes.json"); for (Files_t::iterator it = files.begin(); it != files.end(); ++it) - it->second.data_file = Poco::File(path + escapeForFileName(table_name) + '/' + Poco::Path(it->second.data_file.path()).getFileName()); + it->second.data_file = Poco::File(path + Poco::Path(it->second.data_file.path()).getFileName()); } @@ -431,12 +432,12 @@ void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWri std::unique_lock lock(rwlock); - auto file = Poco::File(path + escapeForFileName(table_name)); + auto file = Poco::File(path); file.remove(true); file.createDirectories(); files.clear(); - file_checker = FileChecker{path + escapeForFileName(table_name) + '/' + "sizes.json"}; + file_checker = FileChecker{path + "sizes.json"}; for (const auto &column : getColumns().getAllPhysical()) addFiles(column.name, *column.type); @@ -453,8 +454,8 @@ void registerStorageTinyLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageTinyLog::create( - args.data_path, args.database_name, args.table_name, args.columns, args.constraints, - args.attach, args.context.getSettings().max_compress_block_size); + args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, + args.attach, args.context.getSettings().max_compress_block_size, args.context); }); } diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 173535e24fa..8c33b9ff164 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -50,7 +50,7 @@ public: }; using Files_t = std::map; - std::string fullPath() const { return path + escapeForFileName(table_name) + '/';} + std::string fullPath() const { return path; } Strings getDataPaths() const override { return {fullPath()}; } @@ -75,13 +75,14 @@ private: protected: StorageTinyLog( - const std::string & path_, + const std::string & relative_path_, const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool attach, - size_t max_compress_block_size_); + size_t max_compress_block_size_, + const Context & context_); }; } diff --git a/dbms/src/Storages/System/StorageSystemDatabases.cpp b/dbms/src/Storages/System/StorageSystemDatabases.cpp index 5948f540324..e94eab0d565 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.cpp +++ b/dbms/src/Storages/System/StorageSystemDatabases.cpp @@ -9,6 +9,7 @@ namespace DB NamesAndTypesList StorageSystemDatabases::getNamesAndTypes() { + //FIXME data_path return { {"name", std::make_shared()}, {"engine", std::make_shared()}, From ec7de6d137e424e5d73017f09ca0f9029296aab3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 28 Oct 2019 23:12:14 +0300 Subject: [PATCH 004/743] use arbitrary relative path in *MergeTree --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 7 +++++-- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 ++ dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp | 4 ++-- dbms/src/Storages/StorageMergeTree.cpp | 3 ++- dbms/src/Storages/StorageMergeTree.h | 1 + dbms/src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- dbms/src/Storages/StorageReplicatedMergeTree.h | 1 + 7 files changed, 15 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index af985c02927..8c8e76c3e7e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -99,6 +99,7 @@ namespace ErrorCodes MergeTreeData::MergeTreeData( const String & database_, const String & table_, + const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_, @@ -122,6 +123,7 @@ MergeTreeData::MergeTreeData( , require_part_metadata(require_part_metadata_) , database_name(database_) , table_name(table_) + , relative_data_path(relative_data_path_) , broken_part_callback(broken_part_callback_) , log_name(database_name + "." + table_name) , log(&Logger::get(log_name)) @@ -1195,7 +1197,7 @@ void MergeTreeData::rename( for (const auto & disk : disks) { - auto full_path = disk->getClickHouseDataPath() + old_file_db_name + '/' + old_file_table_name + '/'; + auto full_path = disk->getPath() + relative_data_path; auto new_db_path = disk->getClickHouseDataPath() + new_file_db_name + '/'; Poco::File db_file{new_db_path}; @@ -1210,6 +1212,7 @@ void MergeTreeData::rename( database_name = new_database_name; table_name = new_table_name; + relative_data_path = "data/" + old_file_db_name + '/' + old_file_table_name + '/'; } void MergeTreeData::dropAllData() @@ -3250,7 +3253,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPart(const Merg String MergeTreeData::getFullPathOnDisk(const DiskSpace::DiskPtr & disk) const { - return disk->getClickHouseDataPath() + escapeForFileName(database_name) + '/' + escapeForFileName(table_name) + '/'; + return disk->getPath() + relative_data_path; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index a8bd661fafa..dd33cd472ac 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -332,6 +332,7 @@ public: /// require_part_metadata - should checksums.txt and columns.txt exist in the part directory. /// attach - whether the existing table is attached or the new table is created. MergeTreeData(const String & database_, const String & table_, + const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_, @@ -760,6 +761,7 @@ protected: String database_name; String table_name; + String relative_data_path; /// Current column sizes in compressed and uncompressed form. diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 5458f3f7d9b..633699d1529 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -639,14 +639,14 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) return StorageReplicatedMergeTree::create( - zookeeper_path, replica_name, args.attach, args.database_name, args.table_name, + zookeeper_path, replica_name, args.attach, args.database_name, args.table_name, args.relative_data_path, args.columns, indices_description, args.constraints, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, std::move(storage_settings), args.has_force_restore_data_flag); else return StorageMergeTree::create( - args.database_name, args.table_name, args.columns, indices_description, + args.database_name, args.table_name, args.relative_data_path, args.columns, indices_description, args.constraints, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, std::move(storage_settings), args.has_force_restore_data_flag); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index c752109e328..0a6f162f625 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -54,6 +54,7 @@ namespace ActionLocks StorageMergeTree::StorageMergeTree( const String & database_name_, const String & table_name_, + const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_, @@ -68,7 +69,7 @@ StorageMergeTree::StorageMergeTree( const MergingParams & merging_params_, std::unique_ptr storage_settings_, bool has_force_restore_data_flag) - : MergeTreeData(database_name_, table_name_, + : MergeTreeData(database_name_, table_name_, relative_data_path_, columns_, indices_, constraints_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sample_by_ast_, ttl_table_ast_, merging_params_, diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 15080cfcbf8..6b84e1bea4e 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -147,6 +147,7 @@ protected: StorageMergeTree( const String & database_name_, const String & table_name_, + const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_, diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index c55378d8526..ff85cecad61 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -192,6 +192,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( bool attach, const String & database_name_, const String & table_name_, + const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_, @@ -205,7 +206,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const MergingParams & merging_params_, std::unique_ptr settings_, bool has_force_restore_data_flag) - : MergeTreeData(database_name_, table_name_, + : MergeTreeData(database_name_, table_name_, relative_data_path_, columns_, indices_, constraints_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sample_by_ast_, ttl_table_ast_, merging_params_, diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index facdb1660f0..f5e384f058d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -531,6 +531,7 @@ protected: const String & replica_name_, bool attach, const String & database_name_, const String & name_, + const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_, From a3b4762cdea8106e55b9fef7211b44896f9c07e8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 30 Oct 2019 15:17:52 +0300 Subject: [PATCH 005/743] fixes --- dbms/src/Databases/DatabaseAtomic.cpp | 7 +++++++ dbms/src/Databases/DatabaseAtomic.h | 6 ++++++ dbms/src/Databases/DatabaseOnDisk.cpp | 5 +++-- dbms/src/Databases/DatabaseOnDisk.h | 2 +- dbms/src/Interpreters/loadMetadata.cpp | 5 +++-- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 4 +--- dbms/src/Storages/tests/storage_log.cpp | 7 ++++--- 7 files changed, 25 insertions(+), 11 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index cc67b45f8a1..dfbb6d1057e 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB @@ -9,6 +10,12 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Contex { } +void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, + const String & to_table_name, TableStructureWriteLockHolder & lock) +{ + //TODO + DatabaseOnDisk::renameTable(*this, context, table_name, to_database, to_table_name, lock); +} } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index fa1e3573a8a..9bd1b1184c5 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -15,6 +15,12 @@ public: String getEngineName() const override { return "Atomic"; } + void renameTable(const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name, + TableStructureWriteLockHolder &) override; + }; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 9a281d5d745..f9acc47e5b0 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -126,7 +126,7 @@ ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, - const String & database_data_path, + const String & database_data_path_relative, Context & context, bool has_force_restore_data_flag) { @@ -149,12 +149,13 @@ std::pair createTableFromAST( ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context); ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints); + String table_data_path_relative = database_data_path_relative + escapeForFileName(ast_create_query.table) + '/'; return { ast_create_query.table, StorageFactory::instance().get( ast_create_query, - database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(), + table_data_path_relative, ast_create_query.table, database_name, context, context.getGlobalContext(), columns, constraints, true, has_force_restore_data_flag) }; diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 92a6f48959a..cc7c7ef3f1e 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -24,7 +24,7 @@ ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, - const String & database_data_path, + const String & database_data_path_relative, Context & context, bool has_force_restore_data_flag); diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 00090d1d309..2a1563b7819 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -56,7 +56,7 @@ static void loadDatabase( bool force_restore_data) { /// There may exist .sql file with database creation statement. - /// Or, if it is absent, then database with default engine is created. + /// Or, if it is absent, then database with Ordinary engine is created. String database_attach_query; String database_metadata_file = database_path + ".sql"; @@ -67,7 +67,8 @@ static void loadDatabase( readStringUntilEOF(database_attach_query, in); } else - database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database); + //FIXME + database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Atomic"; executeCreateQuery(database_attach_query, context, database, database_metadata_file, force_restore_data); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 6c9fc5514b1..4d9a5762659 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1180,9 +1180,7 @@ void MergeTreeData::rename( const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { - auto old_file_db_name = escapeForFileName(database_name); auto new_file_db_name = escapeForFileName(new_database_name); - auto old_file_table_name = escapeForFileName(table_name); auto new_file_table_name = escapeForFileName(new_table_name); auto disks = storage_policy->getDisks(); @@ -1212,7 +1210,7 @@ void MergeTreeData::rename( database_name = new_database_name; table_name = new_table_name; - relative_data_path = "data/" + old_file_db_name + '/' + old_file_table_name + '/'; + relative_data_path = "data/" + new_file_db_name + '/' + new_file_table_name + '/'; } void MergeTreeData::dropAllData() diff --git a/dbms/src/Storages/tests/storage_log.cpp b/dbms/src/Storages/tests/storage_log.cpp index 02a845cf61c..f06988504f7 100644 --- a/dbms/src/Storages/tests/storage_log.cpp +++ b/dbms/src/Storages/tests/storage_log.cpp @@ -25,11 +25,12 @@ try names_and_types.emplace_back("a", std::make_shared()); names_and_types.emplace_back("b", std::make_shared()); - StoragePtr table = StorageLog::create("./", "test", "test", ColumnsDescription{names_and_types}, ConstraintsDescription{}, 1048576); - table->startup(); - auto context = Context::createGlobal(); context.makeGlobalContext(); + context.setPath("./"); + + StoragePtr table = StorageLog::create("./", "test", "test", ColumnsDescription{names_and_types}, ConstraintsDescription{}, 1048576, context); + table->startup(); /// write into it { From 9bbeb6359a24d1bc037838d96f215c8edf97b49e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 30 Oct 2019 17:17:55 +0300 Subject: [PATCH 006/743] refactor StorageFile construction --- dbms/src/Storages/StorageFile.cpp | 157 +++++++++--------- dbms/src/Storages/StorageFile.h | 35 ++-- dbms/src/TableFunctions/TableFunctionFile.cpp | 12 +- 3 files changed, 97 insertions(+), 107 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index c27fb3b9b9a..63742535fab 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -26,7 +26,6 @@ #include #include -#include #include namespace fs = std::filesystem; @@ -118,63 +117,52 @@ static void checkCreationIsAllowed(Context & context_global, const std::string & } } -StorageFile::StorageFile( - const std::string & table_path_, - int table_fd_, - const std::string & relative_table_dir_path, - const std::string & database_name_, - const std::string & table_name_, - const std::string & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - Context & context_) - : - table_name(table_name_), database_name(database_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_) +StorageFile::StorageFile(int table_fd_, CommonArguments args) + : StorageFile(args) { - setColumns(columns_); - setConstraints(constraints_); + if (context_global.getApplicationType() == Context::ApplicationType::SERVER) + throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED); - if (table_fd < 0) /// Will use file - { - String table_dir_path = context_global.getPath() + relative_table_dir_path + "/"; - use_table_fd = false; + is_db_table = false; + use_table_fd = true; + table_fd = table_fd_; - if (!table_path_.empty()) /// Is user's file - { - Poco::Path poco_path = Poco::Path(table_path_); - if (poco_path.isRelative()) - poco_path = Poco::Path(table_dir_path, poco_path); - - const std::string path = poco_path.absolute().toString(); - paths = listFilesWithRegexpMatching("/", path); - for (const auto & cur_path : paths) - checkCreationIsAllowed(context_global, table_dir_path, cur_path); - is_db_table = false; - } - else /// Is DB's file - { - if (relative_table_dir_path.empty()) - throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - - paths = {getTablePath(table_dir_path, format_name)}; - is_db_table = true; - Poco::File(Poco::Path(paths.back()).parent()).createDirectories(); - } - } - else /// Will use FD - { - if (context_global.getApplicationType() == Context::ApplicationType::SERVER) - throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED); - - is_db_table = false; - use_table_fd = true; - - /// Save initial offset, it will be used for repeating SELECTs - /// If FD isn't seekable (lseek returns -1), then the second and subsequent SELECTs will fail. - table_fd_init_offset = lseek(table_fd, 0, SEEK_CUR); - } + /// Save initial offset, it will be used for repeating SELECTs + /// If FD isn't seekable (lseek returns -1), then the second and subsequent SELECTs will fail. + table_fd_init_offset = lseek(table_fd, 0, SEEK_CUR); } +StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_absolute_path, CommonArguments args) + : StorageFile(args) +{ + is_db_table = false; + Poco::Path poco_path = Poco::Path(table_path_); + if (poco_path.isRelative()) + poco_path = Poco::Path(user_files_absolute_path, poco_path); + + const std::string path = poco_path.absolute().toString(); + paths = listFilesWithRegexpMatching("/", path); + for (const auto & cur_path : paths) + checkCreationIsAllowed(context_global, user_files_absolute_path, cur_path); +} + +StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args) + : StorageFile(args) +{ + if (relative_table_dir_path.empty()) + throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); + + String table_dir_path = context_global.getPath() + relative_table_dir_path + "/"; + Poco::File(table_dir_path).createDirectories(); + paths = {getTablePath(table_dir_path, format_name)}; +} + +StorageFile::StorageFile(CommonArguments args) + : table_name(args.table_name), database_name(args.database_name), format_name(args.format_name), context_global(args.context) +{ + setColumns(args.columns); + setConstraints(args.constraints); +} class StorageFileBlockInputStream : public IBlockInputStream { @@ -369,41 +357,44 @@ void registerStorageFile(StorageFactory & factory) engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); String format_name = engine_args[0]->as().value.safeGet(); + StorageFile::CommonArguments common_args{args.database_name, args.table_name, format_name, + args.columns, args.constraints,args.context}; + + if (engine_args.size() == 1) /// Table in database + return StorageFile::create(args.relative_data_path, common_args); + + /// Will use FD if engine_args[1] is int literal or identifier with std* name int source_fd = -1; String source_path; - if (engine_args.size() >= 2) + if (auto opt_name = tryGetIdentifierName(engine_args[1])) { - /// Will use FD if engine_args[1] is int literal or identifier with std* name - - if (auto opt_name = tryGetIdentifierName(engine_args[1])) - { - if (*opt_name == "stdin") - source_fd = STDIN_FILENO; - else if (*opt_name == "stdout") - source_fd = STDOUT_FILENO; - else if (*opt_name == "stderr") - source_fd = STDERR_FILENO; - else - throw Exception("Unknown identifier '" + *opt_name + "' in second arg of File storage constructor", - ErrorCodes::UNKNOWN_IDENTIFIER); - } - else if (const auto * literal = engine_args[1]->as()) - { - auto type = literal->value.getType(); - if (type == Field::Types::Int64) - source_fd = static_cast(literal->value.get()); - else if (type == Field::Types::UInt64) - source_fd = static_cast(literal->value.get()); - else if (type == Field::Types::String) - source_path = literal->value.get(); - } + if (*opt_name == "stdin") + source_fd = STDIN_FILENO; + else if (*opt_name == "stdout") + source_fd = STDOUT_FILENO; + else if (*opt_name == "stderr") + source_fd = STDERR_FILENO; + else + throw Exception("Unknown identifier '" + *opt_name + "' in second arg of File storage constructor", + ErrorCodes::UNKNOWN_IDENTIFIER); + } + else if (const auto * literal = engine_args[1]->as()) + { + auto type = literal->value.getType(); + if (type == Field::Types::Int64) + source_fd = static_cast(literal->value.get()); + else if (type == Field::Types::UInt64) + source_fd = static_cast(literal->value.get()); + else if (type == Field::Types::String) + source_path = literal->value.get(); + else + throw Exception("Second argument must be path or file descriptor", ErrorCodes::BAD_ARGUMENTS); } - return StorageFile::create( - source_path, source_fd, - args.relative_data_path, - args.database_name, args.table_name, format_name, args.columns, args.constraints, - args.context); + if (0 <= source_fd) /// File descriptor + return StorageFile::create(source_fd, common_args); + else /// User's file + return StorageFile::create(source_path, args.context.getUserFilesPath(), common_args); }); } diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 019fb96c75b..e5fd8a40094 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -42,27 +42,32 @@ public: Strings getDataPaths() const override; + struct CommonArguments + { + const std::string & database_name; + const std::string & table_name; + const std::string & format_name; + const ColumnsDescription & columns; + const ConstraintsDescription & constraints; + Context & context; + }; + protected: friend class StorageFileBlockInputStream; friend class StorageFileBlockOutputStream; - /** there are three options (ordered by priority): - - use specified file descriptor if (fd >= 0) - - use specified table_path if it isn't empty - - create own table inside data/db/table/ - */ - StorageFile( - const std::string & table_path_, - int table_fd_, - const std::string & relative_table_dir_path, - const std::string & database_name_, - const std::string & table_name_, - const std::string & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - Context & context_); + /// From file descriptor + StorageFile(int table_fd_, CommonArguments args); + + /// From user's file + StorageFile(const std::string & table_path_, const std::string & user_files_absolute_path, CommonArguments args); + + /// From table in database + StorageFile(const std::string & relative_table_dir_path, CommonArguments args); private: + explicit StorageFile(CommonArguments args); + std::string table_name; std::string database_name; std::string format_name; diff --git a/dbms/src/TableFunctions/TableFunctionFile.cpp b/dbms/src/TableFunctions/TableFunctionFile.cpp index 7cf2c500f1e..eaa2d8f27cb 100644 --- a/dbms/src/TableFunctions/TableFunctionFile.cpp +++ b/dbms/src/TableFunctions/TableFunctionFile.cpp @@ -8,15 +8,9 @@ namespace DB StoragePtr TableFunctionFile::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const { - return StorageFile::create(source, - -1, - global_context.getUserFilesPath(), - getDatabaseName(), - table_name, - format, - columns, - ConstraintsDescription{}, - global_context); + StorageFile::CommonArguments args{getDatabaseName(), table_name, format, columns,ConstraintsDescription{},global_context}; + + return StorageFile::create(source, global_context.getUserFilesPath(), args); } void registerTableFunctionFile(TableFunctionFactory & factory) From d0af0c8703bb26113355cb572ba74acd94213139 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 1 Nov 2019 15:47:55 +0300 Subject: [PATCH 007/743] refactor IDatabase --- dbms/src/Databases/DatabaseDictionary.cpp | 115 +-------------------- dbms/src/Databases/DatabaseDictionary.h | 55 +--------- dbms/src/Databases/DatabaseLazy.cpp | 78 ++------------ dbms/src/Databases/DatabaseLazy.h | 40 +------- dbms/src/Databases/DatabaseMemory.cpp | 67 +----------- dbms/src/Databases/DatabaseMemory.h | 34 +------ dbms/src/Databases/DatabaseMySQL.cpp | 15 ++- dbms/src/Databases/DatabaseMySQL.h | 63 +----------- dbms/src/Databases/DatabaseOnDisk.cpp | 24 +---- dbms/src/Databases/DatabaseOnDisk.h | 23 +---- dbms/src/Databases/DatabaseOrdinary.cpp | 36 ++----- dbms/src/Databases/DatabaseOrdinary.h | 24 ++--- dbms/src/Databases/DatabasesCommon.cpp | 8 +- dbms/src/Databases/DatabasesCommon.h | 4 +- dbms/src/Databases/IDatabase.h | 119 ++++++++++++++++------ 15 files changed, 150 insertions(+), 555 deletions(-) diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index 52dde4efe9b..cea3ea3617d 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -22,13 +22,9 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } -DatabaseDictionary::DatabaseDictionary(const String & name_) - : name(name_), - log(&Logger::get("DatabaseDictionary(" + name + ")")) -{ -} - -void DatabaseDictionary::loadStoredObjects(Context &, bool) +DatabaseDictionary::DatabaseDictionary(String name_) + : IDatabase(std::move(name_)), + log(&Logger::get("DatabaseDictionary(" + database_name + ")")) { } @@ -69,15 +65,6 @@ bool DatabaseDictionary::isTableExist( return context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST; } - -bool DatabaseDictionary::isDictionaryExist( - const Context & /*context*/, - const String & /*table_name*/) const -{ - return false; -} - - DatabaseDictionariesIteratorPtr DatabaseDictionary::getDictionariesIterator( const Context & /*context*/, const FilterByNameFunction & /*filter_by_dictionary_name*/) @@ -85,50 +72,6 @@ DatabaseDictionariesIteratorPtr DatabaseDictionary::getDictionariesIterator( return std::make_unique(); } - -void DatabaseDictionary::createDictionary( - const Context & /*context*/, - const String & /*dictionary_name*/, - const ASTPtr & /*query*/) -{ - throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD); -} - -void DatabaseDictionary::removeDictionary( - const Context & /*context*/, - const String & /*table_name*/) -{ - throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD); -} - -void DatabaseDictionary::attachDictionary( - const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/) -{ - throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD); -} - -void DatabaseDictionary::detachDictionary( - const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/) -{ - throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD); -} - - -ASTPtr DatabaseDictionary::tryGetCreateDictionaryQuery( - const Context & /*context*/, - const String & /*table_name*/) const -{ - return nullptr; -} - - -ASTPtr DatabaseDictionary::getCreateDictionaryQuery( - const Context & /*context*/, - const String & /*table_name*/) const -{ - throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD); -} - StoragePtr DatabaseDictionary::tryGetTable( const Context & context, const String & table_name) const @@ -154,39 +97,6 @@ bool DatabaseDictionary::empty(const Context & context) const return !context.getExternalDictionariesLoader().hasCurrentlyLoadedObjects(); } -StoragePtr DatabaseDictionary::detachTable(const String & /*table_name*/) -{ - throw Exception("DatabaseDictionary: detachTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - -void DatabaseDictionary::attachTable(const String & /*table_name*/, const StoragePtr & /*table*/) -{ - throw Exception("DatabaseDictionary: attachTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - -void DatabaseDictionary::createTable( - const Context &, - const String &, - const StoragePtr &, - const ASTPtr &) -{ - throw Exception("DatabaseDictionary: createTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - -void DatabaseDictionary::removeTable( - const Context &, - const String &) -{ - throw Exception("DatabaseDictionary: removeTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - -time_t DatabaseDictionary::getObjectMetadataModificationTime( - const Context &, - const String &) -{ - return static_cast(0); -} - ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const { @@ -199,7 +109,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context, : dictionaries.tryGetDictionary(table_name); auto names_and_types = StorageDictionary::getNamesAndTypes(dictionary->getStructure()); - buffer << "CREATE TABLE " << backQuoteIfNeed(name) << '.' << backQuoteIfNeed(table_name) << " ("; + buffer << "CREATE TABLE " << backQuoteIfNeed(database_name) << '.' << backQuoteIfNeed(table_name) << " ("; buffer << StorageDictionary::generateNamesAndTypesDescription(names_and_types.begin(), names_and_types.end()); buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")"; } @@ -216,22 +126,12 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context, return ast; } -ASTPtr DatabaseDictionary::getCreateTableQuery(const Context & context, const String & table_name) const -{ - return getCreateTableQueryImpl(context, table_name, true); -} - -ASTPtr DatabaseDictionary::tryGetCreateTableQuery(const Context & context, const String & table_name) const -{ - return getCreateTableQueryImpl(context, table_name, false); -} - ASTPtr DatabaseDictionary::getCreateDatabaseQuery(const Context & /*context*/) const { String query; { WriteBufferFromString buffer(query); - buffer << "CREATE DATABASE " << backQuoteIfNeed(name) << " ENGINE = Dictionary"; + buffer << "CREATE DATABASE " << backQuoteIfNeed(database_name) << " ENGINE = Dictionary"; } ParserCreateQuery parser; return parseQuery(parser, query.data(), query.data() + query.size(), "", 0); @@ -241,9 +141,4 @@ void DatabaseDictionary::shutdown() { } -String DatabaseDictionary::getDatabaseName() const -{ - return name; -} - } diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index 9e88b50a68e..5a5c21f1720 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -22,25 +22,17 @@ namespace DB class DatabaseDictionary : public IDatabase { public: - DatabaseDictionary(const String & name_); - - String getDatabaseName() const override; + DatabaseDictionary(String name_); String getEngineName() const override { return "Dictionary"; } - void loadStoredObjects( - Context & context, - bool has_force_restore_data_flag) override; - bool isTableExist( const Context & context, const String & table_name) const override; - bool isDictionaryExist(const Context & context, const String & table_name) const override; - StoragePtr tryGetTable( const Context & context, const String & table_name) const override; @@ -51,58 +43,19 @@ public: bool empty(const Context & context) const override; - void createTable( - const Context & context, - const String & table_name, - const StoragePtr & table, - const ASTPtr & query) override; - - void createDictionary( - const Context & context, const String & dictionary_name, const ASTPtr & query) override; - - void removeTable( - const Context & context, - const String & table_name) override; - - void removeDictionary(const Context & context, const String & table_name) override; - - void attachTable(const String & table_name, const StoragePtr & table) override; - - StoragePtr detachTable(const String & table_name) override; - - time_t getObjectMetadataModificationTime( - const Context & context, - const String & table_name) override; - - ASTPtr getCreateTableQuery( - const Context & context, - const String & table_name) const override; - - ASTPtr tryGetCreateTableQuery( - const Context & context, - const String & table_name) const override; - ASTPtr getCreateDatabaseQuery(const Context & context) const override; - ASTPtr getCreateDictionaryQuery(const Context & context, const String & table_name) const override; - - ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & table_name) const override; - - - void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override; - - void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override; - void shutdown() override; +protected: + ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const override; + private: - const String name; mutable std::mutex mutex; Poco::Logger * log; Tables listTables(const Context & context, const FilterByNameFunction & filter_by_name); - ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const; }; } diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 98699d572aa..20d25b12676 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -29,12 +29,12 @@ namespace ErrorCodes -DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_) - : name(name_) +DatabaseLazy::DatabaseLazy(String name_, const String & metadata_path_, time_t expiration_time_, const Context & context_) + : IDatabase(std::move(name_)) , metadata_path(metadata_path_) - , data_path("data/" + escapeForFileName(name) + "/") + , data_path("data/" + escapeForFileName(database_name) + "/") , expiration_time(expiration_time_) - , log(&Logger::get("DatabaseLazy (" + name + ")")) + , log(&Logger::get("DatabaseLazy (" + database_name + ")")) { Poco::File(context_.getPath() + getDataPath()).createDirectories(); } @@ -70,16 +70,6 @@ void DatabaseLazy::createTable( it->second.metadata_modification_time = DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name); } - -void DatabaseLazy::createDictionary( - const Context & /*context*/, - const String & /*dictionary_name*/, - const ASTPtr & /*query*/) -{ - throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD); -} - - void DatabaseLazy::removeTable( const Context & context, const String & table_name) @@ -88,31 +78,6 @@ void DatabaseLazy::removeTable( DatabaseOnDisk::removeTable(*this, context, table_name, log); } -void DatabaseLazy::removeDictionary( - const Context & /*context*/, - const String & /*table_name*/) -{ - throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD); -} - -ASTPtr DatabaseLazy::getCreateDictionaryQuery( - const Context & /*context*/, - const String & /*table_name*/) const -{ - throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD); -} - -ASTPtr DatabaseLazy::tryGetCreateDictionaryQuery(const Context & /*context*/, const String & /*table_name*/) const -{ - return nullptr; -} - -bool DatabaseLazy::isDictionaryExist(const Context & /*context*/, const String & /*table_name*/) const -{ - return false; -} - - DatabaseDictionariesIteratorPtr DatabaseLazy::getDictionariesIterator( const Context & /*context*/, const FilterByNameFunction & /*filter_by_dictionary_name*/) @@ -120,19 +85,6 @@ DatabaseDictionariesIteratorPtr DatabaseLazy::getDictionariesIterator( return std::make_unique(); } -void DatabaseLazy::attachDictionary( - const String & /*dictionary_name*/, - const Context & /*context*/, - bool /*load*/) -{ - throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD); -} - -void DatabaseLazy::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/) -{ - throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD); -} - void DatabaseLazy::renameTable( const Context & context, const String & table_name, @@ -157,14 +109,9 @@ time_t DatabaseLazy::getObjectMetadataModificationTime( throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); } -ASTPtr DatabaseLazy::getCreateTableQuery(const Context & context, const String & table_name) const +ASTPtr DatabaseLazy::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const { - return DatabaseOnDisk::getCreateTableQuery(*this, context, table_name); -} - -ASTPtr DatabaseLazy::tryGetCreateTableQuery(const Context & context, const String & table_name) const -{ - return DatabaseOnDisk::tryGetCreateTableQuery(*this, context, table_name); + return DatabaseOnDisk::getCreateTableQueryImpl(*this, context, table_name, throw_on_error); } ASTPtr DatabaseLazy::getCreateDatabaseQuery(const Context & context) const @@ -313,11 +260,6 @@ String DatabaseLazy::getMetadataPath() const return metadata_path; } -String DatabaseLazy::getDatabaseName() const -{ - return name; -} - String DatabaseLazy::getObjectMetadataPath(const String & table_name) const { return DatabaseOnDisk::getObjectMetadataPath(*this, table_name); @@ -333,14 +275,16 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table try { - String table_name_; StoragePtr table; Context context_copy(context); /// some tables can change context, but not LogTables auto ast = parseCreateQueryFromMetadataFile(table_metadata_path, log); if (ast) - std::tie(table_name_, table) = createTableFromAST( - ast->as(), name, getDataPath(), context_copy, false); + { + auto & ast_create = ast->as(); + String table_data_path_relative = getDataPath() + escapeForFileName(ast_create.table) + '/'; + table = createTableFromAST(ast_create, database_name, table_data_path_relative, context_copy, false).second; + } if (!ast || !endsWith(table->getName(), "Log")) throw Exception("Only *Log tables can be used with Lazy database engine.", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index 68e29b36d10..aa902d7db44 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -18,7 +18,7 @@ class DatabaseLazyIterator; class DatabaseLazy : public IDatabase { public: - DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_); + DatabaseLazy(String name_, const String & metadata_path_, time_t expiration_time_, const Context & context_); String getEngineName() const override { return "Lazy"; } @@ -32,19 +32,10 @@ public: const StoragePtr & table, const ASTPtr & query) override; - void createDictionary( - const Context & context, - const String & dictionary_name, - const ASTPtr & query) override; - void removeTable( const Context & context, const String & table_name) override; - void removeDictionary( - const Context & context, - const String & table_name) override; - void renameTable( const Context & context, const String & table_name, @@ -64,26 +55,9 @@ public: const Context & context, const String & table_name) override; - ASTPtr getCreateTableQuery( - const Context & context, - const String & table_name) const override; - - ASTPtr tryGetCreateTableQuery( - const Context & context, - const String & table_name) const override; - - ASTPtr getCreateDictionaryQuery( - const Context & context, - const String & dictionary_name) const override; - - ASTPtr tryGetCreateDictionaryQuery( - const Context & context, - const String & dictionary_name) const override; - ASTPtr getCreateDatabaseQuery(const Context & context) const override; String getDataPath() const override; - String getDatabaseName() const override; String getMetadataPath() const override; String getObjectMetadataPath(const String & table_name) const override; @@ -93,10 +67,6 @@ public: const Context & context, const String & table_name) const override; - bool isDictionaryExist( - const Context & context, - const String & table_name) const override; - StoragePtr tryGetTable( const Context & context, const String & table_name) const override; @@ -111,14 +81,13 @@ public: StoragePtr detachTable(const String & table_name) override; - void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override; - - void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override; - void shutdown() override; ~DatabaseLazy() override; +protected: + ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const override; + private: struct CacheExpirationQueueElement { @@ -147,7 +116,6 @@ private: using TablesCache = std::unordered_map; - String name; const String metadata_path; const String data_path; diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index 0badc9b4df4..20db4d29a63 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -15,16 +15,9 @@ namespace ErrorCodes DatabaseMemory::DatabaseMemory(String name_) : DatabaseWithOwnTablesBase(std::move(name_)) - , log(&Logger::get("DatabaseMemory(" + name + ")")) + , log(&Logger::get("DatabaseMemory(" + database_name + ")")) {} -void DatabaseMemory::loadStoredObjects( - Context & /*context*/, - bool /*has_force_restore_data_flag*/) -{ - /// Nothing to load. -} - void DatabaseMemory::createTable( const Context & /*context*/, const String & table_name, @@ -34,21 +27,6 @@ void DatabaseMemory::createTable( attachTable(table_name, table); } - -void DatabaseMemory::attachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/) -{ - throw Exception("There is no ATTACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD); -} - -void DatabaseMemory::createDictionary( - const Context & /*context*/, - const String & /*dictionary_name*/, - const ASTPtr & /*query*/) -{ - throw Exception("There is no CREATE DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD); -} - - void DatabaseMemory::removeTable( const Context & /*context*/, const String & table_name) @@ -56,52 +34,11 @@ void DatabaseMemory::removeTable( detachTable(table_name); } - -void DatabaseMemory::detachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/) -{ - throw Exception("There is no DETACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD); -} - - -void DatabaseMemory::removeDictionary( - const Context & /*context*/, - const String & /*dictionary_name*/) -{ - throw Exception("There is no DROP DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD); -} - - -time_t DatabaseMemory::getObjectMetadataModificationTime( - const Context &, const String &) -{ - return static_cast(0); -} - -ASTPtr DatabaseMemory::getCreateTableQuery( - const Context &, - const String &) const -{ - throw Exception("There is no CREATE TABLE query for DatabaseMemory tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY); -} - - -ASTPtr DatabaseMemory::getCreateDictionaryQuery( - const Context &, - const String &) const -{ - throw Exception("There is no CREATE DICTIONARY query for DatabaseMemory dictionaries", ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY); -} - - ASTPtr DatabaseMemory::getCreateDatabaseQuery( const Context &) const { + //FIXME throw Exception("There is no CREATE DATABASE query for DatabaseMemory", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY); } -String DatabaseMemory::getDatabaseName() const -{ - return name; -} - } diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 45f51a177f7..536c8de44f0 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -19,52 +19,22 @@ class DatabaseMemory : public DatabaseWithOwnTablesBase public: DatabaseMemory(String name_); - String getDatabaseName() const override; - String getEngineName() const override { return "Memory"; } - void loadStoredObjects( - Context & context, - bool has_force_restore_data_flag) override; - void createTable( const Context & context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override; - void createDictionary( - const Context & context, - const String & dictionary_name, - const ASTPtr & query) override; - - void attachDictionary( - const String & name, - const Context & context, - bool reload) override; - void removeTable( const Context & context, const String & table_name) override; - void removeDictionary( - const Context & context, - const String & dictionary_name) override; - - void detachDictionary( - const String & name, - const Context & context, - bool reload) override; - - time_t getObjectMetadataModificationTime(const Context & context, const String & table_name) override; - - ASTPtr getCreateTableQuery(const Context & context, const String & table_name) const override; - ASTPtr getCreateDictionaryQuery(const Context & context, const String & table_name) const override; - ASTPtr tryGetCreateTableQuery(const Context &, const String &) const override { return nullptr; } - ASTPtr tryGetCreateDictionaryQuery(const Context &, const String &) const override { return nullptr; } - ASTPtr getCreateDatabaseQuery(const Context & context) const override; + //FIXME isDictionaryExist(...) + private: Poco::Logger * log; }; diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index 8e0e092b1d8..6eea60b6d1c 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -49,7 +49,7 @@ String toQueryStringWithQuote(const std::vector & quote_list) DatabaseMySQL::DatabaseMySQL( const Context & context_, const String & database_name_, const String & mysql_host_name_, const UInt16 & mysql_port_, const String & mysql_database_name_, const String & mysql_user_name_, const String & mysql_user_password_) - : global_context(context_), database_name(database_name_), mysql_host_name(mysql_host_name_), mysql_port(mysql_port_), + : IDatabase(database_name_), global_context(context_), mysql_host_name(mysql_host_name_), mysql_port(mysql_port_), mysql_database_name(mysql_database_name_), mysql_user_name(mysql_user_name_), mysql_user_password(mysql_user_password_), mysql_pool(mysql_database_name, mysql_host_name, mysql_user_name, mysql_user_password, mysql_port) { @@ -95,14 +95,19 @@ StoragePtr DatabaseMySQL::tryGetTable(const Context &, const String & mysql_tabl return StoragePtr{}; } -ASTPtr DatabaseMySQL::tryGetCreateTableQuery(const Context &, const String & table_name) const +ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const Context &, const String & table_name, bool throw_on_error) const { std::lock_guard lock(mutex); fetchTablesIntoLocalCache(); if (local_tables_cache.find(table_name) == local_tables_cache.end()) - throw Exception("MySQL table " + mysql_database_name + "." + table_name + " doesn't exist..", ErrorCodes::UNKNOWN_TABLE); + { + if (throw_on_error) + throw Exception("MySQL table " + mysql_database_name + "." + table_name + " doesn't exist..", + ErrorCodes::UNKNOWN_TABLE); + return nullptr; + } return local_tables_cache[table_name].create_table_query; } @@ -230,8 +235,8 @@ DatabaseMySQL::MySQLStorageInfo DatabaseMySQL::createStorageInfo( const String & table_name, const NamesAndTypesList & columns_name_and_type, const UInt64 & table_modification_time) const { const auto & mysql_table = StorageMySQL::create( - database_name, table_name, std::move(mysql_pool), mysql_database_name, table_name, - false, "", ColumnsDescription{columns_name_and_type}, ConstraintsDescription{}, global_context); + database_name, table_name, std::move(mysql_pool), mysql_database_name, table_name, + false, "", ColumnsDescription{columns_name_and_type}, ConstraintsDescription{}, global_context); const auto & create_table_query = std::make_shared(); diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index 3ca8722b248..413a42d5202 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -26,8 +26,6 @@ public: String getEngineName() const override { return "MySQL"; } - String getDatabaseName() const override { return database_name; } - bool empty(const Context & context) const override; DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override; @@ -41,70 +39,14 @@ public: bool isTableExist(const Context & context, const String & name) const override; - bool isDictionaryExist(const Context &, const String &) const override { return false; } - StoragePtr tryGetTable(const Context & context, const String & name) const override; - ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const override; - - ASTPtr getCreateDictionaryQuery(const Context &, const String &) const override - { - throw Exception("MySQL database engine does not support dictionaries.", ErrorCodes::NOT_IMPLEMENTED); - } - - ASTPtr tryGetCreateDictionaryQuery(const Context &, const String &) const override { return nullptr; } - - time_t getObjectMetadataModificationTime(const Context & context, const String & name) override; void shutdown() override; - StoragePtr detachTable(const String &) override - { - throw Exception("MySQL database engine does not support detach table.", ErrorCodes::NOT_IMPLEMENTED); - } - - void detachDictionary(const String &, const Context &, bool) override - { - throw Exception("MySQL database engine does not support detach dictionary.", ErrorCodes::NOT_IMPLEMENTED); - } - - void loadStoredObjects(Context &, bool) override - { - /// do nothing - } - - void removeTable(const Context &, const String &) override - { - throw Exception("MySQL database engine does not support remove table.", ErrorCodes::NOT_IMPLEMENTED); - } - - void removeDictionary(const Context &, const String &) override - { - throw Exception("MySQL database engine does not support remove dictionary.", ErrorCodes::NOT_IMPLEMENTED); - } - - - void attachTable(const String &, const StoragePtr &) override - { - throw Exception("MySQL database engine does not support attach table.", ErrorCodes::NOT_IMPLEMENTED); - } - - void attachDictionary(const String &, const Context &, bool) override - { - throw Exception("MySQL database engine does not support attach dictionary.", ErrorCodes::NOT_IMPLEMENTED); - } - - void createTable(const Context &, const String &, const StoragePtr &, const ASTPtr &) override - { - throw Exception("MySQL database engine does not support create table.", ErrorCodes::NOT_IMPLEMENTED); - } - - void createDictionary(const Context &, const String &, const ASTPtr &) override - { - throw Exception("MySQL database engine does not support create dictionary.", ErrorCodes::NOT_IMPLEMENTED); - } - +protected: + ASTPtr getCreateTableQueryImpl(const Context & context, const String & name, bool throw_on_error) const override; private: struct MySQLStorageInfo @@ -115,7 +57,6 @@ private: }; const Context global_context; - const String database_name; const String mysql_host_name; const UInt16 mysql_port; const String mysql_database_name; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index f9acc47e5b0..f157285c987 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -126,7 +126,7 @@ ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, - const String & database_data_path_relative, + const String & table_data_path_relative, Context & context, bool has_force_restore_data_flag) { @@ -149,7 +149,6 @@ std::pair createTableFromAST( ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context); ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints); - String table_data_path_relative = database_data_path_relative + escapeForFileName(ast_create_query.table) + '/'; return { ast_create_query.table, @@ -425,27 +424,6 @@ ASTPtr DatabaseOnDisk::getCreateDictionaryQueryImpl( return ast; } -ASTPtr DatabaseOnDisk::getCreateTableQuery(const IDatabase & database, const Context & context, const String & table_name) -{ - return getCreateTableQueryImpl(database, context, table_name, true); -} - -ASTPtr DatabaseOnDisk::tryGetCreateTableQuery(const IDatabase & database, const Context & context, const String & table_name) -{ - return getCreateTableQueryImpl(database, context, table_name, false); -} - - -ASTPtr DatabaseOnDisk::getCreateDictionaryQuery(const IDatabase & database, const Context & context, const String & dictionary_name) -{ - return getCreateDictionaryQueryImpl(database, context, dictionary_name, true); -} - -ASTPtr DatabaseOnDisk::tryGetCreateDictionaryQuery(const IDatabase & database, const Context & context, const String & dictionary_name) -{ - return getCreateDictionaryQueryImpl(database, context, dictionary_name, false); -} - ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const IDatabase & database, const Context & /*context*/) { ASTPtr ast; diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index cc7c7ef3f1e..d2aa3d48be7 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -24,7 +24,7 @@ ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, - const String & database_data_path_relative, + const String & table_data_path_relative, Context & context, bool has_force_restore_data_flag); @@ -74,26 +74,6 @@ public: const String & to_table_name, TableStructureWriteLockHolder & lock); - static ASTPtr getCreateTableQuery( - const IDatabase & database, - const Context & context, - const String & table_name); - - static ASTPtr tryGetCreateTableQuery( - const IDatabase & database, - const Context & context, - const String & table_name); - - static ASTPtr getCreateDictionaryQuery( - const IDatabase & database, - const Context & context, - const String & dictionary_name); - - static ASTPtr tryGetCreateDictionaryQuery( - const IDatabase & database, - const Context & context, - const String & dictionary_name); - static ASTPtr getCreateDatabaseQuery( const IDatabase & database, const Context & context); @@ -112,7 +92,6 @@ public: using IteratingFunction = std::function; static void iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function); -private: static ASTPtr getCreateTableQueryImpl( const IDatabase & database, const Context & context, diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index a676f8fb0c4..bd99a2553ed 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -54,7 +54,7 @@ void loadObject( Context & context, const ASTCreateQuery & query, DatabaseOrdinary & database, - const String database_data_path, + const String & database_data_path_relative, const String & database_name, bool has_force_restore_data_flag) try @@ -66,10 +66,8 @@ try } else { - String table_name; - StoragePtr table; - std::tie(table_name, table) - = createTableFromAST(query, database_name, database_data_path, context, has_force_restore_data_flag); + String table_data_path_relative = database_data_path_relative + escapeForFileName(query.table) + '/'; + auto [table_name, table] = createTableFromAST(query, database_name, table_data_path_relative, context, has_force_restore_data_flag); database.attachTable(table_name, table); } } @@ -97,8 +95,8 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context_) : DatabaseWithOwnTablesBase(std::move(name_)) , metadata_path(metadata_path_) - , data_path("data/" + escapeForFileName(name) + "/") - , log(&Logger::get("DatabaseOrdinary (" + name + ")")) + , data_path("data/" + escapeForFileName(database_name) + "/") + , log(&Logger::get("DatabaseOrdinary (" + database_name + ")")) { Poco::File(context_.getPath() + getDataPath()).createDirectories(); } @@ -257,25 +255,14 @@ time_t DatabaseOrdinary::getObjectMetadataModificationTime( return DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name); } -ASTPtr DatabaseOrdinary::getCreateTableQuery(const Context & context, const String & table_name) const +ASTPtr DatabaseOrdinary::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const { - return DatabaseOnDisk::getCreateTableQuery(*this, context, table_name); + return DatabaseOnDisk::getCreateTableQueryImpl(*this, context, table_name, throw_on_error); } -ASTPtr DatabaseOrdinary::tryGetCreateTableQuery(const Context & context, const String & table_name) const +ASTPtr DatabaseOrdinary::getCreateDictionaryQueryImpl(const Context & context, const String & dictionary_name, bool throw_on_error) const { - return DatabaseOnDisk::tryGetCreateTableQuery(*this, context, table_name); -} - - -ASTPtr DatabaseOrdinary::getCreateDictionaryQuery(const Context & context, const String & dictionary_name) const -{ - return DatabaseOnDisk::getCreateDictionaryQuery(*this, context, dictionary_name); -} - -ASTPtr DatabaseOrdinary::tryGetCreateDictionaryQuery(const Context & context, const String & dictionary_name) const -{ - return DatabaseOnDisk::tryGetCreateTableQuery(*this, context, dictionary_name); + return DatabaseOnDisk::getCreateDictionaryQueryImpl(*this, context, dictionary_name, throw_on_error); } ASTPtr DatabaseOrdinary::getCreateDatabaseQuery(const Context & context) const @@ -360,11 +347,6 @@ String DatabaseOrdinary::getMetadataPath() const return metadata_path; } -String DatabaseOrdinary::getDatabaseName() const -{ - return name; -} - String DatabaseOrdinary::getObjectMetadataPath(const String & table_name) const { return DatabaseOnDisk::getObjectMetadataPath(*this, table_name); diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index 7809d63caba..c51950e12e8 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -60,31 +60,21 @@ public: const Context & context, const String & table_name) override; - ASTPtr getCreateTableQuery( - const Context & context, - const String & table_name) const override; - - ASTPtr tryGetCreateTableQuery( - const Context & context, - const String & table_name) const override; - - ASTPtr tryGetCreateDictionaryQuery( - const Context & context, - const String & name) const override; - - ASTPtr getCreateDictionaryQuery( - const Context & context, - const String & name) const override; - ASTPtr getCreateDatabaseQuery(const Context & context) const override; String getDataPath() const override; - String getDatabaseName() const override; String getMetadataPath() const override; String getObjectMetadataPath(const String & table_name) const override; void drop(const Context & context) override; +protected: + ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, + bool throw_on_error) const override; + + ASTPtr getCreateDictionaryQueryImpl(const Context & context, const String & name, + bool throw_on_error) const override; + private: const String metadata_path; const String data_path; diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 399c5c5700f..8425c36b417 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -142,11 +142,11 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name) { std::lock_guard lock(mutex); if (dictionaries.count(table_name)) - throw Exception("Cannot detach dictionary " + name + "." + table_name + " as table, use DETACH DICTIONARY query.", ErrorCodes::UNKNOWN_TABLE); + throw Exception("Cannot detach dictionary " + database_name + "." + table_name + " as table, use DETACH DICTIONARY query.", ErrorCodes::UNKNOWN_TABLE); auto it = tables.find(table_name); if (it == tables.end()) - throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + throw Exception("Table " + database_name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); res = it->second; tables.erase(it); } @@ -160,7 +160,7 @@ void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name, std::lock_guard lock(mutex); auto it = dictionaries.find(dictionary_name); if (it == dictionaries.end()) - throw Exception("Dictionary " + name + "." + dictionary_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + throw Exception("Dictionary " + database_name + "." + dictionary_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); dictionaries.erase(it); } @@ -173,7 +173,7 @@ void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const Sto { std::lock_guard lock(mutex); if (!tables.emplace(table_name, table).second) - throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 056a228a78f..0be3dd5b068 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -51,13 +51,11 @@ public: virtual ~DatabaseWithOwnTablesBase() override; protected: - String name; - mutable std::mutex mutex; Tables tables; Dictionaries dictionaries; - DatabaseWithOwnTablesBase(String name_) : name(std::move(name_)) { } + DatabaseWithOwnTablesBase(String name_) : IDatabase(std::move(name_)) { } }; } diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 4be0a7ad7ae..57d785bb000 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -25,6 +25,9 @@ using Dictionaries = std::set; namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int CANNOT_GET_CREATE_TABLE_QUERY; + extern const int CANNOT_GET_CREATE_TABLE_QUERY; + extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY; } class IDatabaseTablesIterator @@ -96,14 +99,15 @@ using DatabaseDictionariesIteratorPtr = std::unique_ptr { public: + IDatabase() = delete; + IDatabase(String database_name_) : database_name(std::move(database_name_)) {} + /// Get name of database engine. virtual String getEngineName() const = 0; /// Load a set of existing tables. /// You can call only once, right after the object is created. - virtual void loadStoredObjects( - Context & context, - bool has_force_restore_data_flag) = 0; + virtual void loadStoredObjects(Context & /*context*/, bool /*has_force_restore_data_flag*/) {} /// Check the existence of the table. virtual bool isTableExist( @@ -112,8 +116,11 @@ public: /// Check the existence of the dictionary virtual bool isDictionaryExist( - const Context & context, - const String & name) const = 0; + const Context & /*context*/, + const String & /*name*/) const + { + return false; + } /// Get the table for work. Return nullptr if there is no table. virtual StoragePtr tryGetTable( @@ -140,39 +147,63 @@ public: /// Add the table to the database. Record its presence in the metadata. virtual void createTable( - const Context & context, - const String & name, - const StoragePtr & table, - const ASTPtr & query) = 0; + const Context & /*context*/, + const String & /*name*/, + const StoragePtr & /*table*/, + const ASTPtr & /*query*/) + { + throw Exception("There is no CREATE TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); + } /// Add the dictionary to the database. Record its presence in the metadata. virtual void createDictionary( - const Context & context, - const String & dictionary_name, - const ASTPtr & query) = 0; + const Context & /*context*/, + const String & /*dictionary_name*/, + const ASTPtr & /*query*/) + { + throw Exception("There is no CREATE DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); + } /// Delete the table from the database. Delete the metadata. virtual void removeTable( - const Context & context, - const String & name) = 0; + const Context & /*context*/, + const String & /*name*/) + { + throw Exception("There is no DROP TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); + } /// Delete the dictionary from the database. Delete the metadata. virtual void removeDictionary( - const Context & context, - const String & dictionary_name) = 0; + const Context & /*context*/, + const String & /*dictionary_name*/) + { + throw Exception("There is no DROP DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); + } /// Add a table to the database, but do not add it to the metadata. The database may not support this method. - virtual void attachTable(const String & name, const StoragePtr & table) = 0; + virtual void attachTable(const String & /*name*/, const StoragePtr & /*table*/) + { + throw Exception("There is no ATTACH TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); + } /// Add dictionary to the database, but do not add it to the metadata. The database may not support this method. /// load is false when we starting up and lazy_load is true, so we don't want to load dictionaries synchronously. - virtual void attachDictionary(const String & name, const Context & context, bool reload = true) = 0; + virtual void attachDictionary(const String & /*name*/, const Context & /*context*/, [[maybe_unused]] bool reload = true) + { + throw Exception("There is no ATTACH DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); + } /// Forget about the table without deleting it, and return it. The database may not support this method. - virtual StoragePtr detachTable(const String & name) = 0; + virtual StoragePtr detachTable(const String & /*name*/) + { + throw Exception("There is no DETACH TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); + } /// Forget about the dictionary without deleting it, and return it. The database may not support this method. - virtual void detachDictionary(const String & name, const Context & context, bool reload = true) = 0; + virtual void detachDictionary(const String & /*name*/, const Context & /*context*/, [[maybe_unused]] bool reload = true) + { + throw Exception("There is no DETACH DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); + } /// Rename the table and possibly move the table to another database. virtual void renameTable( @@ -201,31 +232,38 @@ public: } /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. - virtual time_t getObjectMetadataModificationTime( - const Context & context, - const String & name) = 0; + virtual time_t getObjectMetadataModificationTime(const Context & /*context*/, const String & /*name*/) + { + return static_cast(0); + } /// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata. - virtual ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const = 0; - - virtual ASTPtr getCreateTableQuery(const Context & context, const String & name) const + ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const noexcept { - return tryGetCreateTableQuery(context, name); + return getCreateTableQueryImpl(context, name, false); + } + + ASTPtr getCreateTableQuery(const Context & context, const String & name) const + { + return getCreateTableQueryImpl(context, name, true); } /// Get the CREATE DICTIONARY query for the dictionary. Returns nullptr if dictionary doesn't exists. - virtual ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & name) const = 0; - - virtual ASTPtr getCreateDictionaryQuery(const Context & context, const String & name) const + ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & name) const noexcept { - return tryGetCreateDictionaryQuery(context, name); + return getCreateDictionaryQueryImpl(context, name, false); + } + + ASTPtr getCreateDictionaryQuery(const Context & context, const String & name) const + { + return getCreateDictionaryQueryImpl(context, name, true); } /// Get the CREATE DATABASE query for current database. virtual ASTPtr getCreateDatabaseQuery(const Context & context) const = 0; /// Get name of database. - virtual String getDatabaseName() const = 0; + String getDatabaseName() const { return database_name; } /// Returns path for persistent data storage if the database supports it, empty string otherwise virtual String getDataPath() const { return {}; } /// Returns metadata path if the database supports it, empty string otherwise @@ -240,6 +278,23 @@ public: virtual void drop(const Context & /*context*/) {} virtual ~IDatabase() {} + +protected: + virtual ASTPtr getCreateTableQueryImpl(const Context & /*context*/, const String & /*name*/, bool throw_on_error) const + { + if (throw_on_error) + throw Exception("There is no SHOW CREATE TABLE query for Database" + getEngineName(), ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY); + return nullptr; + } + + virtual ASTPtr getCreateDictionaryQueryImpl(const Context & /*context*/, const String & /*name*/, bool throw_on_error) const + { + if (throw_on_error) + throw Exception("There is no SHOW CREATE DICTIONARY query for Database" + getEngineName(), ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY); + return nullptr; + } + + String database_name; }; using DatabasePtr = std::shared_ptr; From b6867380565f7a9693e9c856dcc273da7ec85394 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 5 Nov 2019 23:26:14 +0300 Subject: [PATCH 008/743] refactor databases: add DatabaseWithDictionaries --- dbms/src/Databases/DatabaseDictionary.cpp | 7 - dbms/src/Databases/DatabaseDictionary.h | 2 - dbms/src/Databases/DatabaseLazy.cpp | 7 - dbms/src/Databases/DatabaseLazy.h | 2 - dbms/src/Databases/DatabaseMemory.h | 2 - dbms/src/Databases/DatabaseMySQL.h | 5 - dbms/src/Databases/DatabaseOrdinary.cpp | 2 +- dbms/src/Databases/DatabaseOrdinary.h | 4 +- .../Databases/DatabaseWithDictionaries.cpp | 124 ++++++++++++++++++ dbms/src/Databases/DatabaseWithDictionaries.h | 29 ++++ dbms/src/Databases/DatabasesCommon.cpp | 120 +---------------- dbms/src/Databases/DatabasesCommon.h | 10 -- dbms/src/Databases/IDatabase.h | 5 +- 13 files changed, 165 insertions(+), 154 deletions(-) create mode 100644 dbms/src/Databases/DatabaseWithDictionaries.cpp create mode 100644 dbms/src/Databases/DatabaseWithDictionaries.h diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index cea3ea3617d..e00b3661f68 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -65,13 +65,6 @@ bool DatabaseDictionary::isTableExist( return context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST; } -DatabaseDictionariesIteratorPtr DatabaseDictionary::getDictionariesIterator( - const Context & /*context*/, - const FilterByNameFunction & /*filter_by_dictionary_name*/) -{ - return std::make_unique(); -} - StoragePtr DatabaseDictionary::tryGetTable( const Context & context, const String & table_name) const diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index 5a5c21f1720..3cb971a134e 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -39,8 +39,6 @@ public: DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override; - DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override; - bool empty(const Context & context) const override; ASTPtr getCreateDatabaseQuery(const Context & context) const override; diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 20d25b12676..94a2585b91e 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -78,13 +78,6 @@ void DatabaseLazy::removeTable( DatabaseOnDisk::removeTable(*this, context, table_name, log); } -DatabaseDictionariesIteratorPtr DatabaseLazy::getDictionariesIterator( - const Context & /*context*/, - const FilterByNameFunction & /*filter_by_dictionary_name*/) -{ - return std::make_unique(); -} - void DatabaseLazy::renameTable( const Context & context, const String & table_name, diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index aa902d7db44..b138e2e15fa 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -75,8 +75,6 @@ public: DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override; - DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override; - void attachTable(const String & table_name, const StoragePtr & table) override; StoragePtr detachTable(const String & table_name) override; diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 536c8de44f0..633eadb7598 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -33,8 +33,6 @@ public: ASTPtr getCreateDatabaseQuery(const Context & context) const override; - //FIXME isDictionaryExist(...) - private: Poco::Logger * log; }; diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index 413a42d5202..b94129de47f 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -30,11 +30,6 @@ public: DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override; - DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context &, const FilterByNameFunction & = {}) override - { - return std::make_unique(); - } - ASTPtr getCreateDatabaseQuery(const Context & context) const override; bool isTableExist(const Context & context, const String & name) const override; diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index bd99a2553ed..50fcf9fc261 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -93,7 +93,7 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context_) - : DatabaseWithOwnTablesBase(std::move(name_)) + : DatabaseWithDictionaries(std::move(name_)) , metadata_path(metadata_path_) , data_path("data/" + escapeForFileName(database_name) + "/") , log(&Logger::get("DatabaseOrdinary (" + database_name + ")")) diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index c51950e12e8..8ebe0142459 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -11,7 +11,7 @@ namespace DB * It stores tables list in filesystem using list of .sql files, * that contain declaration of table represented by SQL ATTACH TABLE query. */ -class DatabaseOrdinary : public DatabaseWithOwnTablesBase +class DatabaseOrdinary : public DatabaseWithDictionaries //DatabaseWithOwnTablesBase { public: DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context); diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp new file mode 100644 index 00000000000..2947d712a50 --- /dev/null +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -0,0 +1,124 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int EMPTY_LIST_OF_COLUMNS_PASSED; + extern const int TABLE_ALREADY_EXISTS; + extern const int UNKNOWN_TABLE; + extern const int LOGICAL_ERROR; + extern const int DICTIONARY_ALREADY_EXISTS; +} + + +void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const Context & context, bool load) +{ + const auto & external_loader = context.getExternalDictionariesLoader(); + + String full_name = getDatabaseName() + "." + dictionary_name; + { + std::lock_guard lock(mutex); + auto status = external_loader.getCurrentStatus(full_name); + if (status != ExternalLoader::Status::NOT_EXIST || !dictionaries.emplace(dictionary_name).second) + throw Exception( + "Dictionary " + full_name + " already exists.", + ErrorCodes::DICTIONARY_ALREADY_EXISTS); + } + + if (load) + external_loader.reload(full_name, true); +} + +void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name, const Context & context, bool reload) +{ + { + std::lock_guard lock(mutex); + auto it = dictionaries.find(dictionary_name); + if (it == dictionaries.end()) + throw Exception("Dictionary " + database_name + "." + dictionary_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + dictionaries.erase(it); + } + + if (reload) + context.getExternalDictionariesLoader().reload(getDatabaseName() + "." + dictionary_name); + +} + +StoragePtr DatabaseWithDictionaries::tryGetTable(const Context & context, const String & table_name) const +{ + if (auto table_ptr = DatabaseWithOwnTablesBase::tryGetTable(context, table_name)) + return table_ptr; + + if (isDictionaryExist(context, table_name)) + /// We don't need lock database here, because database doesn't store dictionary itself + /// just metadata + return getDictionaryStorage(context, table_name); + + return {}; +} + +DatabaseTablesIteratorPtr DatabaseWithDictionaries::getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name) +{ + /// NOTE: it's not atomic + auto tables_it = getTablesIterator(context, filter_by_name); + auto dictionaries_it = getDictionariesIterator(context, filter_by_name); + + Tables result; + while (tables_it && tables_it->isValid()) + { + result.emplace(tables_it->name(), tables_it->table()); + tables_it->next(); + } + + while (dictionaries_it && dictionaries_it->isValid()) + { + auto table_name = dictionaries_it->name(); + auto table_ptr = getDictionaryStorage(context, table_name); + if (table_ptr) + result.emplace(table_name, table_ptr); + dictionaries_it->next(); + } + + return std::make_unique(result); +} + +DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_dictionary_name) +{ + std::lock_guard lock(mutex); + if (!filter_by_dictionary_name) + return std::make_unique(dictionaries); + + Dictionaries filtered_dictionaries; + for (const auto & dictionary_name : dictionaries) + if (filter_by_dictionary_name(dictionary_name)) + filtered_dictionaries.emplace(dictionary_name); + return std::make_unique(std::move(filtered_dictionaries)); +} + +bool DatabaseWithDictionaries::isDictionaryExist(const Context & /*context*/, const String & dictionary_name) const +{ + std::lock_guard lock(mutex); + return dictionaries.find(dictionary_name) != dictionaries.end(); +} + +StoragePtr DatabaseWithDictionaries::getDictionaryStorage(const Context & context, const String & table_name) const +{ + auto dict_name = database_name + "." + table_name; + const auto & external_loader = context.getExternalDictionariesLoader(); + auto dict_ptr = external_loader.tryGetDictionary(dict_name); + if (dict_ptr) + { + const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); + auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); + return StorageDictionary::create(database_name, table_name, ColumnsDescription{columns}, context, true, dict_name); + } + return nullptr; +} + +} diff --git a/dbms/src/Databases/DatabaseWithDictionaries.h b/dbms/src/Databases/DatabaseWithDictionaries.h new file mode 100644 index 00000000000..4495f851164 --- /dev/null +++ b/dbms/src/Databases/DatabaseWithDictionaries.h @@ -0,0 +1,29 @@ +#include + +namespace DB +{ + + +class DatabaseWithDictionaries : public DatabaseWithOwnTablesBase +{ +public: + void attachDictionary(const String & name, const Context & context, bool reload) override; + + void detachDictionary(const String & name, const Context & context, bool reload) override; + + StoragePtr tryGetTable(const Context & context, const String & table_name) const override; + + DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override; + + DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override; + + bool isDictionaryExist(const Context & context, const String & dictionary_name) const override; + +protected: + DatabaseWithDictionaries(String name) : DatabaseWithOwnTablesBase(std::move(name)) {} + + StoragePtr getDictionaryStorage(const Context & context, const String & table_name) const; + +}; + +} diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 8425c36b417..6230cdf3fdb 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -1,7 +1,4 @@ #include - -#include -#include #include #include #include @@ -24,25 +21,6 @@ namespace ErrorCodes extern const int DICTIONARY_ALREADY_EXISTS; } -namespace -{ - -StoragePtr getDictionaryStorage(const Context & context, const String & table_name, const String & db_name) -{ - auto dict_name = db_name + "." + table_name; - const auto & external_loader = context.getExternalDictionariesLoader(); - auto dict_ptr = external_loader.tryGetDictionary(dict_name); - if (dict_ptr) - { - const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); - auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - return StorageDictionary::create(db_name, table_name, ColumnsDescription{columns}, context, true, dict_name); - } - return nullptr; -} - -} - bool DatabaseWithOwnTablesBase::isTableExist( const Context & /*context*/, const String & table_name) const @@ -51,57 +29,17 @@ bool DatabaseWithOwnTablesBase::isTableExist( return tables.find(table_name) != tables.end() || dictionaries.find(table_name) != dictionaries.end(); } -bool DatabaseWithOwnTablesBase::isDictionaryExist( - const Context & /*context*/, - const String & dictionary_name) const -{ - std::lock_guard lock(mutex); - return dictionaries.find(dictionary_name) != dictionaries.end(); -} - StoragePtr DatabaseWithOwnTablesBase::tryGetTable( - const Context & context, + const Context & /*context*/, const String & table_name) const { - { - std::lock_guard lock(mutex); - auto it = tables.find(table_name); - if (it != tables.end()) - return it->second; - } - - if (isDictionaryExist(context, table_name)) - /// We don't need lock database here, because database doesn't store dictionary itself - /// just metadata - return getDictionaryStorage(context, table_name, getDatabaseName()); - + std::lock_guard lock(mutex); + auto it = tables.find(table_name); + if (it != tables.end()) + return it->second; return {}; } -DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name) -{ - auto tables_it = getTablesIterator(context, filter_by_name); - auto dictionaries_it = getDictionariesIterator(context, filter_by_name); - - Tables result; - while (tables_it && tables_it->isValid()) - { - result.emplace(tables_it->name(), tables_it->table()); - tables_it->next(); - } - - while (dictionaries_it && dictionaries_it->isValid()) - { - auto table_name = dictionaries_it->name(); - auto table_ptr = getDictionaryStorage(context, table_name, getDatabaseName()); - if (table_ptr) - result.emplace(table_name, table_ptr); - dictionaries_it->next(); - } - - return std::make_unique(result); -} - DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); @@ -116,20 +54,6 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Con return std::make_unique(std::move(filtered_tables)); } - -DatabaseDictionariesIteratorPtr DatabaseWithOwnTablesBase::getDictionariesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_dictionary_name) -{ - std::lock_guard lock(mutex); - if (!filter_by_dictionary_name) - return std::make_unique(dictionaries); - - Dictionaries filtered_dictionaries; - for (const auto & dictionary_name : dictionaries) - if (filter_by_dictionary_name(dictionary_name)) - filtered_dictionaries.emplace(dictionary_name); - return std::make_unique(std::move(filtered_dictionaries)); -} - bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const { std::lock_guard lock(mutex); @@ -154,21 +78,6 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name) return res; } -void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name, const Context & context, bool reload) -{ - { - std::lock_guard lock(mutex); - auto it = dictionaries.find(dictionary_name); - if (it == dictionaries.end()) - throw Exception("Dictionary " + database_name + "." + dictionary_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - dictionaries.erase(it); - } - - if (reload) - context.getExternalDictionariesLoader().reload(getDatabaseName() + "." + dictionary_name); - -} - void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table) { std::lock_guard lock(mutex); @@ -176,25 +85,6 @@ void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const Sto throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } - -void DatabaseWithOwnTablesBase::attachDictionary(const String & dictionary_name, const Context & context, bool load) -{ - const auto & external_loader = context.getExternalDictionariesLoader(); - - String full_name = getDatabaseName() + "." + dictionary_name; - { - std::lock_guard lock(mutex); - auto status = external_loader.getCurrentStatus(full_name); - if (status != ExternalLoader::Status::NOT_EXIST || !dictionaries.emplace(dictionary_name).second) - throw Exception( - "Dictionary " + full_name + " already exists.", - ErrorCodes::DICTIONARY_ALREADY_EXISTS); - } - - if (load) - external_loader.reload(full_name, true); -} - void DatabaseWithOwnTablesBase::shutdown() { /// You can not hold a lock during shutdown. diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 0be3dd5b068..38d5087e775 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -24,8 +24,6 @@ public: const Context & context, const String & table_name) const override; - bool isDictionaryExist(const Context & context, const String & dictionary_name) const override; - StoragePtr tryGetTable( const Context & context, const String & table_name) const override; @@ -34,18 +32,10 @@ public: void attachTable(const String & table_name, const StoragePtr & table) override; - void attachDictionary(const String & name, const Context & context, bool reload) override; - StoragePtr detachTable(const String & table_name) override; - void detachDictionary(const String & name, const Context & context, bool reload) override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override; - DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override; - - DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override; - void shutdown() override; virtual ~DatabaseWithOwnTablesBase() override; diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 57d785bb000..7ef53394f0f 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -134,7 +134,10 @@ public: virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0; /// Get an iterator to pass through all the dictionaries. - virtual DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) = 0; + virtual DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & /*context*/, [[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {}) + { + return std::make_unique(); + } /// Get an iterator to pass through all the tables and dictionary tables. virtual DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name = {}) From 14e871535a335f9ae55a3cb321d4957556a1030c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 6 Nov 2019 19:05:04 +0300 Subject: [PATCH 009/743] refactor databases: inherit from DatabaseOnDisk --- dbms/src/Databases/DatabaseAtomic.cpp | 12 +- dbms/src/Databases/DatabaseAtomic.h | 10 +- dbms/src/Databases/DatabaseDictionary.cpp | 4 +- dbms/src/Databases/DatabaseDictionary.h | 2 +- dbms/src/Databases/DatabaseLazy.cpp | 80 ++--- dbms/src/Databases/DatabaseLazy.h | 29 +- dbms/src/Databases/DatabaseMemory.cpp | 5 +- dbms/src/Databases/DatabaseMemory.h | 5 +- dbms/src/Databases/DatabaseMySQL.cpp | 2 +- dbms/src/Databases/DatabaseMySQL.h | 2 +- dbms/src/Databases/DatabaseOnDisk.cpp | 313 +++++++----------- dbms/src/Databases/DatabaseOnDisk.h | 134 ++------ dbms/src/Databases/DatabaseOrdinary.cpp | 97 +----- dbms/src/Databases/DatabaseOrdinary.h | 50 +-- .../Databases/DatabaseWithDictionaries.cpp | 104 +++++- dbms/src/Databases/DatabaseWithDictionaries.h | 20 +- dbms/src/Databases/DatabasesCommon.cpp | 7 +- dbms/src/Databases/DatabasesCommon.h | 3 +- dbms/src/Databases/IDatabase.h | 2 +- ...ExternalLoaderDatabaseConfigRepository.cpp | 2 +- .../Storages/System/StorageSystemTables.cpp | 2 +- 21 files changed, 320 insertions(+), 565 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index dfbb6d1057e..f45baaa213e 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -10,12 +10,12 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Contex { } -void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name, TableStructureWriteLockHolder & lock) -{ - //TODO - DatabaseOnDisk::renameTable(*this, context, table_name, to_database, to_table_name, lock); -} +//void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, +// const String & to_table_name, TableStructureWriteLockHolder & lock) +//{ +// //TODO +// DatabaseOnDisk::renameTable(*this, context, table_name, to_database, to_table_name, lock); +//} } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index 9bd1b1184c5..8b6ea1075f0 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -15,11 +15,11 @@ public: String getEngineName() const override { return "Atomic"; } - void renameTable(const Context & context, - const String & table_name, - IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder &) override; + //void renameTable(const Context & context, + // const String & table_name, + // IDatabase & to_database, + // const String & to_table_name, + // TableStructureWriteLockHolder &) override; }; diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index e00b3661f68..771a9f5924a 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -22,8 +22,8 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } -DatabaseDictionary::DatabaseDictionary(String name_) - : IDatabase(std::move(name_)), +DatabaseDictionary::DatabaseDictionary(const String & name_) + : IDatabase(name_), log(&Logger::get("DatabaseDictionary(" + database_name + ")")) { } diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index 3cb971a134e..cd5dde3177c 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -22,7 +22,7 @@ namespace DB class DatabaseDictionary : public IDatabase { public: - DatabaseDictionary(String name_); + DatabaseDictionary(const String & name_); String getEngineName() const override { diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 94a2585b91e..58619cd2993 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -29,12 +29,9 @@ namespace ErrorCodes -DatabaseLazy::DatabaseLazy(String name_, const String & metadata_path_, time_t expiration_time_, const Context & context_) - : IDatabase(std::move(name_)) - , metadata_path(metadata_path_) - , data_path("data/" + escapeForFileName(database_name) + "/") +DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_) + : DatabaseOnDisk(name_, metadata_path_, "DatabaseLazy (" + name_ + ")") , expiration_time(expiration_time_) - , log(&Logger::get("DatabaseLazy (" + database_name + ")")) { Poco::File(context_.getPath() + getDataPath()).createDirectories(); } @@ -44,7 +41,7 @@ void DatabaseLazy::loadStoredObjects( Context & context, bool /* has_force_restore_data_flag */) { - DatabaseOnDisk::iterateMetadataFiles(*this, log, context, [this](const String & file_name) + iterateMetadataFiles(context, [this](const String & file_name) { const std::string table_name = file_name.substr(0, file_name.size() - 4); attachTable(table_name, nullptr); @@ -61,13 +58,13 @@ void DatabaseLazy::createTable( SCOPE_EXIT({ clearExpiredTables(); }); if (!endsWith(table->getName(), "Log")) throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD); - DatabaseOnDisk::createTable(*this, context, table_name, table, query); + DatabaseOnDisk::createTable(context, table_name, table, query); /// DatabaseOnDisk::createTable renames file, so we need to get new metadata_modification_time. - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); auto it = tables_cache.find(table_name); if (it != tables_cache.end()) - it->second.metadata_modification_time = DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name); + it->second.metadata_modification_time = DatabaseOnDisk::getObjectMetadataModificationTime(table_name); } void DatabaseLazy::removeTable( @@ -75,7 +72,7 @@ void DatabaseLazy::removeTable( const String & table_name) { SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::removeTable(*this, context, table_name, log); + DatabaseOnDisk::removeTable(context, table_name); } void DatabaseLazy::renameTable( @@ -86,30 +83,17 @@ void DatabaseLazy::renameTable( TableStructureWriteLockHolder & lock) { SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::renameTable(*this, context, table_name, to_database, to_table_name, lock); + DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, lock); } -time_t DatabaseLazy::getObjectMetadataModificationTime( - const Context & /* context */, - const String & table_name) +time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name) const { - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); auto it = tables_cache.find(table_name); if (it != tables_cache.end()) return it->second.metadata_modification_time; - else - throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); -} - -ASTPtr DatabaseLazy::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const -{ - return DatabaseOnDisk::getCreateTableQueryImpl(*this, context, table_name, throw_on_error); -} - -ASTPtr DatabaseLazy::getCreateDatabaseQuery(const Context & context) const -{ - return DatabaseOnDisk::getCreateDatabaseQuery(*this, context); + throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); } void DatabaseLazy::alterTable( @@ -120,22 +104,17 @@ void DatabaseLazy::alterTable( const ConstraintsDescription & /* constraints */, const ASTModifier & /* storage_modifier */) { + //FIXME WTF SCOPE_EXIT({ clearExpiredTables(); }); throw Exception("ALTER query is not supported for Lazy database.", ErrorCodes::UNSUPPORTED_METHOD); } - -void DatabaseLazy::drop(const Context & context) -{ - DatabaseOnDisk::drop(*this, context); -} - bool DatabaseLazy::isTableExist( const Context & /* context */, const String & table_name) const { SCOPE_EXIT({ clearExpiredTables(); }); - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); return tables_cache.find(table_name) != tables_cache.end(); } @@ -145,7 +124,7 @@ StoragePtr DatabaseLazy::tryGetTable( { SCOPE_EXIT({ clearExpiredTables(); }); { - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); auto it = tables_cache.find(table_name); if (it == tables_cache.end()) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -165,7 +144,7 @@ StoragePtr DatabaseLazy::tryGetTable( DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) { - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); Strings filtered_tables; for (const auto & [table_name, cached_table] : tables_cache) { @@ -184,12 +163,12 @@ bool DatabaseLazy::empty(const Context & /* context */) const void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table) { LOG_DEBUG(log, "Attach table " << backQuote(table_name) << "."); - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); auto [it, inserted] = tables_cache.emplace(std::piecewise_construct, std::forward_as_tuple(table_name), - std::forward_as_tuple(table, current_time, DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name))); + std::forward_as_tuple(table, current_time, DatabaseOnDisk::getObjectMetadataModificationTime(table_name))); if (!inserted) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); @@ -201,7 +180,7 @@ StoragePtr DatabaseLazy::detachTable(const String & table_name) StoragePtr res; { LOG_DEBUG(log, "Detach table " << backQuote(table_name) << "."); - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); auto it = tables_cache.find(table_name); if (it == tables_cache.end()) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -217,7 +196,7 @@ void DatabaseLazy::shutdown() { TablesCache tables_snapshot; { - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); tables_snapshot = tables_cache; } @@ -227,7 +206,7 @@ void DatabaseLazy::shutdown() kv.second.table->shutdown(); } - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); tables_cache.clear(); } @@ -243,21 +222,6 @@ DatabaseLazy::~DatabaseLazy() } } -String DatabaseLazy::getDataPath() const -{ - return data_path; -} - -String DatabaseLazy::getMetadataPath() const -{ - return metadata_path; -} - -String DatabaseLazy::getObjectMetadataPath(const String & table_name) const -{ - return DatabaseOnDisk::getObjectMetadataPath(*this, table_name); -} - StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table_name) const { SCOPE_EXIT({ clearExpiredTables(); }); @@ -282,7 +246,7 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table if (!ast || !endsWith(table->getName(), "Log")) throw Exception("Only *Log tables can be used with Lazy database engine.", ErrorCodes::LOGICAL_ERROR); { - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); auto it = tables_cache.find(table_name); if (it == tables_cache.end()) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -305,7 +269,7 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table void DatabaseLazy::clearExpiredTables() const { - std::lock_guard lock(tables_mutex); + std::lock_guard lock(mutex); auto time_now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); CacheExpirationQueue expired_tables; diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index b138e2e15fa..4ff77f279b6 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -15,10 +15,11 @@ class DatabaseLazyIterator; * Works like DatabaseOrdinary, but stores in memory only cache. * Can be used only with *Log engines. */ -class DatabaseLazy : public IDatabase +class DatabaseLazy : public DatabaseOnDisk { + //TODO rewrite it all public: - DatabaseLazy(String name_, const String & metadata_path_, time_t expiration_time_, const Context & context_); + DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_); String getEngineName() const override { return "Lazy"; } @@ -51,17 +52,7 @@ public: const ConstraintsDescription & constraints, const ASTModifier & engine_modifier) override; - time_t getObjectMetadataModificationTime( - const Context & context, - const String & table_name) override; - - ASTPtr getCreateDatabaseQuery(const Context & context) const override; - - String getDataPath() const override; - String getMetadataPath() const override; - String getObjectMetadataPath(const String & table_name) const override; - - void drop(const Context & context) override; + time_t getObjectMetadataModificationTime(const String & table_name) const override; bool isTableExist( const Context & context, @@ -83,9 +74,6 @@ public: ~DatabaseLazy() override; -protected: - ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const override; - private: struct CacheExpirationQueueElement { @@ -113,18 +101,11 @@ private: using TablesCache = std::unordered_map; - - const String metadata_path; - const String data_path; - const time_t expiration_time; - mutable std::mutex tables_mutex; mutable TablesCache tables_cache; mutable CacheExpirationQueue cache_expiration_queue; - Poco::Logger * log; - StoragePtr loadTable(const Context & context, const String & table_name) const; void clearExpiredTables() const; diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index 20db4d29a63..96d8d7e72aa 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -13,9 +13,8 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } -DatabaseMemory::DatabaseMemory(String name_) - : DatabaseWithOwnTablesBase(std::move(name_)) - , log(&Logger::get("DatabaseMemory(" + database_name + ")")) +DatabaseMemory::DatabaseMemory(const String & name_) + : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")") {} void DatabaseMemory::createTable( diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 633eadb7598..c5239fc88ee 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -17,7 +17,7 @@ namespace DB class DatabaseMemory : public DatabaseWithOwnTablesBase { public: - DatabaseMemory(String name_); + DatabaseMemory(const String & name_); String getEngineName() const override { return "Memory"; } @@ -32,9 +32,6 @@ public: const String & table_name) override; ASTPtr getCreateDatabaseQuery(const Context & context) const override; - -private: - Poco::Logger * log; }; } diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index 6eea60b6d1c..b4933040d99 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -112,7 +112,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const Context &, const String & ta return local_tables_cache[table_name].create_table_query; } -time_t DatabaseMySQL::getObjectMetadataModificationTime(const Context &, const String & table_name) +time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_name) const { std::lock_guard lock(mutex); diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index b94129de47f..0776d9cd9c4 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -36,7 +36,7 @@ public: StoragePtr tryGetTable(const Context & context, const String & name) const override; - time_t getObjectMetadataModificationTime(const Context & context, const String & name) override; + time_t getObjectMetadataModificationTime(const String & name) const override; void shutdown() override; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index f157285c987..2b82ff9f4ed 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -28,8 +28,6 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; namespace ErrorCodes { - extern const int CANNOT_GET_CREATE_TABLE_QUERY; - extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY; extern const int FILE_DOESNT_EXIST; extern const int INCORRECT_FILE_NAME; extern const int SYNTAX_ERROR; @@ -40,63 +38,6 @@ namespace ErrorCodes } -namespace detail -{ - String getObjectMetadataPath(const String & base_path, const String & table_name) - { - return base_path + (endsWith(base_path, "/") ? "" : "/") + escapeForFileName(table_name) + ".sql"; - } - - String getDatabaseMetadataPath(const String & base_path) - { - return (endsWith(base_path, "/") ? base_path.substr(0, base_path.size() - 1) : base_path) + ".sql"; - } - - ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error) - { - String query; - - try - { - ReadBufferFromFile in(metadata_path, 4096); - readStringUntilEOF(query, in); - } - catch (const Exception & e) - { - if (!throw_on_error && e.code() == ErrorCodes::FILE_DOESNT_EXIST) - return nullptr; - else - throw; - } - - ParserCreateQuery parser; - const char * pos = query.data(); - std::string error_message; - auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false, - "in file " + metadata_path, /* allow_multi_statements = */ false, 0); - - if (!ast && throw_on_error) - throw Exception(error_message, ErrorCodes::SYNTAX_ERROR); - - return ast; - } - - ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const String & database, bool throw_on_error) - { - ASTPtr ast = getQueryFromMetadata(metadata_path, throw_on_error); - - if (ast) - { - auto & ast_create_query = ast->as(); - ast_create_query.attach = false; - ast_create_query.database = database; - } - - return ast; - } -} - - ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log) { String definition; @@ -169,6 +110,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) if (!create) { std::ostringstream query_stream; + //FIXME WTF formatAST(*create, query_stream, true); throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR); } @@ -198,7 +140,6 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) } void DatabaseOnDisk::createTable( - IDatabase & database, const Context & context, const String & table_name, const StoragePtr & table, @@ -218,14 +159,14 @@ void DatabaseOnDisk::createTable( /// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH. /// But there is protection from it - see using DDLGuard in InterpreterCreateQuery. - if (database.isDictionaryExist(context, table_name)) - throw Exception("Dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " already exists.", + if (isDictionaryExist(context, table_name)) + throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); - if (database.isTableExist(context, table_name)) - throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + if (isTableExist(context, table_name)) + throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); - String table_metadata_path = database.getObjectMetadataPath(table_name); + String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_tmp_path = table_metadata_path + ".tmp"; String statement; @@ -244,7 +185,7 @@ void DatabaseOnDisk::createTable( try { /// Add a table to the map of known tables. - database.attachTable(table_name, table); + attachTable(table_name, table); /// If it was ATTACH query and file with table metadata already exist /// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one. @@ -257,79 +198,11 @@ void DatabaseOnDisk::createTable( } } - -void DatabaseOnDisk::createDictionary( - IDatabase & database, - const Context & context, - const String & dictionary_name, - const ASTPtr & query) +void DatabaseOnDisk::removeTable(const Context & /* context */, const String & table_name) { - const auto & settings = context.getSettingsRef(); + StoragePtr res = detachTable(table_name); - /** The code is based on the assumption that all threads share the same order of operations - * - creating the .sql.tmp file; - * - adding a dictionary to `dictionaries`; - * - rename .sql.tmp to .sql. - */ - - /// A race condition would be possible if a dictionary with the same name is simultaneously created using CREATE and using ATTACH. - /// But there is protection from it - see using DDLGuard in InterpreterCreateQuery. - if (database.isDictionaryExist(context, dictionary_name)) - throw Exception("Dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); - - if (database.isTableExist(context, dictionary_name)) - throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); - - - String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name); - String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp"; - String statement; - - { - statement = getObjectDefinitionFromCreateQuery(query); - - /// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown. - WriteBufferFromFile out(dictionary_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL); - writeString(statement, out); - out.next(); - if (settings.fsync_metadata) - out.sync(); - out.close(); - } - - try - { - /// Do not load it now because we want more strict loading - database.attachDictionary(dictionary_name, context, false); - /// Load dictionary - bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true); - String dict_name = database.getDatabaseName() + "." + dictionary_name; - context.getExternalDictionariesLoader().addDictionaryWithConfig( - dict_name, database.getDatabaseName(), query->as(), !lazy_load); - - /// If it was ATTACH query and file with dictionary metadata already exist - /// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one. - Poco::File(dictionary_metadata_tmp_path).renameTo(dictionary_metadata_path); - - } - catch (...) - { - database.detachDictionary(dictionary_name, context); - Poco::File(dictionary_metadata_tmp_path).remove(); - throw; - } -} - - -void DatabaseOnDisk::removeTable( - IDatabase & database, - const Context & /* context */, - const String & table_name, - Poco::Logger * log) -{ - StoragePtr res = database.detachTable(table_name); - - String table_metadata_path = database.getObjectMetadataPath(table_name); + String table_metadata_path = getObjectMetadataPath(table_name); try { @@ -346,49 +219,63 @@ void DatabaseOnDisk::removeTable( { LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); } - database.attachTable(table_name, res); + attachTable(table_name, res); throw; } } - -void DatabaseOnDisk::removeDictionary( - IDatabase & database, - const Context & context, - const String & dictionary_name, - Poco::Logger * /*log*/) +void DatabaseOnDisk::renameTable( + const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name, + TableStructureWriteLockHolder & lock) { - database.detachDictionary(dictionary_name, context); + if (typeid(*this) != typeid(to_database)) + throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); - String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name); + StoragePtr table = tryGetTable(context, table_name); + if (!table) + throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + + /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. try { - Poco::File(dictionary_metadata_path).remove(); + table->rename(context.getPath() + "/data/" + escapeForFileName(to_database.getDatabaseName()) + "/", + to_database.getDatabaseName(), + to_table_name, lock); } - catch (...) + catch (const Exception &) { - /// If remove was not possible for some reason - database.attachDictionary(dictionary_name, context); throw; } + catch (const Poco::Exception & e) + { + /// Better diagnostics. + throw Exception{Exception::CreateFromPoco, e}; + } + + ASTPtr ast = getQueryFromMetadata(getObjectMetadataPath(table_name)); + if (!ast) + throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST); + ast->as().table = to_table_name; + + /// NOTE Non-atomic. + to_database.createTable(context, to_table_name, table, ast); + removeTable(context, table_name); } - -ASTPtr DatabaseOnDisk::getCreateTableQueryImpl( - const IDatabase & database, - const Context & context, - const String & table_name, - bool throw_on_error) +ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const { ASTPtr ast; - auto table_metadata_path = detail::getObjectMetadataPath(database.getMetadataPath(), table_name); - ast = detail::getCreateQueryFromMetadata(table_metadata_path, database.getDatabaseName(), throw_on_error); + auto table_metadata_path = getObjectMetadataPath(table_name); + ast = getCreateQueryFromMetadata(table_metadata_path, throw_on_error); if (!ast && throw_on_error) { /// Handle system.* tables for which there are no table.sql files. - bool has_table = database.tryGetTable(context, table_name) != nullptr; + bool has_table = tryGetTable(context, table_name) != nullptr; auto msg = has_table ? "There is no CREATE TABLE query for table " @@ -400,40 +287,17 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl( return ast; } - -ASTPtr DatabaseOnDisk::getCreateDictionaryQueryImpl( - const IDatabase & database, - const Context & context, - const String & dictionary_name, - bool throw_on_error) +ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const Context & /*context*/) const { ASTPtr ast; - auto dictionary_metadata_path = detail::getObjectMetadataPath(database.getMetadataPath(), dictionary_name); - ast = detail::getCreateQueryFromMetadata(dictionary_metadata_path, database.getDatabaseName(), throw_on_error); - if (!ast && throw_on_error) - { - /// Handle system.* tables for which there are no table.sql files. - bool has_dictionary = database.isDictionaryExist(context, dictionary_name); - - auto msg = has_dictionary ? "There is no CREATE DICTIONARY query for table " : "There is no metadata file for dictionary "; - - throw Exception(msg + backQuote(dictionary_name), ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY); - } - - return ast; -} - -ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const IDatabase & database, const Context & /*context*/) -{ - ASTPtr ast; - - auto database_metadata_path = detail::getDatabaseMetadataPath(database.getMetadataPath()); - ast = detail::getCreateQueryFromMetadata(database_metadata_path, database.getDatabaseName(), true); + auto database_metadata_path = getDatabaseMetadataPath(getMetadataPath()); + ast = getCreateQueryFromMetadata(database_metadata_path, true); if (!ast) { /// Handle databases (such as default) for which there are no database.sql files. - String query = "CREATE DATABASE " + backQuoteIfNeed(database.getDatabaseName()) + " ENGINE = Lazy"; + //FIXME WTF + String query = "CREATE DATABASE " + backQuoteIfNeed(getDatabaseName()) + " ENGINE = Lazy"; ParserCreateQuery parser; ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0); } @@ -441,22 +305,22 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const IDatabase & database, const return ast; } -void DatabaseOnDisk::drop(const IDatabase & database, const Context & context) +void DatabaseOnDisk::drop(const Context & context) { - Poco::File(context.getPath() + database.getDataPath()).remove(false); - Poco::File(database.getMetadataPath()).remove(false); + Poco::File(context.getPath() + getDataPath()).remove(false); + Poco::File(getMetadataPath()).remove(false); } -String DatabaseOnDisk::getObjectMetadataPath(const IDatabase & database, const String & table_name) +String DatabaseOnDisk::getObjectMetadataPath(const String & table_name) const { - return detail::getObjectMetadataPath(database.getMetadataPath(), table_name); + String base_path = getMetadataPath(); + //FIXME + return base_path + (endsWith(base_path, "/") ? "" : "/") + escapeForFileName(table_name) + ".sql"; } -time_t DatabaseOnDisk::getObjectMetadataModificationTime( - const IDatabase & database, - const String & table_name) +time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & table_name) const { - String table_metadata_path = getObjectMetadataPath(database, table_name); + String table_metadata_path = getObjectMetadataPath(table_name); Poco::File meta_file(table_metadata_path); if (meta_file.exists()) @@ -465,10 +329,10 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime( return static_cast(0); } -void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function) +void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & iterating_function) const { Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator dir_it(database.getMetadataPath()); dir_it != dir_end; ++dir_it) + for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) { /// For '.svn', '.gitignore' directory and similar. if (dir_it.name().at(0) == '.') @@ -483,7 +347,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logg if (endsWith(dir_it.name(), tmp_drop_ext)) { const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext)); - if (Poco::File(context.getPath() + database.getDataPath() + '/' + object_name).exists()) + if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists()) { Poco::File(dir_it->path()).renameTo(object_name + ".sql"); LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously"); @@ -510,9 +374,58 @@ void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logg iterating_function(dir_it.name()); } else - throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + database.getMetadataPath(), + throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + getMetadataPath(), ErrorCodes::INCORRECT_FILE_NAME); } } +String DatabaseOnDisk::getDatabaseMetadataPath(const String & base_path) const +{ + //FIXME + return (endsWith(base_path, "/") ? base_path.substr(0, base_path.size() - 1) : base_path) + ".sql"; +} + +ASTPtr DatabaseOnDisk::getQueryFromMetadata(const String & database_metadata_path, bool throw_on_error) const +{ + String query; + + try + { + ReadBufferFromFile in(database_metadata_path, 4096); + readStringUntilEOF(query, in); + } + catch (const Exception & e) + { + if (!throw_on_error && e.code() == ErrorCodes::FILE_DOESNT_EXIST) + return nullptr; + else + throw; + } + + ParserCreateQuery parser; + const char * pos = query.data(); + std::string error_message; + auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false, + "in file " + metadata_path, /* allow_multi_statements = */ false, 0); + + if (!ast && throw_on_error) + throw Exception(error_message, ErrorCodes::SYNTAX_ERROR); + + return ast; +} + +ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metadata_path, bool throw_on_error) const +{ + ASTPtr ast = getQueryFromMetadata(database_metadata_path, throw_on_error); + + if (ast) + { + auto & ast_create_query = ast->as(); + ast_create_query.attach = false; + ast_create_query.database = database_name; + } + + return ast; +} + } diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index d2aa3d48be7..9833b5135b4 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -11,14 +11,6 @@ namespace DB { -namespace detail -{ - String getObjectMetadataPath(const String & base_path, const String & dictionary_name); - String getDatabaseMetadataPath(const String & base_path); - ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true); - ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const String & database, bool throw_on_error); -} - ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log); std::pair createTableFromAST( @@ -37,126 +29,58 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query); /* Class to provide basic operations with tables when metadata is stored on disk in .sql files. */ -class DatabaseOnDisk +class DatabaseOnDisk : public DatabaseWithOwnTablesBase { public: - static void createTable( - IDatabase & database, + DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger) + : DatabaseWithOwnTablesBase(name, logger) + , metadata_path(metadata_path_) + , data_path("data/" + escapeForFileName(database_name) + "/") {} + + void createTable( const Context & context, const String & table_name, const StoragePtr & table, - const ASTPtr & query); + const ASTPtr & query) override; - static void createDictionary( - IDatabase & database, + void removeTable( const Context & context, - const String & dictionary_name, - const ASTPtr & query); + const String & table_name) override; - static void removeTable( - IDatabase & database, - const Context & context, - const String & table_name, - Poco::Logger * log); - - static void removeDictionary( - IDatabase & database, - const Context & context, - const String & dictionary_name, - Poco::Logger * log); - - template - static void renameTable( - IDatabase & database, + void renameTable( const Context & context, const String & table_name, IDatabase & to_database, const String & to_table_name, - TableStructureWriteLockHolder & lock); + TableStructureWriteLockHolder & lock) override; - static ASTPtr getCreateDatabaseQuery( - const IDatabase & database, - const Context & context); + ASTPtr getCreateDatabaseQuery(const Context & context) const override; - static void drop(const IDatabase & database, const Context & context); + void drop(const Context & context) override; - static String getObjectMetadataPath( - const IDatabase & database, - const String & object_name); + String getObjectMetadataPath(const String & object_name) const override; - static time_t getObjectMetadataModificationTime( - const IDatabase & database, - const String & object_name); + time_t getObjectMetadataModificationTime(const String & object_name) const override; + String getDataPath() const override { return data_path; } + String getMetadataPath() const override { return metadata_path; } +protected: using IteratingFunction = std::function; - static void iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function); + void iterateMetadataFiles(const Context & context, const IteratingFunction & iterating_function) const; - static ASTPtr getCreateTableQueryImpl( - const IDatabase & database, + ASTPtr getCreateTableQueryImpl( const Context & context, const String & table_name, - bool throw_on_error); + bool throw_on_error) const override; - static ASTPtr getCreateDictionaryQueryImpl( - const IDatabase & database, - const Context & context, - const String & dictionary_name, - bool throw_on_error); + String getDatabaseMetadataPath(const String & base_path) const; + ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true) const; + ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const; + + + const String metadata_path; + const String data_path; }; - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; - extern const int UNKNOWN_TABLE; - extern const int FILE_DOESNT_EXIST; -} - -template -void DatabaseOnDisk::renameTable( - IDatabase & database, - const Context & context, - const String & table_name, - IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder & lock) -{ - Database * to_database_concrete = typeid_cast(&to_database); - - if (!to_database_concrete) - throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); - - StoragePtr table = database.tryGetTable(context, table_name); - - if (!table) - throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - - /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. - try - { - table->rename(context.getPath() + "/data/" + escapeForFileName(to_database_concrete->getDatabaseName()) + "/", - to_database_concrete->getDatabaseName(), - to_table_name, lock); - } - catch (const Exception &) - { - throw; - } - catch (const Poco::Exception & e) - { - /// Better diagnostics. - throw Exception{Exception::CreateFromPoco, e}; - } - - ASTPtr ast = detail::getQueryFromMetadata(detail::getObjectMetadataPath(database.getMetadataPath(), table_name)); - if (!ast) - throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST); - ast->as().table = to_table_name; - - /// NOTE Non-atomic. - to_database_concrete->createTable(context, to_table_name, table, ast); - database.removeTable(context, table_name); -} - } diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 50fcf9fc261..ec4850bcd7c 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -35,10 +35,8 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_CREATE_TABLE_FROM_METADATA; - extern const int CANNOT_CREATE_DICTIONARY_FROM_METADATA; extern const int EMPTY_LIST_OF_COLUMNS_PASSED; extern const int CANNOT_PARSE_TEXT; - extern const int EMPTY_LIST_OF_ATTRIBUTES_PASSED; } @@ -92,11 +90,8 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic } -DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context_) - : DatabaseWithDictionaries(std::move(name_)) - , metadata_path(metadata_path_) - , data_path("data/" + escapeForFileName(database_name) + "/") - , log(&Logger::get("DatabaseOrdinary (" + database_name + ")")) +DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context_) + : DatabaseWithDictionaries(name_, metadata_path_,"DatabaseOrdinary (" + name_ + ")") { Poco::File(context_.getPath() + getDataPath()).createDirectories(); } @@ -114,7 +109,7 @@ void DatabaseOrdinary::loadStoredObjects( FileNames file_names; size_t total_dictionaries = 0; - DatabaseOnDisk::iterateMetadataFiles(*this, log, context, [&file_names, &total_dictionaries, this](const String & file_name) + iterateMetadataFiles(context, [&file_names, &total_dictionaries, this](const String & file_name) { String full_path = metadata_path + "/" + file_name; try @@ -206,70 +201,6 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool) thread_pool.wait(); } -void DatabaseOrdinary::createTable( - const Context & context, - const String & table_name, - const StoragePtr & table, - const ASTPtr & query) -{ - DatabaseOnDisk::createTable(*this, context, table_name, table, query); -} - -void DatabaseOrdinary::createDictionary( - const Context & context, - const String & dictionary_name, - const ASTPtr & query) -{ - DatabaseOnDisk::createDictionary(*this, context, dictionary_name, query); -} - -void DatabaseOrdinary::removeTable( - const Context & context, - const String & table_name) -{ - DatabaseOnDisk::removeTable(*this, context, table_name, log); -} - -void DatabaseOrdinary::removeDictionary( - const Context & context, - const String & table_name) -{ - DatabaseOnDisk::removeDictionary(*this, context, table_name, log); -} - -void DatabaseOrdinary::renameTable( - const Context & context, - const String & table_name, - IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder & lock) -{ - DatabaseOnDisk::renameTable(*this, context, table_name, to_database, to_table_name, lock); -} - - -time_t DatabaseOrdinary::getObjectMetadataModificationTime( - const Context & /* context */, - const String & table_name) -{ - return DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name); -} - -ASTPtr DatabaseOrdinary::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const -{ - return DatabaseOnDisk::getCreateTableQueryImpl(*this, context, table_name, throw_on_error); -} - -ASTPtr DatabaseOrdinary::getCreateDictionaryQueryImpl(const Context & context, const String & dictionary_name, bool throw_on_error) const -{ - return DatabaseOnDisk::getCreateDictionaryQueryImpl(*this, context, dictionary_name, throw_on_error); -} - -ASTPtr DatabaseOrdinary::getCreateDatabaseQuery(const Context & context) const -{ - return DatabaseOnDisk::getCreateDatabaseQuery(*this, context); -} - void DatabaseOrdinary::alterTable( const Context & context, const String & table_name, @@ -330,26 +261,4 @@ void DatabaseOrdinary::alterTable( } } - -void DatabaseOrdinary::drop(const Context & context) -{ - DatabaseOnDisk::drop(*this, context); -} - - -String DatabaseOrdinary::getDataPath() const -{ - return data_path; -} - -String DatabaseOrdinary::getMetadataPath() const -{ - return metadata_path; -} - -String DatabaseOrdinary::getObjectMetadataPath(const String & table_name) const -{ - return DatabaseOnDisk::getObjectMetadataPath(*this, table_name); -} - } diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index 8ebe0142459..c4b624b22a5 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -14,7 +14,7 @@ namespace DB class DatabaseOrdinary : public DatabaseWithDictionaries //DatabaseWithOwnTablesBase { public: - DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context); + DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context); String getEngineName() const override { return "Ordinary"; } @@ -22,32 +22,6 @@ public: Context & context, bool has_force_restore_data_flag) override; - void createTable( - const Context & context, - const String & table_name, - const StoragePtr & table, - const ASTPtr & query) override; - - void createDictionary( - const Context & context, - const String & dictionary_name, - const ASTPtr & query) override; - - void removeTable( - const Context & context, - const String & table_name) override; - - void removeDictionary( - const Context & context, - const String & table_name) override; - - void renameTable( - const Context & context, - const String & table_name, - IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder &) override; - void alterTable( const Context & context, const String & name, @@ -56,29 +30,7 @@ public: const ConstraintsDescription & constraints, const ASTModifier & engine_modifier) override; - time_t getObjectMetadataModificationTime( - const Context & context, - const String & table_name) override; - - ASTPtr getCreateDatabaseQuery(const Context & context) const override; - - String getDataPath() const override; - String getMetadataPath() const override; - String getObjectMetadataPath(const String & table_name) const override; - - void drop(const Context & context) override; - -protected: - ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, - bool throw_on_error) const override; - - ASTPtr getCreateDictionaryQueryImpl(const Context & context, const String & name, - bool throw_on_error) const override; - private: - const String metadata_path; - const String data_path; - Poco::Logger * log; void startupTables(ThreadPool & thread_pool); }; diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index 2947d712a50..445653b8f31 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include namespace DB @@ -17,7 +19,7 @@ namespace ErrorCodes } -void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const Context & context, bool load) +void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const Context & context, bool reload) { const auto & external_loader = context.getExternalDictionariesLoader(); @@ -31,7 +33,7 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, ErrorCodes::DICTIONARY_ALREADY_EXISTS); } - if (load) + if (reload) external_loader.reload(full_name, true); } @@ -50,6 +52,82 @@ void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name, } +void DatabaseWithDictionaries::createDictionary( const Context & context, const String & dictionary_name, const ASTPtr & query) +{ + const auto & settings = context.getSettingsRef(); + + /** The code is based on the assumption that all threads share the same order of operations + * - creating the .sql.tmp file; + * - adding a dictionary to `dictionaries`; + * - rename .sql.tmp to .sql. + */ + + /// A race condition would be possible if a dictionary with the same name is simultaneously created using CREATE and using ATTACH. + /// But there is protection from it - see using DDLGuard in InterpreterCreateQuery. + if (isDictionaryExist(context, dictionary_name)) + throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); + + if (isTableExist(context, dictionary_name)) + throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + + + String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); + String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp"; + String statement; + + { + statement = getObjectDefinitionFromCreateQuery(query); + + /// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown. + WriteBufferFromFile out(dictionary_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL); + writeString(statement, out); + out.next(); + if (settings.fsync_metadata) + out.sync(); + out.close(); + } + + try + { + /// Do not load it now because we want more strict loading + attachDictionary(dictionary_name, context, false); + /// Load dictionary + bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true); + String dict_name = getDatabaseName() + "." + dictionary_name; + context.getExternalDictionariesLoader().addDictionaryWithConfig( + dict_name, getDatabaseName(), query->as(), !lazy_load); + + /// If it was ATTACH query and file with dictionary metadata already exist + /// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one. + Poco::File(dictionary_metadata_tmp_path).renameTo(dictionary_metadata_path); + + } + catch (...) + { + detachDictionary(dictionary_name, context); + Poco::File(dictionary_metadata_tmp_path).remove(); + throw; + } +} + +void DatabaseWithDictionaries::removeDictionary(const Context & context, const String & dictionary_name) +{ + detachDictionary(dictionary_name, context); + + String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); + + try + { + Poco::File(dictionary_metadata_path).remove(); + } + catch (...) + { + /// If remove was not possible for some reason + attachDictionary(dictionary_name, context); + throw; + } +} + StoragePtr DatabaseWithDictionaries::tryGetTable(const Context & context, const String & table_name) const { if (auto table_ptr = DatabaseWithOwnTablesBase::tryGetTable(context, table_name)) @@ -121,4 +199,26 @@ StoragePtr DatabaseWithDictionaries::getDictionaryStorage(const Context & contex return nullptr; } +ASTPtr DatabaseWithDictionaries::getCreateDictionaryQueryImpl( + const Context & context, + const String & dictionary_name, + bool throw_on_error) const +{ + ASTPtr ast; + + auto dictionary_metadata_path = getObjectMetadataPath(dictionary_name); + ast = getCreateQueryFromMetadata(dictionary_metadata_path, throw_on_error); + if (!ast && throw_on_error) + { + /// Handle system.* tables for which there are no table.sql files. + bool has_dictionary = isDictionaryExist(context, dictionary_name); + + auto msg = has_dictionary ? "There is no CREATE DICTIONARY query for table " : "There is no metadata file for dictionary "; + + throw Exception(msg + backQuote(dictionary_name), ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY); + } + + return ast; +} + } diff --git a/dbms/src/Databases/DatabaseWithDictionaries.h b/dbms/src/Databases/DatabaseWithDictionaries.h index 4495f851164..59e8023b730 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.h +++ b/dbms/src/Databases/DatabaseWithDictionaries.h @@ -1,15 +1,21 @@ -#include +#include namespace DB { -class DatabaseWithDictionaries : public DatabaseWithOwnTablesBase +class DatabaseWithDictionaries : public DatabaseOnDisk { public: - void attachDictionary(const String & name, const Context & context, bool reload) override; + void attachDictionary(const String & name, const Context & context, bool reload = true) override; - void detachDictionary(const String & name, const Context & context, bool reload) override; + void detachDictionary(const String & name, const Context & context, bool reload = true) override; + + void createDictionary(const Context & context, + const String & dictionary_name, + const ASTPtr & query) override; + + void removeDictionary(const Context & context, const String & dictionary_name) override; StoragePtr tryGetTable(const Context & context, const String & table_name) const override; @@ -20,10 +26,14 @@ public: bool isDictionaryExist(const Context & context, const String & dictionary_name) const override; protected: - DatabaseWithDictionaries(String name) : DatabaseWithOwnTablesBase(std::move(name)) {} + DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & logger) + : DatabaseOnDisk(name, metadata_path_, logger) {} StoragePtr getDictionaryStorage(const Context & context, const String & table_name) const; + ASTPtr getCreateDictionaryQueryImpl(const Context & context, + const String & dictionary_name, + bool throw_on_error) const override; }; } diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 6230cdf3fdb..9c85d97f190 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -21,6 +21,11 @@ namespace ErrorCodes extern const int DICTIONARY_ALREADY_EXISTS; } +DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger) + : IDatabase(name_), log(&Logger::get(logger)) +{ +} + bool DatabaseWithOwnTablesBase::isTableExist( const Context & /*context*/, const String & table_name) const @@ -110,7 +115,7 @@ DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase() { try { - shutdown(); + DatabaseWithOwnTablesBase::shutdown(); } catch (...) { diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 38d5087e775..8dc4bf59985 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -44,8 +44,9 @@ protected: mutable std::mutex mutex; Tables tables; Dictionaries dictionaries; + Poco::Logger * log; - DatabaseWithOwnTablesBase(String name_) : IDatabase(std::move(name_)) { } + DatabaseWithOwnTablesBase(const String & name_, const String & logger); }; } diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 7ef53394f0f..fb80ece7482 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -235,7 +235,7 @@ public: } /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. - virtual time_t getObjectMetadataModificationTime(const Context & /*context*/, const String & /*name*/) + virtual time_t getObjectMetadataModificationTime(const String & /*name*/) const { return static_cast(0); } diff --git a/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp b/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp index bd89f27def1..3f5e271bd57 100644 --- a/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp +++ b/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp @@ -37,7 +37,7 @@ bool ExternalLoaderDatabaseConfigRepository::exists(const std::string & loadable Poco::Timestamp ExternalLoaderDatabaseConfigRepository::getUpdateTime(const std::string & loadable_definition_name) { - return database->getObjectMetadataModificationTime(context, trimDatabaseName(loadable_definition_name, database)); + return database->getObjectMetadataModificationTime(trimDatabaseName(loadable_definition_name, database)); } std::set ExternalLoaderDatabaseConfigRepository::getAllLoadablesDefinitionNames() const diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 01f8704f681..cfa417f24a3 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -258,7 +258,7 @@ protected: res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(context, table_name))); + res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); { Array dependencies_table_name_array; From 075edf03cabc33bf1ac7d985bd5c5247543baffc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 7 Nov 2019 17:52:12 +0300 Subject: [PATCH 010/743] fixes --- dbms/src/Databases/DatabaseLazy.h | 2 +- dbms/src/Databases/DatabaseWithDictionaries.cpp | 2 +- dbms/src/Storages/StorageFile.cpp | 4 ++-- dbms/src/Storages/StorageFile.h | 4 ++-- .../queries/0_stateless/01018_ddl_dictionaries_create.sql | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index 4ff77f279b6..0a4ca04a143 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -94,7 +94,7 @@ private: time_t metadata_modification_time; CacheExpirationQueue::iterator expiration_iterator; - CachedTable() {} + CachedTable() = delete; CachedTable(const StoragePtr & table_, time_t last_touched_, time_t metadata_modification_time_) : table(table_), last_touched(last_touched_), metadata_modification_time(metadata_modification_time_) {} }; diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index 445653b8f31..b536d8a21b1 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -52,7 +52,7 @@ void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name, } -void DatabaseWithDictionaries::createDictionary( const Context & context, const String & dictionary_name, const ASTPtr & query) +void DatabaseWithDictionaries::createDictionary(const Context & context, const String & dictionary_name, const ASTPtr & query) { const auto & settings = context.getSettingsRef(); diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 63742535fab..eb6e45ecc72 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -101,7 +101,7 @@ static std::string getTablePath(const std::string & table_dir_path, const std::s } /// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..'). -static void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path) +static void checkCreationIsAllowed(const Context & context_global, const std::string & db_dir_path, const std::string & table_path) { if (context_global.getApplicationType() != Context::ApplicationType::SERVER) return; @@ -358,7 +358,7 @@ void registerStorageFile(StorageFactory & factory) String format_name = engine_args[0]->as().value.safeGet(); StorageFile::CommonArguments common_args{args.database_name, args.table_name, format_name, - args.columns, args.constraints,args.context}; + args.columns, args.constraints, args.context}; if (engine_args.size() == 1) /// Table in database return StorageFile::create(args.relative_data_path, common_args); diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index e5fd8a40094..66e3760821e 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -49,7 +49,7 @@ public: const std::string & format_name; const ColumnsDescription & columns; const ConstraintsDescription & constraints; - Context & context; + const Context & context; }; protected: @@ -71,7 +71,7 @@ private: std::string table_name; std::string database_name; std::string format_name; - Context & context_global; + const Context & context_global; int table_fd = -1; diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql index 55d280cf045..16ca5412e32 100644 --- a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql @@ -91,7 +91,7 @@ CREATE DICTIONARY memory_db.dict2 PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) -LAYOUT(FLAT()); -- {serverError 1} +LAYOUT(FLAT()); -- {serverError 48} SHOW CREATE DICTIONARY memory_db.dict2; -- {serverError 487} @@ -116,7 +116,7 @@ CREATE DICTIONARY lazy_db.dict3 PRIMARY KEY key_column, second_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) -LAYOUT(COMPLEX_KEY_HASHED()); -- {serverError 1} +LAYOUT(COMPLEX_KEY_HASHED()); -- {serverError 48} DROP DATABASE IF EXISTS lazy_db; From ab4d4521452300e9a41a3eccca24b1e96a5ab92a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 11 Nov 2019 14:34:03 +0300 Subject: [PATCH 011/743] use UUIDs --- dbms/src/Databases/DatabaseAtomic.cpp | 57 +++++++++++++++++-- dbms/src/Databases/DatabaseAtomic.h | 23 ++++++-- dbms/src/Databases/DatabaseLazy.cpp | 4 +- dbms/src/Databases/DatabaseLazy.h | 2 +- dbms/src/Databases/DatabaseOnDisk.cpp | 42 +++++++++----- dbms/src/Databases/DatabaseOnDisk.h | 4 +- dbms/src/Databases/DatabaseOrdinary.cpp | 11 ++-- dbms/src/Databases/DatabasesCommon.cpp | 2 +- dbms/src/Databases/DatabasesCommon.h | 2 +- dbms/src/Databases/IDatabase.h | 7 ++- dbms/src/Interpreters/Context.cpp | 1 + .../Interpreters/InterpreterCreateQuery.cpp | 33 +++++++++-- .../src/Interpreters/InterpreterCreateQuery.h | 2 +- .../src/Interpreters/InterpreterDropQuery.cpp | 9 +-- .../InterpreterShowCreateQuery.cpp | 6 ++ .../Interpreters/InterpreterSystemQuery.cpp | 4 +- dbms/src/Parsers/ASTCreateQuery.cpp | 3 +- dbms/src/Parsers/ParserCreateQuery.cpp | 29 ++++++++-- dbms/src/Parsers/ParserCreateQuery.h | 4 +- dbms/src/Storages/StorageStripeLog.cpp | 7 +-- dbms/src/Storages/StorageTinyLog.cpp | 7 +-- .../Storages/System/attachSystemTables.cpp | 1 + .../queries/0_stateless/00116_storage_set.sql | 2 +- .../00180_attach_materialized_view.sql | 3 +- .../0_stateless/00311_array_primary_key.sql | 2 +- .../00423_storage_log_single_thread.sql | 6 +- 26 files changed, 197 insertions(+), 76 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index f45baaa213e..c0bbc7e78f0 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -1,21 +1,66 @@ #include #include +#include +#include namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_TABLE; + extern const int TABLE_ALREADY_EXISTS; +} DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Context & context_) : DatabaseOrdinary(name_, metadata_path_, context_) { + data_path = "store/"; } -//void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, -// const String & to_table_name, TableStructureWriteLockHolder & lock) -//{ -// //TODO -// DatabaseOnDisk::renameTable(*this, context, table_name, to_database, to_table_name, lock); -//} +String DatabaseAtomic::getDataPath(const String & table_name) const +{ + auto it = table_name_to_path.find(table_name); + if (it == table_name_to_path.end()) + throw Exception("Table " + table_name + " not found in database " + getDatabaseName(), ErrorCodes::UNKNOWN_TABLE); + return data_path + it->second; +} + +String DatabaseAtomic::getDataPath(const ASTCreateQuery & query) const +{ + stringToUUID(query.uuid); /// Check UUID is valid + const size_t uuid_prefix_len = 3; + return data_path + query.uuid.substr(0, uuid_prefix_len) + '/' + query.uuid + '/'; +} + +void DatabaseAtomic::drop(const Context &) +{ + Poco::File(getMetadataPath()).remove(false); +} + +void DatabaseAtomic::createTable(const Context & context, const String & table_name, const StoragePtr & table, + const ASTPtr & query) +{ + String relative_table_path = getDataPath(query->as()); + DatabaseOnDisk::createTable(context, table_name, table, query); + +} + +void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) +{ + DatabaseWithDictionaries::attachTable(name, table, relative_table_path); + std::lock_guard lock(mutex); + table_name_to_path.emplace(std::make_pair(table->getTableName(), relative_table_path)); +} + +StoragePtr DatabaseAtomic::detachTable(const String & name) +{ + { + std::lock_guard lock(mutex); + table_name_to_path.erase(name); + } + return DatabaseWithDictionaries::detachTable(name); +} } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index 8b6ea1075f0..8e1be19daf9 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -7,7 +7,7 @@ namespace DB { -class DatabaseAtomic : /* public DatabaseWithOwnTablesBase */ public DatabaseOrdinary +class DatabaseAtomic : public DatabaseOrdinary { public: @@ -15,12 +15,23 @@ public: String getEngineName() const override { return "Atomic"; } - //void renameTable(const Context & context, - // const String & table_name, - // IDatabase & to_database, - // const String & to_table_name, - // TableStructureWriteLockHolder &) override; + void createTable( + const Context & context, + const String & table_name, + const StoragePtr & table, + const ASTPtr & query) override; + void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path = {}) override; + StoragePtr detachTable(const String & name) override; + + String getDataPath(const String & table_name) const override; + String getDataPath(const ASTCreateQuery & query) const override; + + void drop(const Context & /*context*/) override; + +private: + //TODO store path in DatabaseWithOwnTables::tables + std::map table_name_to_path; }; diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 58619cd2993..fb3efaf95e4 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -160,7 +160,7 @@ bool DatabaseLazy::empty(const Context & /* context */) const return tables_cache.empty(); } -void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table) +void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table, const String &) { LOG_DEBUG(log, "Attach table " << backQuote(table_name) << "."); std::lock_guard lock(mutex); @@ -239,7 +239,7 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table if (ast) { auto & ast_create = ast->as(); - String table_data_path_relative = getDataPath() + escapeForFileName(ast_create.table) + '/'; + String table_data_path_relative = getDataPath(table_name); table = createTableFromAST(ast_create, database_name, table_data_path_relative, context_copy, false).second; } diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index 0a4ca04a143..66729fecd1d 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -66,7 +66,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override; - void attachTable(const String & table_name, const StoragePtr & table) override; + void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path = {}) override; StoragePtr detachTable(const String & table_name) override; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 2b82ff9f4ed..aede23ae225 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -185,7 +185,7 @@ void DatabaseOnDisk::createTable( try { /// Add a table to the map of known tables. - attachTable(table_name, table); + attachTable(table_name, table, getDataPath(query->as())); /// If it was ATTACH query and file with table metadata already exist /// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one. @@ -200,6 +200,8 @@ void DatabaseOnDisk::createTable( void DatabaseOnDisk::removeTable(const Context & /* context */, const String & table_name) { + String table_data = getDataPath(table_name); + StoragePtr res = detachTable(table_name); String table_metadata_path = getObjectMetadataPath(table_name); @@ -219,7 +221,7 @@ void DatabaseOnDisk::removeTable(const Context & /* context */, const String & t { LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); } - attachTable(table_name, res); + attachTable(table_name, res, data_path); throw; } } @@ -329,7 +331,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & table_na return static_cast(0); } -void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & iterating_function) const +void DatabaseOnDisk::iterateMetadataFiles(const Context & /*context*/, const IteratingFunction & iterating_function) const { Poco::DirectoryIterator dir_end; for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) @@ -346,17 +348,19 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati static const char * tmp_drop_ext = ".sql.tmp_drop"; if (endsWith(dir_it.name(), tmp_drop_ext)) { - const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext)); - if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists()) - { - Poco::File(dir_it->path()).renameTo(object_name + ".sql"); - LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously"); - } - else - { - LOG_INFO(log, "Removing file " << dir_it->path()); - Poco::File(dir_it->path()).remove(); - } + //const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext)); + + //FIXME + //if (Poco::File(context.getPath() + getDataPath(object_name)).exists()) + //{ + // Poco::File(dir_it->path()).renameTo(object_name + ".sql"); + // LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously"); + //} + //else + //{ + // LOG_INFO(log, "Removing file " << dir_it->path()); + // Poco::File(dir_it->path()).remove(); + //} continue; } @@ -428,4 +432,14 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metada return ast; } +String DatabaseOnDisk::getDataPath(const String & table_name) const +{ + return data_path + escapeForFileName(table_name) + "/"; +} + +String DatabaseOnDisk::getDataPath(const ASTCreateQuery & query) const +{ + return getDataPath(query.table); +} + } diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 9833b5135b4..36c33b1042e 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -63,6 +63,8 @@ public: time_t getObjectMetadataModificationTime(const String & object_name) const override; String getDataPath() const override { return data_path; } + String getDataPath(const String & table_name) const override; + String getDataPath(const ASTCreateQuery & query) const override; String getMetadataPath() const override { return metadata_path; } protected: @@ -80,7 +82,7 @@ protected: const String metadata_path; - const String data_path; + /*const*/ String data_path; }; } diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index ec4850bcd7c..66685dcd335 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -52,7 +52,7 @@ void loadObject( Context & context, const ASTCreateQuery & query, DatabaseOrdinary & database, - const String & database_data_path_relative, + const String & /*database_data_path_relative*/, const String & database_name, bool has_force_restore_data_flag) try @@ -64,9 +64,11 @@ try } else { - String table_data_path_relative = database_data_path_relative + escapeForFileName(query.table) + '/'; + //FIMXE + //String table_data_path_relative = database_data_path_relative + escapeForFileName(query.table) + '/'; + String table_data_path_relative = database.getDataPath(query); auto [table_name, table] = createTableFromAST(query, database_name, table_data_path_relative, context, has_force_restore_data_flag); - database.attachTable(table_name, table); + database.attachTable(table_name, table, database.getDataPath(query)); } } catch (const Exception & e) @@ -141,7 +143,8 @@ void DatabaseOrdinary::loadStoredObjects( auto loadOneObject = [&](const ASTCreateQuery & query) { - loadObject(context, query, *this, getDataPath(), getDatabaseName(), has_force_restore_data_flag); + //FIXME + loadObject(context, query, *this, getDataPath(query), getDatabaseName(), has_force_restore_data_flag); /// Messages, so that it's not boring to wait for the server to load for a long time. if (query.is_dictionary) diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 9c85d97f190..682b5c8b16e 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -83,7 +83,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name) return res; } -void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table) +void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table, const String &) { std::lock_guard lock(mutex); if (!tables.emplace(table_name, table).second) diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 8dc4bf59985..bca09eb8dcb 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -30,7 +30,7 @@ public: bool empty(const Context & context) const override; - void attachTable(const String & table_name, const StoragePtr & table) override; + void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path = {}) override; StoragePtr detachTable(const String & table_name) override; diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index fb80ece7482..d5b7847aacf 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -21,6 +21,7 @@ class ColumnsDescription; struct IndicesDescription; struct TableStructureWriteLockHolder; using Dictionaries = std::set; +class ASTCreateQuery; namespace ErrorCodes { @@ -184,7 +185,7 @@ public: } /// Add a table to the database, but do not add it to the metadata. The database may not support this method. - virtual void attachTable(const String & /*name*/, const StoragePtr & /*table*/) + virtual void attachTable(const String & /*name*/, const StoragePtr & /*table*/, [[maybe_unused]] const String & relative_table_path = {}) { throw Exception("There is no ATTACH TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } @@ -269,6 +270,10 @@ public: String getDatabaseName() const { return database_name; } /// Returns path for persistent data storage if the database supports it, empty string otherwise virtual String getDataPath() const { return {}; } + /// Returns path for persistent data storage for table if the database supports it, empty string otherwise + virtual String getDataPath(const String & /*table_name*/) const { return {}; } + /// Returns path for persistent data storage for CREATE/ATTACH query if the database supports it, empty string otherwise + virtual String getDataPath(const ASTCreateQuery & /*query*/) const { return {}; } /// Returns metadata path if the database supports it, empty string otherwise virtual String getMetadataPath() const { return {}; } /// Returns metadata path of a concrete table if the database supports it, empty string otherwise diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 3c89d029784..100f2d40d2c 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -514,6 +514,7 @@ DatabasePtr Context::tryGetDatabase(const String & database_name) String Context::getPath() const { + //FIXME there is no in clickhouse-local, so context->getPath() + relative_data_path may not work correctly (it may create StorageFile for example) auto lock = getLock(); return shared->path; } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 8c586b0d7c8..32dc935cad4 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -48,6 +48,8 @@ #include +#include +#include namespace DB { @@ -545,6 +547,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.to_table.empty()) databases.emplace(create.to_database); + /// NOTE: if it's CREATE query and create.database is DatabaseAtomic, different UUIDs will be generated on all servers. + /// However, it allows to use UUID as replica name. + return executeDDLQueryOnCluster(query_ptr, context, std::move(databases)); } @@ -587,7 +592,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) return fillTableIfNeeded(create, database_name); } -bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, +bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties, const String & database_name) { @@ -601,17 +606,33 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, if (need_add_to_database) { database = context.getDatabase(database_name); - if (!create.uuid.empty() && database->getEngineName() != "Atomic") - throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", - ErrorCodes::INCORRECT_QUERY); + if (database->getEngineName() == "Atomic") + { + //TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/ + if (create.attach && create.uuid.empty()) + throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY); + if (!create.attach && create.uuid.empty()) + create.uuid = boost::uuids::to_string(boost::uuids::random_generator()()); + } + else + { + if (!create.uuid.empty()) + throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", ErrorCodes::INCORRECT_QUERY); + } - data_path = database->getDataPath(); + if (!create.attach && create.uuid.empty() && database->getEngineName() == "Atomic") + create.uuid = boost::uuids::to_string(boost::uuids::random_generator()()); + + data_path = database->getDataPath(create); /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. */ guard = context.getDDLGuard(database_name, table_name); + if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists()) + throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS); + /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. if (database->isTableExist(context, table_name)) { @@ -646,7 +667,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, else { res = StorageFactory::instance().get(create, - data_path + escapeForFileName(table_name) + "/", + data_path, table_name, database_name, context, diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index fa0ea4380b9..af868b016ac 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -67,7 +67,7 @@ private: void checkAccess(const ASTCreateQuery & create); /// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false. - bool doCreateTable(const ASTCreateQuery & create, const TableProperties & properties, const String & database_name); + bool doCreateTable(/*const*/ ASTCreateQuery & create, const TableProperties & properties, const String & database_name); /// Inserts data in created table if it's CREATE ... SELECT BlockIO fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name); diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 17a36e00eb4..2f7240b5ddc 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -113,6 +113,7 @@ BlockIO InterpreterDropQuery::executeToTable( const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop"; /// Try to rename metadata file and delete the data + //TODO move this logic to DatabaseOnDisk try { /// There some kind of tables that have no metadata - ignore renaming @@ -128,16 +129,16 @@ BlockIO InterpreterDropQuery::executeToTable( throw; } + String table_data_path = database_and_table.first->getDataPath(table_name); + /// Delete table metadata and table itself from memory database_and_table.first->removeTable(context, database_and_table.second->getTableName()); database_and_table.second->is_dropped = true; - String database_data_path = context.getPath() + database_and_table.first->getDataPath(); - /// If it is not virtual database like Dictionary then drop remaining data dir - if (!database_data_path.empty()) + if (!table_data_path.empty()) { - String table_data_path = database_data_path + "/" + escapeForFileName(database_and_table.second->getTableName()); + table_data_path = context.getPath() + table_data_path; if (Poco::File(table_data_path).exists()) Poco::File(table_data_path).remove(true); diff --git a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp index 1bc253c8aaf..f1c0d8937ed 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -14,6 +14,8 @@ #include +#include + namespace DB { @@ -67,6 +69,10 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (!create_query && show_query->temporary) throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY); + //FIXME temporary print create query without UUID for tests (remove it) + auto & create = create_query->as(); + create.uuid.clear(); + std::stringstream stream; formatAST(*create_query, stream, false, true); String res = stream.str(); diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 772675889ee..7a866c525b8 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -285,12 +285,12 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam auto & create = create_ast->as(); create.attach = true; - std::string data_path = database->getDataPath(); + std::string data_path = database->getDataPath(create); auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context); auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints); StoragePtr table = StorageFactory::instance().get(create, - data_path + escapeForFileName(table_name) + "/", + data_path, table_name, database_name, system_context, diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index 8c3e9163caa..32ac43129f8 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -233,7 +233,8 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat << what << " " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") - << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table) + << (!uuid.empty() ? " UUID " + quoteString(uuid) : ""); formatOnCluster(settings); } else diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index be6eb4d7924..aa0af4ea739 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -401,7 +401,6 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (attach && s_uuid.ignore(pos, expected)) { - /// For CREATE query uuid will be generated if (!uuid_p.parse(pos, uuid, expected)) return false; } @@ -505,6 +504,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserKeyword s_uuid("UUID"); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_live("LIVE"); @@ -515,9 +515,11 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserIdentifier name_p; ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; + ParserStringLiteral uuid_p; ASTPtr database; ASTPtr table; + ASTPtr uuid; ASTPtr columns_list; ASTPtr to_database; ASTPtr to_table; @@ -563,6 +565,12 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e return false; } + if (attach && s_uuid.ignore(pos, expected)) + { + if (!uuid_p.parse(pos, uuid, expected)) + return false; + } + if (ParserKeyword{"ON"}.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) @@ -615,6 +623,8 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e tryGetIdentifierNameInto(to_database, query->to_database); tryGetIdentifierNameInto(to_table, query->to_table); + if (uuid) + query->uuid = uuid->as()->value.get(); query->set(query->columns_list, columns_list); @@ -687,6 +697,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserKeyword s_uuid("UUID"); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_materialized("MATERIALIZED"); @@ -700,9 +711,11 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; ParserNameList names_p; + ParserStringLiteral uuid_p; ASTPtr database; ASTPtr table; + ASTPtr uuid; ASTPtr columns_list; ASTPtr to_database; ASTPtr to_table; @@ -756,6 +769,12 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } + if (attach && s_uuid.ignore(pos, expected)) + { + if (!uuid_p.parse(pos, uuid, expected)) + return false; + } + if (ParserKeyword{"ON"}.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) @@ -816,6 +835,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec tryGetIdentifierNameInto(database, query->database); tryGetIdentifierNameInto(table, query->table); + if (uuid) + query->uuid = uuid->as()->value.get(); query->cluster = cluster_str; tryGetIdentifierNameInto(to_database, query->to_database); @@ -898,10 +919,8 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E query->is_dictionary = true; query->attach = attach; - if (database) - query->database = typeid_cast(*database).name; - - query->table = typeid_cast(*name).name; + tryGetIdentifierNameInto(database, query->database); + tryGetIdentifierNameInto(name, query->table); query->if_not_exists = if_not_exists; query->set(query->dictionary_attributes_list, attributes); diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 99e63058fdd..5aed3221af3 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -312,7 +312,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); }; -/// CREATE|ATTACH LIVE VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] AS SELECT ... +/// CREATE|ATTACH LIVE VIEW [IF NOT EXISTS] [db.]name [UUID 'uuid'] [TO [db.]name] AS SELECT ... class ParserCreateLiveViewQuery : public IParserBase { protected: @@ -328,7 +328,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); }; -/// CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... +/// CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [VIEW]] [IF NOT EXISTS] [db.]name [UUID 'uuid'] [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... class ParserCreateViewQuery : public IParserBase { protected: diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index b7ae8de92f4..f6e8259ac77 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -214,12 +214,7 @@ StorageStripeLog::StorageStripeLog( throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); if (!attach) - { - /// create files if they do not exist - if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST) - throwFromErrnoWithPath("Cannot create directory " + path, path, - ErrorCodes::CANNOT_CREATE_DIRECTORY); - } + Poco::File(path).createDirectories(); } diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index e62fab8c2b6..3cc64e11154 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -343,12 +343,7 @@ StorageTinyLog::StorageTinyLog( throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); if (!attach) - { - /// create files if they do not exist - if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST) - throwFromErrnoWithPath("Cannot create directory " + path, path, - ErrorCodes::CANNOT_CREATE_DIRECTORY); - } + Poco::File(path).createDirectories(); for (const auto & col : getColumns().getAllPhysical()) addFiles(col.name, *col.type); diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index 528bdd06a21..131c68aec6a 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -41,6 +41,7 @@ namespace DB { +//FIXME cannot store system tables in DatabaseAtomic void attachSystemTablesLocal(IDatabase & system_database) { system_database.attachTable("one", StorageSystemOne::create("one")); diff --git a/dbms/tests/queries/0_stateless/00116_storage_set.sql b/dbms/tests/queries/0_stateless/00116_storage_set.sql index aa93a0620d0..2b573c87f4a 100644 --- a/dbms/tests/queries/0_stateless/00116_storage_set.sql +++ b/dbms/tests/queries/0_stateless/00116_storage_set.sql @@ -19,7 +19,7 @@ INSERT INTO set2 VALUES ('abc'), ('World'); SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; DETACH TABLE set2; -ATTACH TABLE set2 (x String) ENGINE = Set; +ATTACH TABLE set2; -- (x String) ENGINE = Set; SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; diff --git a/dbms/tests/queries/0_stateless/00180_attach_materialized_view.sql b/dbms/tests/queries/0_stateless/00180_attach_materialized_view.sql index 089e4926bcf..0e20b15a438 100644 --- a/dbms/tests/queries/0_stateless/00180_attach_materialized_view.sql +++ b/dbms/tests/queries/0_stateless/00180_attach_materialized_view.sql @@ -6,7 +6,8 @@ CREATE TABLE t_00180 (x UInt8) ENGINE = Null; CREATE MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; DETACH TABLE mv_00180; -ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; +--ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; +ATTACH TABLE mv_00180; DROP TABLE t_00180; DROP TABLE mv_00180; diff --git a/dbms/tests/queries/0_stateless/00311_array_primary_key.sql b/dbms/tests/queries/0_stateless/00311_array_primary_key.sql index 0ea368609da..0598583c411 100644 --- a/dbms/tests/queries/0_stateless/00311_array_primary_key.sql +++ b/dbms/tests/queries/0_stateless/00311_array_primary_key.sql @@ -11,7 +11,7 @@ INSERT INTO array_pk VALUES ([5, 6], 'ghi', 6); SELECT * FROM array_pk ORDER BY n; DETACH TABLE array_pk; -ATTACH TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +ATTACH TABLE array_pk; -- (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); SELECT * FROM array_pk ORDER BY n; diff --git a/dbms/tests/queries/0_stateless/00423_storage_log_single_thread.sql b/dbms/tests/queries/0_stateless/00423_storage_log_single_thread.sql index 7d5e14c9ee5..512118dd55b 100644 --- a/dbms/tests/queries/0_stateless/00423_storage_log_single_thread.sql +++ b/dbms/tests/queries/0_stateless/00423_storage_log_single_thread.sql @@ -5,7 +5,7 @@ SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; -- (s String) ENGINE = Log; SELECT * FROM log; SELECT * FROM log LIMIT 1; @@ -15,13 +15,13 @@ INSERT INTO log VALUES ('Hello'), ('World'); SELECT * FROM log LIMIT 1; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; -- (s String) ENGINE = Log; SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; -- (s String) ENGINE = Log; SELECT * FROM log; SELECT * FROM log LIMIT 1; From cff9b02db27fab0593d06ad7da879e69483cacd9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 11 Nov 2019 17:28:28 +0300 Subject: [PATCH 012/743] refactor IStorage::rename(...) --- dbms/src/Databases/DatabaseAtomic.cpp | 26 +++++++++++++++++++ dbms/src/Databases/DatabaseAtomic.h | 7 +++++ dbms/src/Storages/IStorage.h | 10 +++---- dbms/src/Storages/Kafka/StorageKafka.cpp | 2 +- dbms/src/Storages/Kafka/StorageKafka.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 19 +++++--------- dbms/src/Storages/MergeTree/MergeTreeData.h | 8 +++++- dbms/src/Storages/StorageBuffer.h | 2 +- dbms/src/Storages/StorageDistributed.h | 3 ++- dbms/src/Storages/StorageFile.cpp | 4 +-- dbms/src/Storages/StorageFile.h | 8 +++++- dbms/src/Storages/StorageHDFS.cpp | 2 +- dbms/src/Storages/StorageHDFS.h | 2 +- dbms/src/Storages/StorageLog.cpp | 9 +++---- dbms/src/Storages/StorageLog.h | 9 ++++++- dbms/src/Storages/StorageMaterializedView.cpp | 3 +-- dbms/src/Storages/StorageMaterializedView.h | 2 +- dbms/src/Storages/StorageMemory.h | 2 +- dbms/src/Storages/StorageMerge.h | 2 +- dbms/src/Storages/StorageNull.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 4 +-- .../src/Storages/StorageReplicatedMergeTree.h | 2 +- dbms/src/Storages/StorageS3.cpp | 2 +- dbms/src/Storages/StorageS3.h | 2 +- dbms/src/Storages/StorageSet.cpp | 10 +++---- dbms/src/Storages/StorageSet.h | 9 ++++++- dbms/src/Storages/StorageStripeLog.cpp | 11 ++++---- dbms/src/Storages/StorageStripeLog.h | 9 ++++++- dbms/src/Storages/StorageTinyLog.cpp | 9 +++---- dbms/src/Storages/StorageTinyLog.h | 9 ++++++- dbms/src/Storages/StorageURL.cpp | 2 +- dbms/src/Storages/StorageURL.h | 2 +- dbms/src/Storages/StorageView.h | 2 +- 33 files changed, 131 insertions(+), 66 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index c0bbc7e78f0..8fdffd54623 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -10,6 +10,7 @@ namespace ErrorCodes { extern const int UNKNOWN_TABLE; extern const int TABLE_ALREADY_EXISTS; + extern const int FILE_DOESNT_EXIST; } DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Context & context_) @@ -62,6 +63,31 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) return DatabaseWithDictionaries::detachTable(name); } +void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, + const String & to_table_name, TableStructureWriteLockHolder &) +{ + //FIXME + if (typeid(*this) != typeid(to_database)) + throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); + + StoragePtr table = tryGetTable(context, table_name); + + if (!table) + throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + + /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. + table->renameInMemory(to_database.getDatabaseName(), to_table_name); + + ASTPtr ast = getQueryFromMetadata(getObjectMetadataPath(table_name)); + if (!ast) + throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST); + ast->as().table = to_table_name; + + /// NOTE Non-atomic. + to_database.createTable(context, to_table_name, table, ast); + removeTable(context, table_name); +} + } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index 8e1be19daf9..1f36083d3d0 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -21,6 +21,13 @@ public: const StoragePtr & table, const ASTPtr & query) override; + void renameTable( + const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name, + TableStructureWriteLockHolder & lock) override; + void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path = {}) override; StoragePtr detachTable(const String & name) override; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index e2a4d69d9a1..ced16263544 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -301,21 +301,21 @@ public: * In this function, you need to rename the directory with the data, if any. * Called when the table structure is locked for write. */ - virtual void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/, + virtual void rename(const String & /*new_path_to_table_data*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { - throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + renameInMemory(new_database_name, new_table_name); } - // TODO refactor rename() and renameAtomic() + // TODO refactor rename() and renameInMemory() /** * Just updates names of database and table without moving any data on disk * Can be called only from DatabaseAtomic. */ - virtual void renameAtomic(const String & /*new_database_name*/, const String & /*new_table_name*/) + virtual void renameInMemory(const String & /*new_database_name*/, const String & /*new_table_name*/) { - throw Exception("Method renameAtomic is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } /** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters. diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index f33a237a56a..02d59f42542 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -192,7 +192,7 @@ void StorageKafka::shutdown() } -void StorageKafka::rename(const String & /* new_path_to_db */, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageKafka::renameInMemory(const String & new_database_name, const String & new_table_name) { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index 492d1d65411..492ec987cb8 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -49,7 +49,7 @@ public: const ASTPtr & query, const Context & context) override; - void rename(const String & /* new_path_to_db */, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void renameInMemory(const String & new_database_name, const String & new_table_name) override; void updateDependencies() override; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 29939e5d12f..d07044915cf 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1177,17 +1177,14 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re } void MergeTreeData::rename( - const String & /*new_path_to_db*/, const String & new_database_name, + const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { - auto new_file_db_name = escapeForFileName(new_database_name); - auto new_file_table_name = escapeForFileName(new_table_name); - auto disks = storage_policy->getDisks(); for (const auto & disk : disks) { - auto new_full_path = disk->getClickHouseDataPath() + new_file_db_name + '/' + new_file_table_name + '/'; + auto new_full_path = disk->getPath() + new_path_to_table_data; if (Poco::File{new_full_path}.exists()) throw Exception{"Target path already exists: " + new_full_path, ErrorCodes::DIRECTORY_ALREADY_EXISTS}; @@ -1196,21 +1193,19 @@ void MergeTreeData::rename( for (const auto & disk : disks) { auto full_path = disk->getPath() + relative_data_path; - auto new_db_path = disk->getClickHouseDataPath() + new_file_db_name + '/'; + auto new_full_path = disk->getPath() + new_path_to_table_data; - Poco::File db_file{new_db_path}; - if (!db_file.exists()) - db_file.createDirectory(); + Poco::File new_full_path_parent{Poco::Path(new_full_path).makeParent()}; + if (!new_full_path_parent.exists()) + new_full_path_parent.createDirectories(); - auto new_full_path = new_db_path + new_file_table_name + '/'; Poco::File{full_path}.renameTo(new_full_path); } global_context.dropCaches(); database_name = new_database_name; - table_name = new_table_name; - relative_data_path = "data/" + new_file_db_name + '/' + new_file_table_name + '/'; + renameInMemory(new_database_name, new_table_name); } void MergeTreeData::dropAllData() diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index dcb80f0d1c8..2448769135f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -538,9 +538,15 @@ public: /// Moves the entire data directory. /// Flushes the uncompressed blocks cache and the marks cache. /// Must be called with locked lockStructureForAlter(). - void rename(const String & new_path_to_db, const String & new_database_name, + void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void renameInMemory(const String & new_database_name, const String & new_table_name) override + { + table_name = new_table_name; + database_name = new_database_name; + } + /// Check if the ALTER can be performed: /// - all needed columns are present. /// - all type conversions can be done. diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 1c565a7d8f0..10fd20aec4e 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -73,7 +73,7 @@ public: void shutdown() override; bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + void renameInMemory(const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index b6cf349c770..c8bef378d95 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -82,8 +82,9 @@ public: /// Removes temporary data in local filesystem. void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + void rename(const String & /*new_path_to_table_data*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override { + //TODO do we need no move data on disk and update path? table_name = new_table_name; database_name = new_database_name; } diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index eb6e45ecc72..438c48c3ef6 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -323,7 +323,7 @@ Strings StorageFile::getDataPaths() const return paths; } -void StorageFile::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageFile::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { if (!is_db_table) throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); @@ -333,7 +333,7 @@ void StorageFile::rename(const String & new_path_to_db, const String & new_datab std::unique_lock lock(rwlock); - std::string path_new = getTablePath(new_path_to_db + escapeForFileName(new_table_name), format_name); + std::string path_new = getTablePath(context_global.getPath() + new_path_to_table_data, format_name); Poco::File(Poco::Path(path_new).parent()).createDirectories(); Poco::File(paths[0]).renameTo(path_new); diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 66e3760821e..e8fd83a627d 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -38,7 +38,13 @@ public: const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + + void renameInMemory(const String & new_database_name, const String & new_table_name) override + { + table_name = new_table_name; + database_name = new_database_name; + } Strings getDataPaths() const override; diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 4b68bed871c..18ce7780faf 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -209,7 +209,7 @@ BlockInputStreams StorageHDFS::read( return result; } -void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageHDFS::renameInMemory(const String & new_database_name, const String & new_table_name) { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index 8361916e0e2..319914e78eb 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -30,7 +30,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void renameInMemory(const String & new_database_name, const String & new_table_name) override; protected: StorageHDFS(const String & uri_, diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 1fe41634239..ac6cd79b552 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -425,7 +425,7 @@ StorageLog::StorageLog( const ConstraintsDescription & constraints_, size_t max_compress_block_size_, const Context & context_) - : path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_), + : base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), file_checker(path + "sizes.json") { @@ -513,17 +513,16 @@ void StorageLog::loadMarks() } -void StorageLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { std::unique_lock lock(rwlock); /// Rename directory with data. - String new_path = new_path_to_db + escapeForFileName(new_table_name) + '/'; + String new_path = base_path + new_path_to_table_data; Poco::File(path).renameTo(new_path); path = new_path; - table_name = new_table_name; - database_name = new_database_name; + renameInMemory(new_database_name, new_table_name); file_checker.setPath(path + "sizes.json"); for (auto & file : files) diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index b911c5ab4fe..f52c0d804fb 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -38,7 +38,13 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + + void renameInMemory(const String & new_database_name, const String & new_table_name) override + { + table_name = new_table_name; + database_name = new_database_name; + } CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; @@ -63,6 +69,7 @@ protected: const Context & context_); private: + String base_path; String path; String table_name; String database_name; diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 24a6461bdab..6b3082b9efb 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -299,8 +299,7 @@ static void executeRenameQuery(Context & global_context, const String & database } -void StorageMaterializedView::rename( - const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name) { if (has_inner_table && tryGetTargetTable()) { diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 4ac608d96d2..7a5e2e7a528 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -43,7 +43,7 @@ public: void mutate(const MutationCommands & commands, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void renameInMemory(const String & new_database_name, const String & new_table_name) override; void shutdown() override; diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index eb2d6ff2e21..bc1d46c0017 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -44,7 +44,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + void renameInMemory(const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index debcb4da58e..b143bad901f 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -42,7 +42,7 @@ public: size_t max_block_size, unsigned num_streams) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + void renameInMemory(const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index e1a80f3fbaf..faffa88cebb 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -38,7 +38,7 @@ public: return std::make_shared(getSampleBlock()); } - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + void renameInMemory(const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 327ecf0d1b6..7e83bfe3440 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3746,10 +3746,10 @@ void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &) void StorageReplicatedMergeTree::rename( - const String & new_path_to_db, const String & new_database_name, + const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder & lock) { - MergeTreeData::rename(new_path_to_db, new_database_name, new_table_name, lock); + MergeTreeData::rename(new_path_to_table_data, new_database_name, new_table_name, lock); /// Update table name in zookeeper auto zookeeper = getZooKeeper(); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 0cc2b9930e2..8fab6aad094 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -117,7 +117,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; bool supportsIndexForIn() const override { return true; } diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 488a38c3e83..14560f77f38 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -164,7 +164,7 @@ BlockInputStreams StorageS3::read( return {std::make_shared(block_input, column_defaults, context)}; } -void StorageS3::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageS3::renameInMemory(const String & new_database_name, const String & new_table_name) { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index 65cd65458c6..fbd4c0c9c7d 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -51,7 +51,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void renameInMemory(const String & new_database_name, const String & new_table_name) override; private: Poco::URI uri; diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index 50f8773df90..037c307386d 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -101,7 +101,8 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( if (relative_path_.empty()) throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME); - path = context_.getPath() + relative_path_; + base_path = context_.getPath(); + path = base_path + relative_path_; } @@ -196,15 +197,14 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) void StorageSetOrJoinBase::rename( - const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) + const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { /// Rename directory with data. - String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/"; + String new_path = base_path + new_path_to_table_data; Poco::File(path).renameTo(new_path); path = new_path; - table_name = new_table_name; - database_name = new_database_name; + renameInMemory(new_database_name, new_table_name); } diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index e993039c304..b250e9f9836 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -22,7 +22,13 @@ public: String getTableName() const override { return table_name; } String getDatabaseName() const override { return database_name; } - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + + void renameInMemory(const String & new_database_name, const String & new_table_name) override + { + table_name = new_table_name; + database_name = new_database_name; + } BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; @@ -37,6 +43,7 @@ protected: const ConstraintsDescription & constraints_, const Context & context_); + String base_path; String path; String table_name; String database_name; diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index f6e8259ac77..b8e27bac89a 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -202,7 +202,7 @@ StorageStripeLog::StorageStripeLog( bool attach, size_t max_compress_block_size_, const Context & context_) - : path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_), + : base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), file_checker(path + "sizes.json"), log(&Logger::get("StorageStripeLog")) @@ -218,17 +218,16 @@ StorageStripeLog::StorageStripeLog( } -void StorageStripeLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageStripeLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { std::unique_lock lock(rwlock); /// Rename directory with data. - String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/"; - Poco::File(path).renameTo(new_path_to_db + escapeForFileName(new_table_name)); + String new_path = base_path + new_path_to_table_data; + Poco::File(path).renameTo(new_path); path = new_path; - table_name = new_table_name; - database_name = new_database_name; + renameInMemory(new_database_name, new_table_name); file_checker.setPath(path + "sizes.json"); } diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 4263ab14946..78f51264031 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -40,7 +40,13 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + + void renameInMemory(const String & new_database_name, const String & new_table_name) override + { + table_name = new_table_name; + database_name = new_database_name; + } CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; @@ -58,6 +64,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; private: + String base_path; String path; String table_name; String database_name; diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 3cc64e11154..ecb1496363e 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -331,7 +331,7 @@ StorageTinyLog::StorageTinyLog( bool attach, size_t max_compress_block_size_, const Context & context_) - : path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_), + : base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_), max_compress_block_size(max_compress_block_size_), file_checker(path + "sizes.json"), log(&Logger::get("StorageTinyLog")) @@ -373,17 +373,16 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type } -void StorageTinyLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageTinyLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { std::unique_lock lock(rwlock); /// Rename directory with data. - String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/"; + String new_path = base_path + new_path_to_table_data; Poco::File(path).renameTo(new_path); path = new_path; - table_name = new_table_name; - database_name = new_database_name; + renameInMemory(new_database_name, new_table_name); file_checker.setPath(path + "sizes.json"); for (Files_t::iterator it = files.begin(); it != files.end(); ++it) diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 8c33b9ff164..1be73181e05 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -39,7 +39,13 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + + void renameInMemory(const String & new_database_name, const String & new_table_name) override + { + table_name = new_table_name; + database_name = new_database_name; + } CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; @@ -57,6 +63,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; private: + String base_path; String path; String table_name; String database_name; diff --git a/dbms/src/Storages/StorageURL.cpp b/dbms/src/Storages/StorageURL.cpp index 70c401ba417..0832f56d2cf 100644 --- a/dbms/src/Storages/StorageURL.cpp +++ b/dbms/src/Storages/StorageURL.cpp @@ -186,7 +186,7 @@ BlockInputStreams IStorageURLBase::read(const Names & column_names, return {std::make_shared(block_input, column_defaults, context)}; } -void IStorageURLBase::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void IStorageURLBase::renameInMemory(const String & new_database_name, const String & new_table_name) { table_name = new_table_name; database_name = new_database_name; diff --git a/dbms/src/Storages/StorageURL.h b/dbms/src/Storages/StorageURL.h index cdd78c7b60f..102fb3d4425 100644 --- a/dbms/src/Storages/StorageURL.h +++ b/dbms/src/Storages/StorageURL.h @@ -29,7 +29,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void renameInMemory(const String & new_database_name, const String & new_table_name) override; protected: IStorageURLBase( diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index 6d2e1d04e6f..13c835f5364 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -30,7 +30,7 @@ public: size_t max_block_size, unsigned num_streams) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override + void renameInMemory(const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; From c0d4d19580a8c73b1f832548029c726bd46582c7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Nov 2019 21:46:02 +0300 Subject: [PATCH 013/743] fixes --- dbms/src/Databases/DatabaseOnDisk.cpp | 13 +++++++------ dbms/src/Storages/StorageDistributed.cpp | 2 +- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 95a541ceb38..77234ad4ee6 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -240,10 +240,16 @@ void DatabaseOnDisk::renameTable( if (!table) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + ASTPtr ast = getQueryFromMetadata(getObjectMetadataPath(table_name)); + if (!ast) + throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST); + auto & create = ast->as(); + create.table = to_table_name; + /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. try { - table->rename(context.getPath() + "/data/" + escapeForFileName(to_database.getDatabaseName()) + "/", + table->rename(to_database.getDataPath(create), to_database.getDatabaseName(), to_table_name, lock); } @@ -257,11 +263,6 @@ void DatabaseOnDisk::renameTable( throw Exception{Exception::CreateFromPoco, e}; } - ASTPtr ast = getQueryFromMetadata(getObjectMetadataPath(table_name)); - if (!ast) - throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST); - ast->as().table = to_table_name; - /// NOTE Non-atomic. to_database.createTable(context, to_table_name, table, ast); removeTable(context, table_name); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 13e5ce6aa08..3e824f6f487 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -472,7 +472,7 @@ void StorageDistributed::createDirectoryMonitors() if (path.empty()) return; - Poco::File{path}.createDirectory(); + Poco::File{path}.createDirectories(); std::filesystem::directory_iterator begin(path); std::filesystem::directory_iterator end; From 8cba35bbab50406363b6a13f5187d25414daa8a7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 26 Nov 2019 22:06:19 +0300 Subject: [PATCH 014/743] attempt to run tests --- dbms/src/Databases/DatabaseOnDisk.cpp | 14 ++++++- dbms/src/Databases/DatabaseOrdinary.cpp | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 40 +++++++++++-------- docker/test/performance/run.sh | 2 +- docker/test/stateful/Dockerfile | 2 +- docker/test/stress/Dockerfile | 2 +- 6 files changed, 40 insertions(+), 22 deletions(-) diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 77234ad4ee6..d5a734c0d98 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -20,6 +20,10 @@ #include +#include +#include +#include +#include namespace DB { @@ -232,8 +236,14 @@ void DatabaseOnDisk::renameTable( const String & to_table_name, TableStructureWriteLockHolder & lock) { + bool from_ordinary_to_atomic = false; if (typeid(*this) != typeid(to_database)) - throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); + { + if (typeid_cast(this) && typeid_cast(&to_database)) + from_ordinary_to_atomic = true; + else + throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); + } StoragePtr table = tryGetTable(context, table_name); @@ -245,6 +255,8 @@ void DatabaseOnDisk::renameTable( throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST); auto & create = ast->as(); create.table = to_table_name; + if (from_ordinary_to_atomic) + create.uuid = boost::uuids::to_string(boost::uuids::random_generator()()); /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. try diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 66685dcd335..62bde28dd17 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -64,7 +64,7 @@ try } else { - //FIMXE + //FIXME //String table_data_path_relative = database_data_path_relative + escapeForFileName(query.table) + '/'; String table_data_path_relative = database.getDataPath(query); auto [table_name, table] = createTableFromAST(query, database_name, table_data_path_relative, context, has_force_restore_data_flag); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 32dc935cad4..49fecc37c21 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -97,7 +97,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception("Database " + database_name + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS); } - String database_engine_name; if (!create.storage) { /// For new-style databases engine is explicitly specified in .sql @@ -105,27 +104,26 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) // FIXME maybe throw an exception if it's an ATTACH DATABASE query from user (it's not server startup) and engine is not specified bool old_style_database = create.attach || context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; - database_engine_name = old_style_database ? "Ordinary" : "Atomic"; auto engine = std::make_shared(); - engine->name = database_engine_name; auto storage = std::make_shared(); + engine->name = old_style_database ? "Ordinary" : "Atomic"; storage->set(storage->engine, engine); create.set(create.storage, storage); - } - else - { - const ASTStorage & storage = *create.storage; - const ASTFunction & engine = *storage.engine; - /// Currently, there are no database engines, that support any arguments. - if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) - { - std::stringstream ostr; - formatAST(storage, ostr, false, false); - throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); - } - database_engine_name = engine.name; + if (database_name == "datasets") + { + //FIXME it's just to run stateful and stress tests without updating docker images + engine->name = "Ordinary"; + } } + else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) + { + /// Currently, there are no database engines, that support any arguments. + std::stringstream ostr; + formatAST(*create.storage, ostr, false, false); + throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); + } + String database_name_escaped = escapeForFileName(database_name); @@ -162,19 +160,27 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) out.close(); } + bool added = false; + bool renamed = false; try { context.addDatabase(database_name, database); + added = true; if (need_write_metadata) + { Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path); + renamed = true; + } database->loadStoredObjects(context, has_force_restore_data_flag); } catch (...) { - if (need_write_metadata) + if (renamed) Poco::File(metadata_file_tmp_path).remove(); + if (added) + context.detachDatabase(database_name); throw; } diff --git a/docker/test/performance/run.sh b/docker/test/performance/run.sh index 2e149264ca0..a23c9bf9c23 100755 --- a/docker/test/performance/run.sh +++ b/docker/test/performance/run.sh @@ -11,7 +11,7 @@ install_packages() { } download_data() { - clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" + clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" /s3downloader --dataset-names $OPEN_DATASETS /s3downloader --dataset-names $PRIVATE_DATASETS --url 'https://s3.mds.yandex.net/clickhouse-private-datasets' diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index ba9d86c5ad4..1a16cc2f93a 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -39,7 +39,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ && /s3downloader --dataset-names $DATASETS \ && chmod 777 -R /var/lib/clickhouse \ && clickhouse-client --query "SHOW DATABASES" \ - && clickhouse-client --query "CREATE DATABASE datasets" \ + && clickhouse-client --query "CREATE DATABASE datasets ENGINE = Ordinary" \ && clickhouse-client --query "CREATE DATABASE test" \ && service clickhouse-server restart && sleep 5 \ && clickhouse-client --query "SHOW TABLES FROM datasets" \ diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index b0b94ccc579..2d607ee6cd9 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -39,7 +39,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ service clickhouse-server start && sleep 5 \ && /s3downloader --dataset-names $DATASETS \ && chmod 777 -R /var/lib/clickhouse \ - && clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" \ + && clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" \ && clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" \ && service clickhouse-server restart && sleep 5 \ && clickhouse-client --query "SHOW TABLES FROM datasets" \ From 80e082d3e26e242b57c92e0564e1b5d0e39c4f6a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 28 Nov 2019 22:40:51 +0300 Subject: [PATCH 015/743] try fix backward compatibility --- dbms/programs/server/Server.cpp | 6 +---- dbms/src/Databases/DatabaseAtomic.cpp | 1 + dbms/src/Databases/DatabaseLazy.cpp | 3 +-- dbms/src/Databases/DatabaseOnDisk.cpp | 10 +++++++ dbms/src/Databases/DatabaseOnDisk.h | 5 +--- dbms/src/Databases/DatabaseOrdinary.cpp | 3 +-- dbms/src/Databases/DatabaseWithDictionaries.h | 4 +-- .../Interpreters/InterpreterCreateQuery.cpp | 10 +++---- dbms/src/Interpreters/loadMetadata.cpp | 27 ++++++++++--------- dbms/src/Interpreters/loadMetadata.h | 2 +- docker/test/stateful/Dockerfile | 2 +- docker/test/stress/Dockerfile | 2 +- 12 files changed, 39 insertions(+), 36 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index bd820e5eb0a..7a30f43f928 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -268,10 +268,6 @@ int Server::main(const std::vector & /*args*/) global_context->setPath(path); - /// Create directories for 'path' and for default database, if not exist. - Poco::File(path + "data/" + default_database).createDirectories(); - Poco::File(path + "metadata/" + default_database).createDirectories(); - /// Check that we have read and write access to all data paths auto disk_selector = global_context->getDiskSelector(); for (const auto & [name, disk] : disk_selector.getDisksMap()) @@ -529,7 +525,7 @@ int Server::main(const std::vector & /*args*/) /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper); /// Then, load remaining databases - loadMetadata(*global_context); + loadMetadata(*global_context, default_database); } catch (...) { diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index 8fdffd54623..2e317bbb0af 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -17,6 +17,7 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Contex : DatabaseOrdinary(name_, metadata_path_, context_) { data_path = "store/"; + log = &Logger::get("DatabaseAtomic (" + name_ + ")"); } String DatabaseAtomic::getDataPath(const String & table_name) const diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index fb3efaf95e4..f853e36761a 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -30,10 +30,9 @@ namespace ErrorCodes DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_) - : DatabaseOnDisk(name_, metadata_path_, "DatabaseLazy (" + name_ + ")") + : DatabaseOnDisk(name_, metadata_path_, "DatabaseLazy (" + name_ + ")", context_) , expiration_time(expiration_time_) { - Poco::File(context_.getPath() + getDataPath()).createDirectories(); } diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index d5a734c0d98..9257ab17c83 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -142,6 +142,16 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) return statement_stream.str(); } +DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context & context_) + : DatabaseWithOwnTablesBase(name, logger) + , metadata_path(metadata_path_) + , data_path("data/" + escapeForFileName(database_name) + "/") +{ + Poco::File(context_.getPath() + getDataPath()).createDirectories(); + Poco::File(getMetadataPath()).createDirectories(); +} + + void DatabaseOnDisk::createTable( const Context & context, const String & table_name, diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 36c33b1042e..fa522ee8ef3 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -32,10 +32,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query); class DatabaseOnDisk : public DatabaseWithOwnTablesBase { public: - DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger) - : DatabaseWithOwnTablesBase(name, logger) - , metadata_path(metadata_path_) - , data_path("data/" + escapeForFileName(database_name) + "/") {} + DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context & context_); void createTable( const Context & context, diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 62bde28dd17..fa17376013b 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -93,9 +93,8 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context_) - : DatabaseWithDictionaries(name_, metadata_path_,"DatabaseOrdinary (" + name_ + ")") + : DatabaseWithDictionaries(name_, metadata_path_,"DatabaseOrdinary (" + name_ + ")", context_) { - Poco::File(context_.getPath() + getDataPath()).createDirectories(); } diff --git a/dbms/src/Databases/DatabaseWithDictionaries.h b/dbms/src/Databases/DatabaseWithDictionaries.h index 59e8023b730..bf900418c23 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.h +++ b/dbms/src/Databases/DatabaseWithDictionaries.h @@ -26,8 +26,8 @@ public: bool isDictionaryExist(const Context & context, const String & dictionary_name) const override; protected: - DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & logger) - : DatabaseOnDisk(name, metadata_path_, logger) {} + DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & logger, const Context & context_) + : DatabaseOnDisk(name, metadata_path_, logger, context_) {} StoragePtr getDictionaryStorage(const Context & context, const String & table_name) const; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 49fecc37c21..74c85c7fce9 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -101,9 +101,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) { /// For new-style databases engine is explicitly specified in .sql /// When attaching old-style database during server startup, we must always use Ordinary engine - // FIXME maybe throw an exception if it's an ATTACH DATABASE query from user (it's not server startup) and engine is not specified - bool old_style_database = create.attach || - context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; + //FIXME is it possible, that database engine is not specified in metadata file? + if (create.attach) + throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE); + bool old_style_database = context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; auto engine = std::make_shared(); auto storage = std::make_shared(); engine->name = old_style_database ? "Ordinary" : "Atomic"; @@ -126,11 +127,8 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) String database_name_escaped = escapeForFileName(database_name); - - /// Create directories for tables metadata. String path = context.getPath(); String metadata_path = path + "metadata/" + database_name_escaped + "/"; - Poco::File(metadata_path).createDirectory(); DatabasePtr database = DatabaseFactory::get(database_name, metadata_path, create.storage, context); diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 2a1563b7819..77c2eb7e08f 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -1,11 +1,6 @@ -#include -#include -#include - #include #include -#include #include #include @@ -21,7 +16,6 @@ #include #include -#include #include @@ -39,7 +33,6 @@ static void executeCreateQuery( ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0); auto & ast_create_query = ast->as(); - ast_create_query.attach = true; ast_create_query.database = database; InterpreterCreateQuery interpreter(ast, context); @@ -55,20 +48,27 @@ static void loadDatabase( const String & database_path, bool force_restore_data) { - /// There may exist .sql file with database creation statement. - /// Or, if it is absent, then database with Ordinary engine is created. String database_attach_query; String database_metadata_file = database_path + ".sql"; if (Poco::File(database_metadata_file).exists()) { + /// There are .sql file with database creation statement. ReadBufferFromFile in(database_metadata_file, 1024); readStringUntilEOF(database_attach_query, in); } + else if (Poco::File(database_path).exists()) + { + /// Database exists, but .sql file is absent. It's old-style Ordinary database (e.g. system or default) + database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Ordinary"; + } else - //FIXME - database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Atomic"; + { + /// It's first server run and we need create default and system databases. + /// .sql file with database engine will be written for CREATE query. + database_attach_query = "CREATE DATABASE " + backQuoteIfNeed(database); + } executeCreateQuery(database_attach_query, context, database, database_metadata_file, force_restore_data); @@ -78,7 +78,7 @@ static void loadDatabase( #define SYSTEM_DATABASE "system" -void loadMetadata(Context & context) +void loadMetadata(Context & context, const String & default_database_name) { String path = context.getPath() + "metadata"; @@ -108,6 +108,9 @@ void loadMetadata(Context & context) databases.emplace(unescapeForFileName(it.name()), it.path().toString()); } + if (!default_database_name.empty() && !databases.count(default_database_name)) + databases.emplace(default_database_name, path + "/metadata/" + escapeForFileName(default_database_name)); + for (const auto & [name, db_path] : databases) loadDatabase(context, name, db_path, has_force_restore_data_flag); diff --git a/dbms/src/Interpreters/loadMetadata.h b/dbms/src/Interpreters/loadMetadata.h index b7c78a169f5..b23887d5282 100644 --- a/dbms/src/Interpreters/loadMetadata.h +++ b/dbms/src/Interpreters/loadMetadata.h @@ -11,6 +11,6 @@ class Context; void loadMetadataSystem(Context & context); /// Load tables from databases and add them to context. Database 'system' is ignored. Use separate function to load system tables. -void loadMetadata(Context & context); +void loadMetadata(Context & context, const String & default_database_name = {}); } diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index 1a16cc2f93a..59e3c037265 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -39,7 +39,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ && /s3downloader --dataset-names $DATASETS \ && chmod 777 -R /var/lib/clickhouse \ && clickhouse-client --query "SHOW DATABASES" \ - && clickhouse-client --query "CREATE DATABASE datasets ENGINE = Ordinary" \ + && clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" \ && clickhouse-client --query "CREATE DATABASE test" \ && service clickhouse-server restart && sleep 5 \ && clickhouse-client --query "SHOW TABLES FROM datasets" \ diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index 2d607ee6cd9..116f4ec03f2 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -39,7 +39,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ service clickhouse-server start && sleep 5 \ && /s3downloader --dataset-names $DATASETS \ && chmod 777 -R /var/lib/clickhouse \ - && clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" \ + && clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" \ && clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" \ && service clickhouse-server restart && sleep 5 \ && clickhouse-client --query "SHOW TABLES FROM datasets" \ From 8bdc855e5030cd3951fbb55d0c5794c7ea22ef47 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 29 Nov 2019 02:36:17 +0300 Subject: [PATCH 016/743] fix --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- docker/test/stateful_with_coverage/run.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index de9f3a56a36..31ffeb94351 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1204,7 +1204,7 @@ void MergeTreeData::rename( global_context.dropCaches(); - database_name = new_database_name; + relative_data_path = new_path_to_table_data; renameInMemory(new_database_name, new_table_name); } diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index a7988736cb2..95b004b77f9 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -75,7 +75,7 @@ while /bin/true; do done & LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW DATABASES" -LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "CREATE DATABASE datasets" +LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "CREATE DATABASE test" kill_clickhouse From 268c8e7eb516452c281914de2122ff56bcbdb5e9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 2 Dec 2019 22:11:18 +0300 Subject: [PATCH 017/743] rename without table lock --- dbms/src/Databases/DatabaseAtomic.cpp | 34 ++++---- dbms/src/Databases/DatabaseAtomic.h | 9 +-- dbms/src/Databases/DatabaseLazy.cpp | 7 +- dbms/src/Databases/DatabaseLazy.h | 3 +- dbms/src/Databases/DatabaseOnDisk.cpp | 80 ++++++++++--------- dbms/src/Databases/DatabaseOnDisk.h | 9 +-- dbms/src/Databases/DatabaseOrdinary.cpp | 6 +- dbms/src/Databases/IDatabase.h | 3 +- .../Interpreters/InterpreterRenameQuery.cpp | 5 +- 9 files changed, 69 insertions(+), 87 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index 2e317bbb0af..1c653da76c8 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -22,6 +22,7 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Contex String DatabaseAtomic::getDataPath(const String & table_name) const { + std::lock_guard lock(mutex); auto it = table_name_to_path.find(table_name); if (it == table_name_to_path.end()) throw Exception("Table " + table_name + " not found in database " + getDatabaseName(), ErrorCodes::UNKNOWN_TABLE); @@ -40,19 +41,11 @@ void DatabaseAtomic::drop(const Context &) Poco::File(getMetadataPath()).remove(false); } -void DatabaseAtomic::createTable(const Context & context, const String & table_name, const StoragePtr & table, - const ASTPtr & query) -{ - String relative_table_path = getDataPath(query->as()); - DatabaseOnDisk::createTable(context, table_name, table, query); - -} - void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) { DatabaseWithDictionaries::attachTable(name, table, relative_table_path); std::lock_guard lock(mutex); - table_name_to_path.emplace(std::make_pair(table->getTableName(), relative_table_path)); + table_name_to_path.emplace(std::make_pair(name, relative_table_path)); } StoragePtr DatabaseAtomic::detachTable(const String & name) @@ -65,28 +58,29 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) } void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name, TableStructureWriteLockHolder &) + const String & to_table_name) { - //FIXME if (typeid(*this) != typeid(to_database)) - throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); + { + if (!typeid_cast(&to_database)) + throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); + /// Allow moving tables from Atomic to Ordinary (with table lock) + DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name); + return; + } StoragePtr table = tryGetTable(context, table_name); if (!table) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. + /// Update database and table name in memory without moving any data on disk table->renameInMemory(to_database.getDatabaseName(), to_table_name); - ASTPtr ast = getQueryFromMetadata(getObjectMetadataPath(table_name)); - if (!ast) - throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST); - ast->as().table = to_table_name; - /// NOTE Non-atomic. - to_database.createTable(context, to_table_name, table, ast); - removeTable(context, table_name); + to_database.attachTable(to_table_name, table, getDataPath(table_name)); + detachTable(table_name); + Poco::File(getObjectMetadataPath(table_name)).renameTo(to_database.getObjectMetadataPath(to_table_name)); } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index 1f36083d3d0..bce5b15a2b1 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -15,18 +15,11 @@ public: String getEngineName() const override { return "Atomic"; } - void createTable( - const Context & context, - const String & table_name, - const StoragePtr & table, - const ASTPtr & query) override; - void renameTable( const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder & lock) override; + const String & to_table_name) override; void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path = {}) override; StoragePtr detachTable(const String & name) override; diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index f853e36761a..738488aac41 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -78,11 +78,10 @@ void DatabaseLazy::renameTable( const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder & lock) + const String & to_table_name) { SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, lock); + DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name); } @@ -234,7 +233,7 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table StoragePtr table; Context context_copy(context); /// some tables can change context, but not LogTables - auto ast = parseCreateQueryFromMetadataFile(table_metadata_path, log); + auto ast = parseQueryFromMetadata(table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false); if (ast) { auto & ast_create = ast->as(); diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index 66729fecd1d..ae8a058c3ec 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -41,8 +41,7 @@ public: const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder &) override; + const String & to_table_name) override; void alterTable( const Context & context, diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 9257ab17c83..ed7453af663 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -29,6 +29,7 @@ namespace DB { static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; +static constexpr char const * TABLE_WITH_UUID_NAME_PLACEHOLDER = "_"; namespace ErrorCodes { @@ -42,32 +43,6 @@ namespace ErrorCodes } -ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log) -{ - String definition; - { - char in_buf[METADATA_FILE_BUFFER_SIZE]; - ReadBufferFromFile in(filepath, METADATA_FILE_BUFFER_SIZE, -1, in_buf); - readStringUntilEOF(definition, in); - } - - /** Empty files with metadata are generated after a rough restart of the server. - * Remove these files to slightly reduce the work of the admins on startup. - */ - if (definition.empty()) - { - LOG_ERROR(log, "File " << filepath << " is empty. Removing."); - Poco::File(filepath).remove(); - return nullptr; - } - - ParserCreateQuery parser_create; - ASTPtr result = parseQuery(parser_create, definition, "in file " + filepath, 0); - return result; -} - - - std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, @@ -86,7 +61,7 @@ std::pair createTableFromAST( return {ast_create_query.table, storage}; } /// We do not directly use `InterpreterCreateQuery::execute`, because - /// - the database has not been created yet; + /// - the database has not been loaded yet; /// - the code is simpler, since the query is already brought to a suitable form. if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns) throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); @@ -136,6 +111,9 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) create->format = nullptr; create->out_file = nullptr; + if (!create->uuid.empty()) + create->table = TABLE_WITH_UUID_NAME_PLACEHOLDER; + std::ostringstream statement_stream; formatAST(*create, statement_stream, false); statement_stream << '\n'; @@ -243,14 +221,16 @@ void DatabaseOnDisk::renameTable( const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder & lock) + const String & to_table_name) { bool from_ordinary_to_atomic = false; + bool from_atomic_to_ordinary = false; if (typeid(*this) != typeid(to_database)) { if (typeid_cast(this) && typeid_cast(&to_database)) from_ordinary_to_atomic = true; + else if (typeid_cast(this) && typeid_cast(&to_database)) + from_atomic_to_ordinary = true; else throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); } @@ -260,20 +240,22 @@ void DatabaseOnDisk::renameTable( if (!table) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - ASTPtr ast = getQueryFromMetadata(getObjectMetadataPath(table_name)); - if (!ast) - throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST); + auto table_lock = table->lockExclusively(context.getCurrentQueryId()); + + ASTPtr ast = parseQueryFromMetadata(getObjectMetadataPath(table_name)); auto & create = ast->as(); create.table = to_table_name; if (from_ordinary_to_atomic) create.uuid = boost::uuids::to_string(boost::uuids::random_generator()()); + if (from_atomic_to_ordinary) + create.uuid.clear(); /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. try { table->rename(to_database.getDataPath(create), to_database.getDatabaseName(), - to_table_name, lock); + to_table_name, table_lock); } catch (const Exception &) { @@ -320,8 +302,8 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const Context & /*context*/) const if (!ast) { /// Handle databases (such as default) for which there are no database.sql files. - //FIXME WTF - String query = "CREATE DATABASE " + backQuoteIfNeed(getDatabaseName()) + " ENGINE = Lazy"; + /// If database.sql doesn't exist, then engine is Ordinary + String query = "CREATE DATABASE " + backQuoteIfNeed(getDatabaseName()) + " ENGINE = Ordinary"; ParserCreateQuery parser; ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0); } @@ -411,13 +393,13 @@ String DatabaseOnDisk::getDatabaseMetadataPath(const String & base_path) const return (endsWith(base_path, "/") ? base_path.substr(0, base_path.size() - 1) : base_path) + ".sql"; } -ASTPtr DatabaseOnDisk::getQueryFromMetadata(const String & database_metadata_path, bool throw_on_error) const +ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const String & metadata_file_path, bool throw_on_error, bool remove_empty) const { String query; try { - ReadBufferFromFile in(database_metadata_path, 4096); + ReadBufferFromFile in(metadata_file_path, METADATA_FILE_BUFFER_SIZE); readStringUntilEOF(query, in); } catch (const Exception & e) @@ -428,6 +410,16 @@ ASTPtr DatabaseOnDisk::getQueryFromMetadata(const String & database_metadata_pat throw; } + /** Empty files with metadata are generated after a rough restart of the server. + * Remove these files to slightly reduce the work of the admins on startup. + */ + if (remove_empty && query.empty()) + { + LOG_ERROR(log, "File " << metadata_file_path << " is empty. Removing."); + Poco::File(metadata_file_path).remove(); + return nullptr; + } + ParserCreateQuery parser; const char * pos = query.data(); std::string error_message; @@ -436,13 +428,25 @@ ASTPtr DatabaseOnDisk::getQueryFromMetadata(const String & database_metadata_pat if (!ast && throw_on_error) throw Exception(error_message, ErrorCodes::SYNTAX_ERROR); + else if (!ast) + return nullptr; + + auto & create = ast->as(); + if (!create.uuid.empty()) + { + String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName(); + if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER) + LOG_WARNING(log, "File " << metadata_file_path << " contains both UUID and table name. " + "Will use name `" << table_name << "` instead of `" << create.table << "`"); + create.table = table_name; + } return ast; } ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metadata_path, bool throw_on_error) const { - ASTPtr ast = getQueryFromMetadata(database_metadata_path, throw_on_error); + ASTPtr ast = parseQueryFromMetadata(database_metadata_path, throw_on_error); if (ast) { diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index fa522ee8ef3..f7a56641fa0 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -11,8 +11,6 @@ namespace DB { -ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log); - std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, @@ -20,7 +18,7 @@ std::pair createTableFromAST( Context & context, bool has_force_restore_data_flag); -/** Get the row with the table definition based on the CREATE query. +/** Get the string with the table definition based on the CREATE query. * It is an ATTACH query that you can execute to create a table from the correspondent database. * See the implementation. */ @@ -48,8 +46,7 @@ public: const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name, - TableStructureWriteLockHolder & lock) override; + const String & to_table_name) override; ASTPtr getCreateDatabaseQuery(const Context & context) const override; @@ -74,7 +71,7 @@ protected: bool throw_on_error) const override; String getDatabaseMetadataPath(const String & base_path) const; - ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true) const; + ASTPtr parseQueryFromMetadata(const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false) const; ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const; diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index fa17376013b..6f8edb5dabd 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -52,7 +52,7 @@ void loadObject( Context & context, const ASTCreateQuery & query, DatabaseOrdinary & database, - const String & /*database_data_path_relative*/, + //const String & /*database_data_path_relative*/, const String & database_name, bool has_force_restore_data_flag) try @@ -115,7 +115,7 @@ void DatabaseOrdinary::loadStoredObjects( String full_path = metadata_path + "/" + file_name; try { - auto ast = parseCreateQueryFromMetadataFile(full_path, log); + auto ast = parseQueryFromMetadata(full_path, /*throw_on_error*/ true, /*remove_empty*/false); if (ast) { auto * create_query = ast->as(); @@ -143,7 +143,7 @@ void DatabaseOrdinary::loadStoredObjects( auto loadOneObject = [&](const ASTCreateQuery & query) { //FIXME - loadObject(context, query, *this, getDataPath(query), getDatabaseName(), has_force_restore_data_flag); + loadObject(context, query, *this, getDatabaseName(), has_force_restore_data_flag); /// Messages, so that it's not boring to wait for the server to load for a long time. if (query.is_dictionary) diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index d5b7847aacf..9a04b7a8257 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -214,8 +214,7 @@ public: const Context & /*context*/, const String & /*name*/, IDatabase & /*to_database*/, - const String & /*to_name*/, - TableStructureWriteLockHolder &) + const String & /*to_name*/) { throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index 3e8bc3a88fb..40cab894a2f 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -96,15 +96,12 @@ BlockIO InterpreterRenameQuery::execute() for (auto & elem : descriptions) { context.assertTableDoesntExist(elem.to_database_name, elem.to_table_name); - auto from_table = context.getTable(elem.from_database_name, elem.from_table_name); - auto from_table_lock = from_table->lockExclusively(context.getCurrentQueryId()); context.getDatabase(elem.from_database_name)->renameTable( context, elem.from_table_name, *context.getDatabase(elem.to_database_name), - elem.to_table_name, - from_table_lock); + elem.to_table_name); } return {}; From b047e7b8c44b004b7990bd76a5ea62f205902a4d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 3 Dec 2019 19:25:32 +0300 Subject: [PATCH 018/743] refactor storage name --- dbms/src/Interpreters/Context.h | 1 + .../tests/in_join_subqueries_preprocessor.cpp | 5 +- .../Storages/Distributed/DirectoryMonitor.cpp | 2 +- dbms/src/Storages/IStorage.cpp | 24 +++++++- dbms/src/Storages/IStorage.h | 58 +++++++++++++++---- dbms/src/Storages/Kafka/StorageKafka.cpp | 32 +++++----- dbms/src/Storages/Kafka/StorageKafka.h | 8 +-- dbms/src/Storages/LiveView/ProxyStorage.h | 3 +- .../src/Storages/LiveView/StorageLiveView.cpp | 21 ++++--- dbms/src/Storages/LiveView/StorageLiveView.h | 4 -- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 25 ++++---- dbms/src/Storages/MergeTree/MergeTreeData.h | 11 ---- .../ReplicatedMergeTreeAlterThread.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 5 +- dbms/src/Storages/StorageBuffer.cpp | 6 +- dbms/src/Storages/StorageBuffer.h | 11 ---- dbms/src/Storages/StorageDictionary.cpp | 3 +- dbms/src/Storages/StorageDictionary.h | 4 -- dbms/src/Storages/StorageDistributed.cpp | 12 ++-- dbms/src/Storages/StorageDistributed.h | 7 +-- dbms/src/Storages/StorageFile.cpp | 14 ++--- dbms/src/Storages/StorageFile.h | 10 ---- dbms/src/Storages/StorageHDFS.cpp | 11 +--- dbms/src/Storages/StorageHDFS.h | 6 -- dbms/src/Storages/StorageInput.cpp | 2 +- dbms/src/Storages/StorageInput.h | 2 - dbms/src/Storages/StorageJoin.cpp | 2 +- dbms/src/Storages/StorageLog.cpp | 4 +- dbms/src/Storages/StorageLog.h | 11 ---- dbms/src/Storages/StorageMaterializedView.cpp | 30 +++++----- dbms/src/Storages/StorageMaterializedView.h | 6 +- dbms/src/Storages/StorageMemory.cpp | 2 +- dbms/src/Storages/StorageMemory.h | 11 ---- dbms/src/Storages/StorageMerge.cpp | 7 +-- dbms/src/Storages/StorageMerge.h | 10 ---- dbms/src/Storages/StorageMergeTree.cpp | 13 +++-- dbms/src/Storages/StorageMergeTree.h | 2 - dbms/src/Storages/StorageMySQL.cpp | 3 +- dbms/src/Storages/StorageMySQL.h | 4 -- dbms/src/Storages/StorageNull.cpp | 6 +- dbms/src/Storages/StorageNull.h | 12 +--- .../Storages/StorageReplicatedMergeTree.cpp | 38 +++++++----- .../src/Storages/StorageReplicatedMergeTree.h | 2 - dbms/src/Storages/StorageS3.cpp | 10 +--- dbms/src/Storages/StorageS3.h | 9 --- dbms/src/Storages/StorageSet.cpp | 2 +- dbms/src/Storages/StorageSet.h | 11 ---- dbms/src/Storages/StorageStripeLog.cpp | 5 +- dbms/src/Storages/StorageStripeLog.h | 10 ---- dbms/src/Storages/StorageTinyLog.cpp | 4 +- dbms/src/Storages/StorageTinyLog.h | 11 ---- dbms/src/Storages/StorageURL.cpp | 8 +-- dbms/src/Storages/StorageURL.h | 7 --- dbms/src/Storages/StorageValues.cpp | 2 +- dbms/src/Storages/StorageValues.h | 4 -- dbms/src/Storages/StorageView.cpp | 2 +- dbms/src/Storages/StorageView.h | 10 ---- .../Storages/System/IStorageSystemOneBlock.h | 8 +-- .../Storages/System/StorageSystemColumns.cpp | 2 +- .../Storages/System/StorageSystemColumns.h | 5 -- .../System/StorageSystemDetachedParts.cpp | 3 +- .../Storages/System/StorageSystemDisks.cpp | 2 +- dbms/src/Storages/System/StorageSystemDisks.h | 5 -- .../Storages/System/StorageSystemNumbers.cpp | 2 +- .../Storages/System/StorageSystemNumbers.h | 3 - dbms/src/Storages/System/StorageSystemOne.cpp | 2 +- dbms/src/Storages/System/StorageSystemOne.h | 5 -- .../System/StorageSystemPartsBase.cpp | 2 +- .../Storages/System/StorageSystemPartsBase.h | 5 -- .../Storages/System/StorageSystemReplicas.cpp | 2 +- .../Storages/System/StorageSystemReplicas.h | 5 -- .../System/StorageSystemStoragePolicies.cpp | 2 +- .../System/StorageSystemStoragePolicies.h | 5 -- .../Storages/System/StorageSystemTables.cpp | 2 +- .../src/Storages/System/StorageSystemTables.h | 5 -- 78 files changed, 223 insertions(+), 395 deletions(-) diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index e6d0a7319af..f00fd8046be 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -88,6 +88,7 @@ class CompiledExpressionCache; #endif /// (database name, table name) +//FIXME replace with StorageID using DatabaseAndTableName = std::pair; /// Table -> set of table-views that make SELECT from it. diff --git a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index 43624bf16b8..d3a7b819cc0 100644 --- a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -38,12 +38,9 @@ public: std::string getRemoteDatabaseName() const { return remote_database; } std::string getRemoteTableName() const { return remote_table; } - std::string getTableName() const override { return ""; } - std::string getDatabaseName() const override { return ""; } - protected: StorageDistributedFake(const std::string & remote_database_, const std::string & remote_table_, size_t shard_count_) - : remote_database(remote_database_), remote_table(remote_table_), shard_count(shard_count_) + : IStorage({"", ""}), remote_database(remote_database_), remote_table(remote_table_), shard_count(shard_count_) { } diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 5a6f13b0567..134f9cecbc7 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -612,7 +612,7 @@ bool StorageDistributedDirectoryMonitor::maybeMarkAsBroken(const std::string & f std::string StorageDistributedDirectoryMonitor::getLoggerName() const { - return storage.table_name + '.' + storage.getName() + ".DirectoryMonitor"; + return storage.getStorageID().getFullTableName() + ".DirectoryMonitor"; } } diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index f5c34587fb2..4ab4e149cdd 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; } -IStorage::IStorage(ColumnsDescription virtuals_) : virtuals(std::move(virtuals_)) +IStorage::IStorage(StorageID id_, ColumnsDescription virtuals_) : id(std::move(id_)), virtuals(std::move(virtuals_)) { } @@ -463,4 +463,26 @@ DB::CompressionMethod IStorage::chooseCompressionMethod(const String & uri, cons throw Exception("Only auto, none, gzip supported as compression method", ErrorCodes::NOT_IMPLEMENTED); } +StorageID IStorage::getStorageID(std::unique_lock * id_lock) const +{ + std::unique_lock lock; + if (!id_lock) + lock = std::unique_lock(id_mutex); + else if (!*id_lock) + *id_lock = std::unique_lock(id_mutex); + return id; +} + +void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name, + std::unique_lock * id_lock) +{ + std::unique_lock lock; + if (!id_lock) + lock = std::unique_lock(id_mutex); + else if (!*id_lock) + *id_lock = std::unique_lock(id_mutex); + id.database_name = new_database_name; + id.table_name = new_table_name; +} + } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index fe08047b96b..d4d757f0111 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -65,6 +65,35 @@ struct ColumnSize } }; +struct StorageID +{ + String database_name; + String table_name; + String uuid; + + StorageID() = delete; + StorageID(const String & database, const String & table, const String & uuid_ = {}) + : database_name(database), table_name(table), uuid(uuid_) {} + + String getFullTableName() const + { + return (database_name.empty() ? "" : database_name + ".") + table_name; + } + + String getNameForLogs() const + { + return "`" + getFullTableName() + "` (UUID = " + uuid +")"; + } + + String getId() const + { + //if (uuid.empty()) + return getFullTableName(); + //else + // return uuid; + } +}; + /** Storage. Describes the table. Responsible for * - storage of the table data; * - the definition in which files (or not in files) the data is stored; @@ -75,8 +104,9 @@ struct ColumnSize class IStorage : public std::enable_shared_from_this, public TypePromotion { public: - IStorage() = default; - explicit IStorage(ColumnsDescription virtuals_); + IStorage() = delete; + explicit IStorage(StorageID id_) : id(std::move(id_)) {} + IStorage(StorageID id_, ColumnsDescription virtuals_); virtual ~IStorage() = default; IStorage(const IStorage &) = delete; @@ -86,8 +116,19 @@ public: virtual std::string getName() const = 0; /// The name of the table. - virtual std::string getTableName() const = 0; - virtual std::string getDatabaseName() const { return {}; } + StorageID getStorageID(std::unique_lock * lock = nullptr) const; + + // FIXME remove those methods + std::string getTableName() const + { + std::lock_guard lock(id_mutex); + return id.table_name; + } + std::string getDatabaseName() const + { + std::lock_guard lock(id_mutex); + return id.database_name; + } /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } @@ -166,6 +207,8 @@ protected: /// still thread-unsafe part. IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; private: + StorageID id; + mutable std::mutex id_mutex; ColumnsDescription columns; /// combined real and virtual columns const ColumnsDescription virtuals = {}; IndicesDescription indices; @@ -310,16 +353,11 @@ public: renameInMemory(new_database_name, new_table_name); } - // TODO refactor rename() and renameInMemory() - /** * Just updates names of database and table without moving any data on disk * Can be called only from DatabaseAtomic. */ - virtual void renameInMemory(const String & /*new_database_name*/, const String & /*new_table_name*/) - { - throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } + virtual void renameInMemory(const String & new_database_name, const String & new_table_name, std::unique_lock * id_lock = nullptr); /** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters. * This method must fully execute the ALTER query, taking care of the locks itself. diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 02d59f42542..d15ace31426 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -88,14 +88,12 @@ StorageKafka::StorageKafka( UInt64 max_block_size_, size_t skip_broken_, bool intermediate_commit_) - : IStorage( + : IStorage({database_name_, table_name_}, ColumnsDescription({{"_topic", std::make_shared()}, {"_key", std::make_shared()}, {"_offset", std::make_shared()}, {"_partition", std::make_shared()}, {"_timestamp", std::make_shared(std::make_shared())}}, true)) - , table_name(table_name_) - , database_name(database_name_) , global_context(context_) , topics(global_context.getMacros()->expand(topics_)) , brokers(global_context.getMacros()->expand(brokers_)) @@ -192,13 +190,6 @@ void StorageKafka::shutdown() } -void StorageKafka::renameInMemory(const String & new_database_name, const String & new_table_name) -{ - table_name = new_table_name; - database_name = new_database_name; -} - - void StorageKafka::updateDependencies() { task->activateAndSchedule(); @@ -299,10 +290,10 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & conf) } } -bool StorageKafka::checkDependencies(const String & current_database_name, const String & current_table_name) +bool StorageKafka::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached - auto dependencies = global_context.getDependencies(current_database_name, current_table_name); + auto dependencies = global_context.getDependencies(table_id.database_name, table_id.database_name); //FIXME replace with id if (dependencies.size() == 0) return true; @@ -319,7 +310,7 @@ bool StorageKafka::checkDependencies(const String & current_database_name, const return false; // Check all its dependencies - if (!checkDependencies(db_tab.first, db_tab.second)) + if (!checkDependencies(StorageID(db_tab.first, db_tab.second))) //FIXME replace with id return false; } @@ -330,13 +321,14 @@ void StorageKafka::threadFunc() { try { + auto table_id = getStorageID(); // Check if at least one direct dependency is attached - auto dependencies = global_context.getDependencies(database_name, table_name); + auto dependencies = global_context.getDependencies(table_id.database_name, table_id.database_name); //FIXME replace with id // Keep streaming as long as there are attached views and streaming is not cancelled while (!stream_cancelled && num_created_consumers > 0 && dependencies.size() > 0) { - if (!checkDependencies(database_name, table_name)) + if (!checkDependencies(table_id)) break; LOG_DEBUG(log, "Started streaming to " << dependencies.size() << " attached views"); @@ -359,14 +351,16 @@ void StorageKafka::threadFunc() bool StorageKafka::streamToViews() { - auto table = global_context.getTable(database_name, table_name); + auto table_id = getStorageID(); + auto table = global_context.getTable(table_id.database_name, table_id.table_name); if (!table) - throw Exception("Engine table " + database_name + "." + table_name + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); // Create an INSERT query for streaming data auto insert = std::make_shared(); - insert->database = database_name; - insert->table = table_name; + //FIXME use uid if not empty + insert->database = table_id.database_name; + insert->table = table_id.table_name; const Settings & settings = global_context.getSettingsRef(); size_t block_size = max_block_size; diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index 492ec987cb8..2b66a6f6248 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -28,8 +28,6 @@ class StorageKafka : public ext::shared_ptr_helper, public IStorag friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Kafka"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } bool supportsSettings() const override { return true; } bool noPushingToViews() const override { return true; } @@ -49,8 +47,6 @@ public: const ASTPtr & query, const Context & context) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override; - void updateDependencies() override; void pushReadBuffer(ConsumerBufferPtr buf); @@ -79,8 +75,6 @@ protected: private: // Configuration and state - String table_name; - String database_name; Context global_context; Names topics; const String brokers; @@ -117,7 +111,7 @@ private: void threadFunc(); bool streamToViews(); - bool checkDependencies(const String & database_name, const String & table_name); + bool checkDependencies(const StorageID & table_id); }; } diff --git a/dbms/src/Storages/LiveView/ProxyStorage.h b/dbms/src/Storages/LiveView/ProxyStorage.h index 60faa907209..df8aade6540 100644 --- a/dbms/src/Storages/LiveView/ProxyStorage.h +++ b/dbms/src/Storages/LiveView/ProxyStorage.h @@ -9,11 +9,10 @@ class ProxyStorage : public IStorage { public: ProxyStorage(StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) - : storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {} + : IStorage({"", storage_->getTableName()}), storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {} public: std::string getName() const override { return "ProxyStorage(" + storage->getName() + ")"; } - std::string getTableName() const override { return storage->getTableName(); } bool isRemote() const override { return storage->isRemote(); } bool supportsSampling() const override { return storage->supportsSampling(); } diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 06340dfac38..7e6e5edfa58 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -198,8 +198,7 @@ StorageLiveView::StorageLiveView( Context & local_context, const ASTCreateQuery & query, const ColumnsDescription & columns_) - : table_name(table_name_), - database_name(database_name_), global_context(local_context.getGlobalContext()) + : IStorage({database_name_, table_name_}), global_context(local_context.getGlobalContext()) { setColumns(columns_); @@ -225,7 +224,7 @@ StorageLiveView::StorageLiveView( global_context.addDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name, table_name)); + DatabaseAndTableName(database_name_, table_name_)); //FIXME is_temporary = query.temporary; temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds(); @@ -339,7 +338,8 @@ bool StorageLiveView::getNewBlocks() void StorageLiveView::checkTableCanBeDropped() const { - Dependencies dependencies = global_context.getDependencies(database_name, table_name); + auto table_id = getStorageID(); + Dependencies dependencies = global_context.getDependencies(table_id.database_name, table_id.table_name); //FIXME if (!dependencies.empty()) { DatabaseAndTableName database_and_table_name = dependencies.front(); @@ -354,6 +354,7 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co if (storage->shutdown_called) return; + auto table_id = storage->getStorageID(); { while (1) { @@ -365,7 +366,7 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co return; if (storage->hasUsers()) return; - if (!storage->global_context.getDependencies(storage->database_name, storage->table_name).empty()) + if (!storage->global_context.getDependencies(table_id.database_name, table_id.table_name).empty()) //FIXME continue; drop_table = true; } @@ -375,14 +376,14 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co if (drop_table) { - if (storage->global_context.tryGetTable(storage->database_name, storage->table_name)) + if (storage->global_context.tryGetTable(table_id.database_name, table_id.table_name)) //FIXME { 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); @@ -390,6 +391,7 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co } catch (...) { + tryLogCurrentException(__PRETTY_FUNCTION__); } } } @@ -466,9 +468,10 @@ StorageLiveView::~StorageLiveView() void StorageLiveView::drop(TableStructureWriteLockHolder &) { + auto table_id = getStorageID(); global_context.removeDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name, table_name)); + DatabaseAndTableName(table_id.database_name, table_id.table_name)); //FIXME std::lock_guard lock(mutex); is_dropped = true; diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 3f1dffb898c..54722a22ff9 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -41,8 +41,6 @@ friend class LiveViewBlockOutputStream; public: ~StorageLiveView() override; String getName() const override { return "LiveView"; } - 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; } @@ -144,8 +142,6 @@ public: private: String select_database_name; String select_table_name; - String table_name; - String database_name; ASTPtr inner_query; Context & global_context; bool is_temporary = false; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index f8f26c6419f..492ac4b8cd4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -115,17 +115,16 @@ MergeTreeData::MergeTreeData( bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_) - : global_context(context_) + : IStorage({database_, table_}) + , global_context(context_) , merging_params(merging_params_) , partition_by_ast(partition_by_ast_) , sample_by_ast(sample_by_ast_) , ttl_table_ast(ttl_table_ast_) , require_part_metadata(require_part_metadata_) - , database_name(database_) - , table_name(table_) , relative_data_path(relative_data_path_) , broken_part_callback(broken_part_callback_) - , log_name(database_name + "." + table_name) + , log_name(database_ + "." + table_) , log(&Logger::get(log_name)) , storage_settings(std::move(storage_settings_)) , storage_policy(context_.getStoragePolicy(getSettings()->storage_policy)) @@ -1101,7 +1100,9 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa /// Data parts is still alive (since DataPartsVector holds shared_ptrs) and contain useful metainformation for logging /// NOTE: There is no need to log parts deletion somewhere else, all deleting parts pass through this function and pass away - if (auto part_log = global_context.getPartLog(database_name)) + + auto table_id = getStorageID(); + if (auto part_log = global_context.getPartLog(table_id.database_name)) //FIXME { PartLogElement part_log_elem; @@ -1109,8 +1110,8 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa part_log_elem.event_time = time(nullptr); part_log_elem.duration_ms = 0; - part_log_elem.database_name = database_name; - part_log_elem.table_name = table_name; + part_log_elem.database_name = table_id.database_name; + part_log_elem.table_name = table_id.table_name; for (auto & part : parts) { @@ -3222,7 +3223,8 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePt { MergeTreeData * src_data = dynamic_cast(source_table.get()); if (!src_data) - throw Exception("Table " + table_name + " supports attachPartitionFrom only for MergeTree family of table engines." + throw Exception("Table " + source_table->getStorageID().getNameForLogs() + + " supports attachPartitionFrom only for MergeTree family of table engines." " Got " + source_table->getName(), ErrorCodes::NOT_IMPLEMENTED); if (getColumns().getAllPhysical().sizeOfDifference(src_data->getColumns().getAllPhysical())) @@ -3390,7 +3392,8 @@ void MergeTreeData::writePartLog( const MergeListEntry * merge_entry) try { - auto part_log = global_context.getPartLog(database_name); + auto table_id = getStorageID(); + auto part_log = global_context.getPartLog(table_id.database_name); if (!part_log) return; @@ -3405,8 +3408,8 @@ try /// TODO: Stop stopwatch in outer code to exclude ZK timings and so on part_log_elem.duration_ms = elapsed_ns / 1000000; - part_log_elem.database_name = database_name; - part_log_elem.table_name = table_name; + part_log_elem.database_name = table_id.database_name; + part_log_elem.table_name = table_id.table_name; part_log_elem.partition_id = MergeTreePartInfo::fromPartName(new_part_name, format_version).partition_id; part_log_elem.part_name = new_part_name; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 11c942beb34..50779a734b3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -402,9 +402,6 @@ public: || column_name == "_sample_factor"; } - String getDatabaseName() const override { return database_name; } - String getTableName() const override { return table_name; } - /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); @@ -541,12 +538,6 @@ public: void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - /// Check if the ALTER can be performed: /// - all needed columns are present. /// - all type conversions can be done. @@ -771,8 +762,6 @@ protected: bool require_part_metadata; - String database_name; - String table_name; String relative_data_path; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index c639275158f..cae520781ec 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -25,7 +25,7 @@ static const auto ALTER_ERROR_SLEEP_MS = 10 * 1000; ReplicatedMergeTreeAlterThread::ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , zk_node_cache([&] { return storage.getZooKeeper(); }) - , log_name(storage.database_name + "." + storage.table_name + " (ReplicatedMergeTreeAlterThread)") + , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeAlterThread)") , log(&Logger::get(log_name)) { task = storage_.global_context.getSchedulePool().createTask(log_name, [this]{ run(); }); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index b89fd9629c0..c4d55f2e26f 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -19,7 +19,7 @@ namespace ErrorCodes ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplicatedMergeTree & storage_) : storage(storage_) - , log_name(storage.database_name + "." + storage.table_name + " (ReplicatedMergeTreeCleanupThread)") + , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeCleanupThread)") , log(&Logger::get(log_name)) { task = storage.global_context.getSchedulePool().createTask(log_name, [this]{ run(); }); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index abb7e8e88ce..557aae3e01a 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -24,7 +24,7 @@ static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000; ReplicatedMergeTreePartCheckThread::ReplicatedMergeTreePartCheckThread(StorageReplicatedMergeTree & storage_) : storage(storage_) - , log_name(storage.database_name + "." + storage.table_name + " (ReplicatedMergeTreePartCheckThread)") + , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreePartCheckThread)") , log(&Logger::get(log_name)) { task = storage.global_context.getSchedulePool().createTask(log_name, [this] { run(); }); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index fce4479c16f..fad1196a4ba 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -40,7 +40,7 @@ static String generateActiveNodeIdentifier() ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_) : storage(storage_) - , log_name(storage.database_name + "." + storage.table_name + " (ReplicatedMergeTreeRestartingThread)") + , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeRestartingThread)") , log(&Logger::get(log_name)) , active_node_identifier(generateActiveNodeIdentifier()) { diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 6865cc956fa..195fa899175 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -18,8 +18,6 @@ class StorageFromMergeTreeDataPart : public ext::shared_ptr_helper; public: String getName() const override { return "FromMergeTreeDataPart"; } - String getTableName() const override { return part->storage.getTableName() + " (part " + part->name + ")"; } - String getDatabaseName() const override { return part->storage.getDatabaseName(); } BlockInputStreams read( const Names & column_names, @@ -51,7 +49,8 @@ public: protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) - : IStorage(part_->storage.getVirtuals()), part(part_) + : IStorage({part_->storage.getDatabaseName(), part_->storage.getTableName() + " (part " + part_->name + ")"}, + part_->storage.getVirtuals()), part(part_) { setColumns(part_->storage.getColumns()); setIndices(part_->storage.getIndices()); diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 15e5c0f9d41..e01f1594b85 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -61,13 +61,13 @@ StorageBuffer::StorageBuffer(const std::string & database_name_, const std::stri Context & context_, size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, const String & destination_database_, const String & destination_table_, bool allow_materialized_) - : - table_name(table_name_), database_name(database_name_), global_context(context_), + : IStorage({database_name_, table_name_}), + global_context(context_), num_shards(num_shards_), buffers(num_shards_), min_thresholds(min_thresholds_), max_thresholds(max_thresholds_), destination_database(destination_database_), destination_table(destination_table_), no_destination(destination_database.empty() && destination_table.empty()), - allow_materialized(allow_materialized_), log(&Logger::get("StorageBuffer (" + table_name + ")")) + allow_materialized(allow_materialized_), log(&Logger::get("StorageBuffer (" + table_name_ + ")")) { setColumns(columns_); setConstraints(constraints_); diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 10fd20aec4e..ea08a9bd61a 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -53,8 +53,6 @@ public: }; std::string getName() const override { return "Buffer"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const override; @@ -73,12 +71,6 @@ public: void shutdown() override; bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - bool supportsSampling() const override { return true; } bool supportsPrewhere() const override { @@ -101,9 +93,6 @@ public: ~StorageBuffer() override; private: - String table_name; - String database_name; - Context global_context; struct Buffer diff --git a/dbms/src/Storages/StorageDictionary.cpp b/dbms/src/Storages/StorageDictionary.cpp index 6fb2a774812..1bfa0e86b14 100644 --- a/dbms/src/Storages/StorageDictionary.cpp +++ b/dbms/src/Storages/StorageDictionary.cpp @@ -30,8 +30,7 @@ StorageDictionary::StorageDictionary( const Context & context, bool attach, const String & dictionary_name_) - : table_name(table_name_), - database_name(database_name_), + : IStorage({database_name_, table_name_}), dictionary_name(dictionary_name_), logger(&Poco::Logger::get("StorageDictionary")) { diff --git a/dbms/src/Storages/StorageDictionary.h b/dbms/src/Storages/StorageDictionary.h index 9539240e75d..57f8a3d2907 100644 --- a/dbms/src/Storages/StorageDictionary.h +++ b/dbms/src/Storages/StorageDictionary.h @@ -24,8 +24,6 @@ class StorageDictionary : public ext::shared_ptr_helper, publ friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Dictionary"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } BlockInputStreams read(const Names & column_names, const SelectQueryInfo & query_info, @@ -59,8 +57,6 @@ public: private: using Ptr = MultiVersion::Version; - String table_name; - String database_name; String dictionary_name; Poco::Logger * logger; diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 3e824f6f487..81f60340ad7 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -216,10 +216,9 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & relative_data_path_, bool attach_) - : IStorage(ColumnsDescription({ + : IStorage({database_name_, table_name_}, ColumnsDescription({ {"_shard_num", std::make_shared()}, }, true)), - table_name(table_name_), database_name(database_name_), remote_database(remote_database_), remote_table(remote_table_), global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_), path(relative_data_path_.empty() ? "" : (context_.getPath() + relative_data_path_)) @@ -237,8 +236,8 @@ StorageDistributed::StorageDistributed( if (!attach_ && !cluster_name.empty()) { size_t num_local_shards = global_context.getCluster(cluster_name)->getLocalShardCount(); - if (num_local_shards && remote_database == database_name && remote_table == table_name) - throw Exception("Distributed table " + table_name + " looks at itself", ErrorCodes::INFINITE_LOOP); + if (num_local_shards && remote_database == database_name_ && remote_table == table_name_) + throw Exception("Distributed table " + table_name_ + " looks at itself", ErrorCodes::INFINITE_LOOP); } } @@ -342,17 +341,18 @@ BlockInputStreams StorageDistributed::read( if (has_sharding_key) { auto smaller_cluster = skipUnusedShards(cluster, query_info); + auto storage_id = getStorageID(); if (smaller_cluster) { cluster = smaller_cluster; - LOG_DEBUG(log, "Reading from " << database_name << "." << table_name << ": " + LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): " " " << makeFormattedListOfShards(cluster)); } else { - LOG_DEBUG(log, "Reading from " << database_name << "." << table_name << ": " + LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster"); } } diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index c8bef378d95..7b408b361e0 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -52,8 +52,6 @@ public: const Context & context_); std::string getName() const override { return "Distributed"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } @@ -85,8 +83,7 @@ public: void rename(const String & /*new_path_to_table_data*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override { //TODO do we need no move data on disk and update path? - table_name = new_table_name; - database_name = new_database_name; + renameInMemory(new_database_name, new_table_name); } /// in the sub-tables, you need to manually add and delete columns @@ -120,8 +117,6 @@ public: ActionLock getActionLock(StorageActionBlockType type) override; - String table_name; - String database_name; String remote_database; String remote_table; ASTPtr remote_table_function_ptr; diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 96671df91cc..c2ef924f2a4 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -159,7 +159,7 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu } StorageFile::StorageFile(CommonArguments args) - : table_name(args.table_name), database_name(args.database_name), format_name(args.format_name), + : IStorage({args.database_name, args.table_name}), format_name(args.format_name), context_global(args.context), compression_method(args.compression_method) { setColumns(args.columns); @@ -280,7 +280,8 @@ public: else { if (storage.paths.size() != 1) - throw Exception("Table '" + storage.table_name + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); + throw Exception("Table " + storage.getStorageID().getNameForLogs() + + " is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); write_buf = getWriteBuffer(compression_method, storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); } @@ -327,17 +328,17 @@ BlockOutputStreamPtr StorageFile::write( Strings StorageFile::getDataPaths() const { if (paths.empty()) - throw Exception("Table '" + table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); + throw Exception("Table '" + getStorageID().getNameForLogs() + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); return paths; } void StorageFile::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { if (!is_db_table) - throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); + throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); if (paths.size() != 1) - throw Exception("Can't rename table '" + table_name + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); + throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); std::unique_lock lock(rwlock); @@ -346,8 +347,7 @@ void StorageFile::rename(const String & new_path_to_table_data, const String & n Poco::File(paths[0]).renameTo(path_new); paths[0] = std::move(path_new); - table_name = new_table_name; - database_name = new_database_name; + renameInMemory(new_database_name, new_table_name); } diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 6e49d495790..d7c1ea4d5d1 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -23,8 +23,6 @@ class StorageFile : public ext::shared_ptr_helper, public IStorage friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "File"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } BlockInputStreams read( const Names & column_names, @@ -40,12 +38,6 @@ public: void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - Strings getDataPaths() const override; struct CommonArguments @@ -75,8 +67,6 @@ protected: private: explicit StorageFile(CommonArguments args); - std::string table_name; - std::string database_name; std::string format_name; const Context & context_global; diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index c48d775be2b..95c0617f015 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -40,10 +40,9 @@ StorageHDFS::StorageHDFS(const String & uri_, const ConstraintsDescription & constraints_, Context & context_, const String & compression_method_ = "") - : uri(uri_) + : IStorage({database_name_, table_name_}) + , uri(uri_) , format_name(format_name_) - , table_name(table_name_) - , database_name(database_name_) , context(context_) , compression_method(compression_method_) { @@ -216,12 +215,6 @@ BlockInputStreams StorageHDFS::read( return result; } -void StorageHDFS::renameInMemory(const String & new_database_name, const String & new_table_name) -{ - table_name = new_table_name; - database_name = new_database_name; -} - BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const Context & /*context*/) { return std::make_shared(uri, diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index 13b71dd3dd8..1ce2163099a 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -18,8 +18,6 @@ class StorageHDFS : public ext::shared_ptr_helper, public IStorage friend struct ext::shared_ptr_helper; public: String getName() const override { return "HDFS"; } - String getTableName() const override { return table_name; } - String getDatabaseName() const override { return database_name; } BlockInputStreams read(const Names & column_names, const SelectQueryInfo & query_info, @@ -30,8 +28,6 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override; - protected: StorageHDFS(const String & uri_, const String & database_name_, @@ -45,8 +41,6 @@ protected: private: String uri; String format_name; - String table_name; - String database_name; Context & context; String compression_method; diff --git a/dbms/src/Storages/StorageInput.cpp b/dbms/src/Storages/StorageInput.cpp index 38424f22632..eece306a20a 100644 --- a/dbms/src/Storages/StorageInput.cpp +++ b/dbms/src/Storages/StorageInput.cpp @@ -16,7 +16,7 @@ namespace ErrorCodes } StorageInput::StorageInput(const String &table_name_, const ColumnsDescription & columns_) - : IStorage(columns_), table_name(table_name_) + : IStorage({"", table_name_}, columns_) { setColumns(columns_); } diff --git a/dbms/src/Storages/StorageInput.h b/dbms/src/Storages/StorageInput.h index bfc70633c48..706bb5bbfdf 100644 --- a/dbms/src/Storages/StorageInput.h +++ b/dbms/src/Storages/StorageInput.h @@ -13,7 +13,6 @@ class StorageInput : public ext::shared_ptr_helper, public IStorag friend struct ext::shared_ptr_helper; public: String getName() const override { return "Input"; } - String getTableName() const override { return table_name; } /// A table will read from this stream. void setInputStream(BlockInputStreamPtr input_stream_); @@ -27,7 +26,6 @@ public: unsigned num_streams) override; private: - String table_name; BlockInputStreamPtr input_stream; protected: diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index 7943c97cea8..efe332ef9a1 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -72,7 +72,7 @@ void StorageJoin::assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Stric { /// NOTE Could be more loose. if (!(kind == kind_ && strictness == strictness_)) - throw Exception("Table " + table_name + " has incompatible type of JOIN.", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); + throw Exception("Table " + getStorageID().getNameForLogs() + " has incompatible type of JOIN.", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); } diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index ac6cd79b552..7a948eda200 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -425,7 +425,7 @@ StorageLog::StorageLog( const ConstraintsDescription & constraints_, size_t max_compress_block_size_, const Context & context_) - : base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_), + : IStorage({database_name_, table_name_}), base_path(context_.getPath()), path(base_path + relative_path_), max_compress_block_size(max_compress_block_size_), file_checker(path + "sizes.json") { @@ -522,13 +522,13 @@ void StorageLog::rename(const String & new_path_to_table_data, const String & ne Poco::File(path).renameTo(new_path); path = new_path; - renameInMemory(new_database_name, new_table_name); file_checker.setPath(path + "sizes.json"); for (auto & file : files) file.second.data_file = Poco::File(path + Poco::Path(file.second.data_file.path()).getFileName()); marks_file = Poco::File(path + DBMS_STORAGE_LOG_MARKS_FILE_NAME); + renameInMemory(new_database_name, new_table_name); } void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index f52c0d804fb..6cea51f96e4 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -25,8 +25,6 @@ friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Log"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } BlockInputStreams read( const Names & column_names, @@ -40,12 +38,6 @@ public: void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; @@ -71,8 +63,6 @@ protected: private: String base_path; String path; - String table_name; - String database_name; mutable std::shared_mutex rwlock; @@ -121,7 +111,6 @@ private: void loadMarks(); /// The order of adding files should not change: it corresponds to the order of the columns in the marks file. - void addFile(const String & column_name, const IDataType & type, size_t level = 0); /** For normal columns, the number of rows in the block is specified in the marks. * For array columns and nested structures, there are more than one group of marks that correspond to different files diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 6b3082b9efb..0ffe9c92cb4 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -99,8 +99,7 @@ StorageMaterializedView::StorageMaterializedView( const ASTCreateQuery & query, const ColumnsDescription & columns_, bool attach_) - : table_name(table_name_), - database_name(database_name_), global_context(local_context.getGlobalContext()) + : IStorage({database_name_, table_name_}), global_context(local_context.getGlobalContext()) { setColumns(columns_); @@ -126,7 +125,7 @@ StorageMaterializedView::StorageMaterializedView( if (!select_table_name.empty()) global_context.addDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name, table_name)); + DatabaseAndTableName(database_name_, table_name_)); //FIXME // If the destination table is not set, use inner table if (!query.to_table.empty()) @@ -136,8 +135,8 @@ StorageMaterializedView::StorageMaterializedView( } else { - target_database_name = database_name; - target_table_name = generateInnerTableName(table_name); + target_database_name = database_name_; + target_table_name = generateInnerTableName(table_name_); has_inner_table = true; } @@ -168,7 +167,7 @@ StorageMaterializedView::StorageMaterializedView( if (!select_table_name.empty()) global_context.removeDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name, table_name)); + DatabaseAndTableName(database_name_, table_name_)); //FIXME throw; } @@ -234,9 +233,10 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, void StorageMaterializedView::drop(TableStructureWriteLockHolder &) { + auto table_id = getStorageID(); global_context.removeDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name, table_name)); + DatabaseAndTableName(table_id.database_name, table_id.table_name)); //FIXME if (has_inner_table && tryGetTargetTable()) executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_database_name, target_table_name); @@ -299,8 +299,10 @@ static void executeRenameQuery(Context & global_context, const String & database } -void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name) +void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name, std::unique_lock *) { + //FIXME + if (has_inner_table && tryGetTargetTable()) { String new_target_table_name = generateInnerTableName(new_table_name); @@ -309,25 +311,27 @@ void StorageMaterializedView::renameInMemory(const String & new_database_name, c } auto lock = global_context.getLock(); + std::unique_lock name_lock; + auto table_id = getStorageID(&name_lock); global_context.removeDependencyUnsafe( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name, table_name)); + DatabaseAndTableName(table_id.database_name, table_id.table_name)); - table_name = new_table_name; - database_name = new_database_name; + IStorage::renameInMemory(new_database_name, new_table_name, &name_lock); global_context.addDependencyUnsafe( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name, table_name)); + DatabaseAndTableName(new_database_name, new_table_name)); } void StorageMaterializedView::shutdown() { + auto table_id = getStorageID(); /// Make sure the dependency is removed after DETACH TABLE global_context.removeDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name, table_name)); + DatabaseAndTableName(table_id.database_name, table_id.table_name)); //FIXME } StoragePtr StorageMaterializedView::getTargetTable() const diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 7a5e2e7a528..e8e3272a3c9 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -14,8 +14,6 @@ class StorageMaterializedView : public ext::shared_ptr_helper; public: std::string getName() const override { return "MaterializedView"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } ASTPtr getInnerQuery() const { return inner_query->clone(); } @@ -43,7 +41,7 @@ public: void mutate(const MutationCommands & commands, const Context & context) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override; + void renameInMemory(const String & new_database_name, const String & new_table_name, std::unique_lock * id_lock = nullptr) override; void shutdown() override; @@ -72,8 +70,6 @@ private: String select_table_name; String target_database_name; String target_table_name; - String table_name; - String database_name; ASTPtr inner_query; Context & global_context; bool has_inner_table = false; diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 1b820e55c5e..0489f7bb115 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -75,7 +75,7 @@ private: StorageMemory::StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) - : database_name(std::move(database_name_)), table_name(std::move(table_name_)) + : IStorage({database_name_, table_name_}) { setColumns(std::move(columns_description_)); setConstraints(std::move(constraints_)); diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index bc1d46c0017..d6eb40ef7d0 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -25,8 +25,6 @@ friend struct ext::shared_ptr_helper; public: String getName() const override { return "Memory"; } - String getTableName() const override { return table_name; } - String getDatabaseName() const override { return database_name; } size_t getSize() const { return data.size(); } @@ -44,16 +42,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - private: - String database_name; - String table_name; - /// The data itself. `list` - so that when inserted to the end, the existing iterators are not invalidated. BlocksList data; diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index f74c81750a1..14179773fda 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -52,9 +52,7 @@ StorageMerge::StorageMerge( const String & source_database_, const String & table_name_regexp_, const Context & context_) - : IStorage(ColumnsDescription({{"_table", std::make_shared()}}, true)) - , table_name(table_name_) - , database_name(database_name_) + : IStorage({database_name_, table_name_}, ColumnsDescription({{"_table", std::make_shared()}}, true)) , source_database(source_database_) , table_name_regexp(table_name_regexp_) , global_context(context_) @@ -399,12 +397,13 @@ void StorageMerge::alter( const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + auto table_id = getStorageID(); auto new_columns = getColumns(); auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); setColumns(new_columns); } diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index b143bad901f..b14e8972dff 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -17,8 +17,6 @@ class StorageMerge : public ext::shared_ptr_helper, public IStorag friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Merge"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } bool isRemote() const override; @@ -42,12 +40,6 @@ public: size_t max_block_size, unsigned num_streams) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - /// you need to add and remove columns in the sub-tables manually /// the structure of sub-tables is not checked void alter( @@ -56,8 +48,6 @@ public: bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override; private: - String table_name; - String database_name; String source_database; OptimizedRegularExpression table_name_regexp; Context global_context; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 797b557c3b8..29623232b3f 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -149,12 +149,14 @@ BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Con void StorageMergeTree::checkTableCanBeDropped() const { + auto table_id = getStorageID(); const_cast(*this).recalculateColumnSizes(); - global_context.checkTableCanBeDropped(database_name, table_name, getTotalActiveSizeInBytes()); + global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes()); } void StorageMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition) { + auto table_id = getStorageID(); const_cast(*this).recalculateColumnSizes(); const String partition_id = getPartitionIDFromQuery(partition, global_context); @@ -166,7 +168,7 @@ void StorageMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition) { partition_size += part->bytes_on_disk; } - global_context.checkPartitionCanBeDropped(database_name, table_name, partition_size); + global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size); } void StorageMergeTree::drop(TableStructureWriteLockHolder &) @@ -589,7 +591,9 @@ bool StorageMergeTree::merge( merging_tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, false); } - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(database_name, table_name, future_part); + auto table_id = getStorageID(); + //FIXME + MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); /// Logging Stopwatch stopwatch; @@ -716,7 +720,8 @@ bool StorageMergeTree::tryMutatePart() if (!tagger) return false; - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(database_name, table_name, future_part); + auto table_id = getStorageID(); + MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); Stopwatch stopwatch; MutableDataPartPtr new_part; diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index de0146705f7..b907622513d 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -32,8 +32,6 @@ public: ~StorageMergeTree() override; std::string getName() const override { return merging_params.getModeName() + "MergeTree"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } bool supportsIndexForIn() const override { return true; } diff --git a/dbms/src/Storages/StorageMySQL.cpp b/dbms/src/Storages/StorageMySQL.cpp index ca5313e6ba8..b89a8a8a98a 100644 --- a/dbms/src/Storages/StorageMySQL.cpp +++ b/dbms/src/Storages/StorageMySQL.cpp @@ -47,8 +47,7 @@ StorageMySQL::StorageMySQL( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_) - : table_name(table_name_) - , database_name(database_name_) + : IStorage({database_name_, table_name_}) , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) , replace_query{replace_query_} diff --git a/dbms/src/Storages/StorageMySQL.h b/dbms/src/Storages/StorageMySQL.h index 5f369da4d25..af486cbd964 100644 --- a/dbms/src/Storages/StorageMySQL.h +++ b/dbms/src/Storages/StorageMySQL.h @@ -33,8 +33,6 @@ public: const Context & context_); std::string getName() const override { return "MySQL"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } BlockInputStreams read( const Names & column_names, @@ -48,8 +46,6 @@ public: private: friend class StorageMySQLBlockOutputStream; - std::string table_name; - std::string database_name; std::string remote_database_name; std::string remote_table_name; diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index d3b97f9ad46..67d2ea0e48e 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -34,15 +34,13 @@ void StorageNull::alter( const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); + auto table_id = getStorageID(); ColumnsDescription new_columns = getColumns(); IndicesDescription new_indices = getIndices(); ConstraintsDescription new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {}); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index faffa88cebb..f52780c9993 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -19,8 +19,6 @@ class StorageNull : public ext::shared_ptr_helper, public IStorage friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Null"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } BlockInputStreams read( const Names & column_names, @@ -38,22 +36,14 @@ public: return std::make_shared(getSampleBlock()); } - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - void alter( const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; private: - String table_name; - String database_name; protected: StorageNull(String database_name_, String table_name_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) - : table_name(std::move(table_name_)), database_name(std::move(database_name_)) + : IStorage({database_name_, table_name_}) { setColumns(std::move(columns_description_)); setConstraints(std::move(constraints_)); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 141da3a039c..05870c39eef 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -225,15 +225,15 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( zookeeper_path = "/" + zookeeper_path; replica_path = zookeeper_path + "/replicas/" + replica_name; - queue_updating_task = global_context.getSchedulePool().createTask(database_name + "." + table_name + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); }); + queue_updating_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); }); - mutations_updating_task = global_context.getSchedulePool().createTask(database_name + "." + table_name + " (StorageReplicatedMergeTree::mutationsUpdatingTask)", [this]{ mutationsUpdatingTask(); }); + mutations_updating_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsUpdatingTask)", [this]{ mutationsUpdatingTask(); }); - merge_selecting_task = global_context.getSchedulePool().createTask(database_name + "." + table_name + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); }); + merge_selecting_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); }); /// Will be activated if we win leader election. merge_selecting_task->deactivate(); - mutations_finalizing_task = global_context.getSchedulePool().createTask(database_name + "." + table_name + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); + mutations_finalizing_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); if (global_context.hasZooKeeper()) current_zookeeper = global_context.getZooKeeper(); @@ -484,7 +484,9 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column }; } - global_context.getDatabase(database_name)->alterTable(global_context, table_name, new_columns, new_indices, new_constraints, storage_modifier); + auto table_id = getStorageID(); + //FIXME + global_context.getDatabase(table_id.database_name)->alterTable(global_context, table_id.table_name, new_columns, new_indices, new_constraints, storage_modifier); /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. @@ -642,7 +644,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) if (insane && !skip_sanity_checks) { std::stringstream why; - why << "The local set of parts of table " << database_name << "." << table_name << " doesn't look like the set of parts " + why << "The local set of parts of table " << getStorageID().getNameForLogs() << " doesn't look like the set of parts " << "in ZooKeeper: " << formatReadableQuantity(unexpected_parts_rows) << " rows of " << formatReadableQuantity(total_rows_on_filesystem) << " total rows in filesystem are suspicious."; @@ -1018,7 +1020,8 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) + backQuote(entry.new_part_name), ErrorCodes::BAD_DATA_PART_NAME); } - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(database_name, table_name, future_merged_part); + auto table_id = getStorageID(); + MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_merged_part); Transaction transaction(*this); MutableDataPartPtr part; @@ -1158,8 +1161,9 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM future_mutated_part.part_info = new_part_info; future_mutated_part.name = entry.new_part_name; + auto table_id = getStorageID(); MergeList::EntryPtr merge_entry = global_context.getMergeList().insert( - database_name, table_name, future_mutated_part); + table_id.database_name, table_id.table_name, future_mutated_part); Stopwatch stopwatch; @@ -2870,7 +2874,7 @@ void StorageReplicatedMergeTree::startup() queue.initialize( zookeeper_path, replica_path, - database_name + "." + table_name + " (ReplicatedMergeTreeQueue)", + getStorageID().getFullTableName() + " (ReplicatedMergeTreeQueue)", getDataParts()); StoragePtr ptr = shared_from_this(); @@ -3701,7 +3705,8 @@ void StorageReplicatedMergeTree::checkTableCanBeDropped() const { /// Consider only synchronized data const_cast(*this).recalculateColumnSizes(); - global_context.checkTableCanBeDropped(database_name, table_name, getTotalActiveSizeInBytes()); + auto table_id = getStorageID(); + global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes()); } @@ -3717,7 +3722,8 @@ void StorageReplicatedMergeTree::checkPartitionCanBeDropped(const ASTPtr & parti for (const auto & part : parts_to_remove) partition_size += part->bytes_on_disk; - global_context.checkPartitionCanBeDropped(database_name, table_name, partition_size); + auto table_id = getStorageID(); + global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size); } @@ -4912,6 +4918,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ ReplicatedMergeTreeLogEntryData entry; { + auto src_table_id = src_data.getStorageID(); entry.type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; entry.source_replica = replica_name; entry.create_time = time(nullptr); @@ -4919,8 +4926,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ auto & entry_replace = *entry.replace_range_entry; entry_replace.drop_range_part_name = drop_range_fake_part_name; - entry_replace.from_database = src_data.database_name; - entry_replace.from_table = src_data.table_name; + entry_replace.from_database = src_table_id.database_name; + entry_replace.from_table = src_table_id.table_name; for (const auto & part : src_parts) entry_replace.src_part_names.emplace_back(part->name); for (const auto & part : dst_parts) @@ -5110,13 +5117,14 @@ void StorageReplicatedMergeTree::updatePartHeaderInZooKeeperAndCommit( ReplicatedMergeTreeAddress StorageReplicatedMergeTree::getReplicatedMergeTreeAddress() const { auto host_port = global_context.getInterserverIOAddress(); + auto table_id = getStorageID(); ReplicatedMergeTreeAddress res; res.host = host_port.first; res.replication_port = host_port.second; res.queries_port = global_context.getTCPPort(); - res.database = database_name; - res.table = table_name; + res.database = table_id.database_name; + res.table = table_id.table_name; res.scheme = global_context.getInterserverScheme(); return res; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 20a22565d64..95e5a9a25fd 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -83,8 +83,6 @@ public: ~StorageReplicatedMergeTree() override; std::string getName() const override { return "Replicated" + merging_params.getModeName() + "MergeTree"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } bool supportsReplication() const override { return true; } bool supportsDeduplication() const override { return true; } diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 1b0103c789e..91ae5eaebd7 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -132,12 +132,10 @@ StorageS3::StorageS3( const ConstraintsDescription & constraints_, Context & context_, const String & compression_method_ = "") - : IStorage(columns_) + : IStorage({database_name_, table_name_}, columns_) , uri(uri_) , context_global(context_) , format_name(format_name_) - , database_name(database_name_) - , table_name(table_name_) , min_upload_part_size(min_upload_part_size_) , compression_method(compression_method_) { @@ -170,12 +168,6 @@ BlockInputStreams StorageS3::read( return {std::make_shared(block_input, column_defaults, context)}; } -void StorageS3::renameInMemory(const String & new_database_name, const String & new_table_name) -{ - table_name = new_table_name; - database_name = new_database_name; -} - BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & /*context*/) { return std::make_shared( diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index 902da609309..093f247ab48 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -37,11 +37,6 @@ public: return getSampleBlock(); } - String getTableName() const override - { - return table_name; - } - BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, @@ -52,15 +47,11 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override; - private: Poco::URI uri; const Context & context_global; String format_name; - String database_name; - String table_name; UInt64 min_upload_part_size; String compression_method; }; diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index c9c200a70f7..81267cbc4f6 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -94,7 +94,7 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_) - : table_name(table_name_), database_name(database_name_) + : IStorage({database_name_, table_name_}) { setColumns(columns_); setConstraints(constraints_); diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index 7faac0317ea..8934b16cd7d 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -19,17 +19,8 @@ class StorageSetOrJoinBase : public IStorage friend class SetOrJoinBlockOutputStream; public: - String getTableName() const override { return table_name; } - String getDatabaseName() const override { return database_name; } - void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; Strings getDataPaths() const override { return {path}; } @@ -45,8 +36,6 @@ protected: String base_path; String path; - String table_name; - String database_name; std::atomic increment = 0; /// For the backup file names. diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index b8e27bac89a..e44fe9b505f 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -202,7 +202,8 @@ StorageStripeLog::StorageStripeLog( bool attach, size_t max_compress_block_size_, const Context & context_) - : base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_), + : IStorage({database_name_, table_name_}), + base_path(context_.getPath()), path(base_path + relative_path_), max_compress_block_size(max_compress_block_size_), file_checker(path + "sizes.json"), log(&Logger::get("StorageStripeLog")) @@ -291,7 +292,7 @@ CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Conte void StorageStripeLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { - if (table_name.empty()) + if (getStorageID().table_name.empty()) //FIXME how can it be empty? throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR); std::shared_lock lock(rwlock); diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 78f51264031..5d31c739896 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -27,8 +27,6 @@ friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "StripeLog"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } BlockInputStreams read( const Names & column_names, @@ -42,12 +40,6 @@ public: void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; /// Data of the file. @@ -66,8 +58,6 @@ public: private: String base_path; String path; - String table_name; - String database_name; size_t max_compress_block_size; diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index ecb1496363e..090809009b4 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -331,7 +331,7 @@ StorageTinyLog::StorageTinyLog( bool attach, size_t max_compress_block_size_, const Context & context_) - : base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_), + : IStorage({database_name_, table_name_}), base_path(context_.getPath()), path(base_path + relative_path_), max_compress_block_size(max_compress_block_size_), file_checker(path + "sizes.json"), log(&Logger::get("StorageTinyLog")) @@ -421,7 +421,7 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { - if (table_name.empty()) + if (getStorageID().table_name.empty()) //FIXME how can it be empty? throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR); std::unique_lock lock(rwlock); diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 1be73181e05..17b5e6e94cc 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -26,8 +26,6 @@ friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "TinyLog"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } BlockInputStreams read( const Names & column_names, @@ -41,12 +39,6 @@ public: void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; /// Column data @@ -65,8 +57,6 @@ public: private: String base_path; String path; - String table_name; - String database_name; size_t max_compress_block_size; @@ -77,7 +67,6 @@ private: Logger * log; - void addFile(const String & column_name, const IDataType & type, size_t level = 0); void addFiles(const String & column_name, const IDataType & type); protected: diff --git a/dbms/src/Storages/StorageURL.cpp b/dbms/src/Storages/StorageURL.cpp index ce3bd30a963..583eac646bd 100644 --- a/dbms/src/Storages/StorageURL.cpp +++ b/dbms/src/Storages/StorageURL.cpp @@ -35,7 +35,7 @@ IStorageURLBase::IStorageURLBase( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & compression_method_) - : uri(uri_), context_global(context_), compression_method(compression_method_), format_name(format_name_), table_name(table_name_), database_name(database_name_) + : IStorage({database_name_, table_name_}), uri(uri_), context_global(context_), compression_method(compression_method_), format_name(format_name_) { setColumns(columns_); setConstraints(constraints_); @@ -191,12 +191,6 @@ BlockInputStreams IStorageURLBase::read(const Names & column_names, return {std::make_shared(block_input, column_defaults, context)}; } -void IStorageURLBase::renameInMemory(const String & new_database_name, const String & new_table_name) -{ - table_name = new_table_name; - database_name = new_database_name; -} - BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const Context & /*context*/) { return std::make_shared( diff --git a/dbms/src/Storages/StorageURL.h b/dbms/src/Storages/StorageURL.h index 04e2108b39b..4f752ea4ec4 100644 --- a/dbms/src/Storages/StorageURL.h +++ b/dbms/src/Storages/StorageURL.h @@ -16,9 +16,6 @@ namespace DB class IStorageURLBase : public IStorage { public: - String getTableName() const override { return table_name; } - String getDatabaseName() const override { return database_name; } - BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, @@ -29,8 +26,6 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override; - protected: IStorageURLBase( const Poco::URI & uri_, @@ -48,8 +43,6 @@ protected: private: String format_name; - String table_name; - String database_name; virtual std::string getReadMethod() const; diff --git a/dbms/src/Storages/StorageValues.cpp b/dbms/src/Storages/StorageValues.cpp index 452d815e5ea..ec9cf82a7e7 100644 --- a/dbms/src/Storages/StorageValues.cpp +++ b/dbms/src/Storages/StorageValues.cpp @@ -8,7 +8,7 @@ namespace DB { StorageValues::StorageValues(const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const Block & res_block_) - : database_name(database_name_), table_name(table_name_), res_block(res_block_) + : IStorage({database_name_, table_name_}), res_block(res_block_) { setColumns(columns_); } diff --git a/dbms/src/Storages/StorageValues.h b/dbms/src/Storages/StorageValues.h index c07cf7cbb63..67858516213 100644 --- a/dbms/src/Storages/StorageValues.h +++ b/dbms/src/Storages/StorageValues.h @@ -14,8 +14,6 @@ class StorageValues : public ext::shared_ptr_helper, public IStor friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Values"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } BlockInputStreams read( const Names & column_names, @@ -26,8 +24,6 @@ public: unsigned num_streams) override; private: - std::string database_name; - std::string table_name; Block res_block; protected: diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index 824856dfc4e..6289db95726 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -31,7 +31,7 @@ StorageView::StorageView( const String & table_name_, const ASTCreateQuery & query, const ColumnsDescription & columns_) - : table_name(table_name_), database_name(database_name_) + : IStorage({database_name_, table_name_}) { setColumns(columns_); diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index 13c835f5364..709082b58ef 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -15,8 +15,6 @@ class StorageView : public ext::shared_ptr_helper, public IStorage friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "View"; } - std::string getTableName() const override { return table_name; } - std::string getDatabaseName() const override { return database_name; } /// It is passed inside the query and solved at its level. bool supportsSampling() const override { return true; } @@ -30,15 +28,7 @@ public: size_t max_block_size, unsigned num_streams) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override - { - table_name = new_table_name; - database_name = new_database_name; - } - private: - String table_name; - String database_name; ASTPtr inner_query; void replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery); diff --git a/dbms/src/Storages/System/IStorageSystemOneBlock.h b/dbms/src/Storages/System/IStorageSystemOneBlock.h index b5c8c2ad78f..68a42511061 100644 --- a/dbms/src/Storages/System/IStorageSystemOneBlock.h +++ b/dbms/src/Storages/System/IStorageSystemOneBlock.h @@ -20,14 +20,11 @@ protected: virtual void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const = 0; public: - IStorageSystemOneBlock(const String & name_) : name(name_) + IStorageSystemOneBlock(const String & name_) : IStorage({"system", name_}) { setColumns(ColumnsDescription(Self::getNamesAndTypes())); } - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } - BlockInputStreams read(const Names & column_names, const SelectQueryInfo & query_info, const Context & context, @@ -43,9 +40,6 @@ public: return BlockInputStreams(1, std::make_shared(sample_block.cloneWithColumns(std::move(res_columns)))); } - -private: - const String name; }; } diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 31c7b1c45c3..a370625ef1d 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes } StorageSystemColumns::StorageSystemColumns(const std::string & name_) - : name(name_) + : IStorage({"system", name_}) { setColumns(ColumnsDescription( { diff --git a/dbms/src/Storages/System/StorageSystemColumns.h b/dbms/src/Storages/System/StorageSystemColumns.h index b569c2bafff..4c7457d7f8d 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.h +++ b/dbms/src/Storages/System/StorageSystemColumns.h @@ -16,8 +16,6 @@ class StorageSystemColumns : public ext::shared_ptr_helper friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemColumns"; } - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } BlockInputStreams read( const Names & column_names, @@ -27,9 +25,6 @@ public: size_t max_block_size, unsigned num_streams) override; -private: - const std::string name; - protected: StorageSystemColumns(const std::string & name_); }; diff --git a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp index acda98203db..583df8aefa5 100644 --- a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp @@ -24,11 +24,10 @@ class StorageSystemDetachedParts : friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemDetachedParts"; } - std::string getTableName() const override { return "detached_parts"; } - std::string getDatabaseName() const override { return "system"; } protected: explicit StorageSystemDetachedParts() + : IStorage({"system", "detached_parts"}) { setColumns(ColumnsDescription{{ {"database", std::make_shared()}, diff --git a/dbms/src/Storages/System/StorageSystemDisks.cpp b/dbms/src/Storages/System/StorageSystemDisks.cpp index 3deb9b2399f..3e40a6f1fc2 100644 --- a/dbms/src/Storages/System/StorageSystemDisks.cpp +++ b/dbms/src/Storages/System/StorageSystemDisks.cpp @@ -10,7 +10,7 @@ namespace ErrorCodes StorageSystemDisks::StorageSystemDisks(const std::string & name_) - : name(name_) + : IStorage({"system", name_}) { setColumns(ColumnsDescription( { diff --git a/dbms/src/Storages/System/StorageSystemDisks.h b/dbms/src/Storages/System/StorageSystemDisks.h index e54adbc1da8..8b472177ac0 100644 --- a/dbms/src/Storages/System/StorageSystemDisks.h +++ b/dbms/src/Storages/System/StorageSystemDisks.h @@ -19,8 +19,6 @@ class StorageSystemDisks : public ext::shared_ptr_helper, pu friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemDisks"; } - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } BlockInputStreams read( const Names & column_names, @@ -30,9 +28,6 @@ public: size_t max_block_size, unsigned num_streams) override; -private: - const std::string name; - protected: StorageSystemDisks(const std::string & name_); }; diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index 2f155e22a11..8e3e362332f 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -104,7 +104,7 @@ private: StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_, UInt64 offset_, bool even_distribution_) - : name(name_), multithreaded(multithreaded_), even_distribution(even_distribution_), limit(limit_), offset(offset_) + : IStorage({"system", name_}), multithreaded(multithreaded_), even_distribution(even_distribution_), limit(limit_), offset(offset_) { setColumns(ColumnsDescription({{"number", std::make_shared()}})); } diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index 76070839012..38bbccaa7d9 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -28,8 +28,6 @@ class StorageSystemNumbers : public ext::shared_ptr_helper friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemNumbers"; } - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } BlockInputStreams read( const Names & column_names, @@ -40,7 +38,6 @@ public: unsigned num_streams) override; private: - const std::string name; bool multithreaded; bool even_distribution; std::optional limit; diff --git a/dbms/src/Storages/System/StorageSystemOne.cpp b/dbms/src/Storages/System/StorageSystemOne.cpp index 827ee5ca6a1..10375aaf028 100644 --- a/dbms/src/Storages/System/StorageSystemOne.cpp +++ b/dbms/src/Storages/System/StorageSystemOne.cpp @@ -11,7 +11,7 @@ namespace DB StorageSystemOne::StorageSystemOne(const std::string & name_) - : name(name_) + : IStorage({"system", name_}) { setColumns(ColumnsDescription({{"dummy", std::make_shared()}})); } diff --git a/dbms/src/Storages/System/StorageSystemOne.h b/dbms/src/Storages/System/StorageSystemOne.h index 3e35fdb3477..a64c25e36eb 100644 --- a/dbms/src/Storages/System/StorageSystemOne.h +++ b/dbms/src/Storages/System/StorageSystemOne.h @@ -20,8 +20,6 @@ class StorageSystemOne : public ext::shared_ptr_helper, public friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemOne"; } - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } BlockInputStreams read( const Names & column_names, @@ -31,9 +29,6 @@ public: size_t max_block_size, unsigned num_streams) override; -private: - const std::string name; - protected: StorageSystemOne(const std::string & name_); }; diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 2e4f5dee01f..9be407913f1 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -257,7 +257,7 @@ bool StorageSystemPartsBase::hasColumn(const String & column_name) const } StorageSystemPartsBase::StorageSystemPartsBase(std::string name_, NamesAndTypesList && columns_) - : name(std::move(name_)) + : IStorage({"system", name_}) { ColumnsDescription tmp_columns(std::move(columns_)); diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.h b/dbms/src/Storages/System/StorageSystemPartsBase.h index 080153a2a91..1357160c50b 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.h +++ b/dbms/src/Storages/System/StorageSystemPartsBase.h @@ -53,9 +53,6 @@ private: class StorageSystemPartsBase : public IStorage { public: - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } - NameAndTypePair getColumn(const String & column_name) const override; bool hasColumn(const String & column_name) const override; @@ -69,8 +66,6 @@ public: unsigned num_streams) override; private: - const std::string name; - bool hasStateColumn(const Names & column_names) const; protected: diff --git a/dbms/src/Storages/System/StorageSystemReplicas.cpp b/dbms/src/Storages/System/StorageSystemReplicas.cpp index 3e319e19bd7..08f49ba97ee 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicas.cpp @@ -15,7 +15,7 @@ namespace DB StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) - : name(name_) + : IStorage({"system", name_}) { setColumns(ColumnsDescription({ { "database", std::make_shared() }, diff --git a/dbms/src/Storages/System/StorageSystemReplicas.h b/dbms/src/Storages/System/StorageSystemReplicas.h index fb006f4cbde..d6cf8480eaf 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.h +++ b/dbms/src/Storages/System/StorageSystemReplicas.h @@ -17,8 +17,6 @@ class StorageSystemReplicas : public ext::shared_ptr_helper; public: std::string getName() const override { return "SystemReplicas"; } - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } BlockInputStreams read( const Names & column_names, @@ -28,9 +26,6 @@ public: size_t max_block_size, unsigned num_streams) override; -private: - const std::string name; - protected: StorageSystemReplicas(const std::string & name_); }; diff --git a/dbms/src/Storages/System/StorageSystemStoragePolicies.cpp b/dbms/src/Storages/System/StorageSystemStoragePolicies.cpp index 5f4e0a63e39..a134d4fa379 100644 --- a/dbms/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/dbms/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & name_) - : name(name_) + : IStorage({"system", name_}) { setColumns( ColumnsDescription({ diff --git a/dbms/src/Storages/System/StorageSystemStoragePolicies.h b/dbms/src/Storages/System/StorageSystemStoragePolicies.h index 3848052f99e..6d596b5f0c9 100644 --- a/dbms/src/Storages/System/StorageSystemStoragePolicies.h +++ b/dbms/src/Storages/System/StorageSystemStoragePolicies.h @@ -19,8 +19,6 @@ class StorageSystemStoragePolicies : public ext::shared_ptr_helper; public: std::string getName() const override { return "SystemStoragePolicies"; } - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } BlockInputStreams read( const Names & column_names, @@ -30,9 +28,6 @@ public: size_t max_block_size, unsigned num_streams) override; -private: - const std::string name; - protected: StorageSystemStoragePolicies(const std::string & name_); }; diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index cfa417f24a3..364b9dbad0d 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes StorageSystemTables::StorageSystemTables(const std::string & name_) - : name(name_) + : IStorage({"system", name_}) { setColumns(ColumnsDescription( { diff --git a/dbms/src/Storages/System/StorageSystemTables.h b/dbms/src/Storages/System/StorageSystemTables.h index fc5a82e9a52..05c96cd14bc 100644 --- a/dbms/src/Storages/System/StorageSystemTables.h +++ b/dbms/src/Storages/System/StorageSystemTables.h @@ -17,8 +17,6 @@ class StorageSystemTables : public ext::shared_ptr_helper, friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemTables"; } - std::string getTableName() const override { return name; } - std::string getDatabaseName() const override { return "system"; } BlockInputStreams read( const Names & column_names, @@ -28,9 +26,6 @@ public: size_t max_block_size, unsigned num_streams) override; -private: - const std::string name; - protected: StorageSystemTables(const std::string & name_); }; From 8e319aca97dc0350891fcfc49d79df6d9dbc3bd4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 4 Dec 2019 19:06:55 +0300 Subject: [PATCH 019/743] better constructors of storages --- dbms/programs/server/TCPHandler.cpp | 2 +- dbms/src/Core/ExternalTable.cpp | 2 +- .../PushingToViewsBlockOutputStream.cpp | 2 +- dbms/src/Databases/DatabaseDictionary.cpp | 4 +- dbms/src/Databases/DatabaseMySQL.cpp | 2 +- dbms/src/Databases/DatabaseOnDisk.cpp | 9 ++-- .../Databases/DatabaseWithDictionaries.cpp | 2 +- .../Interpreters/GlobalSubqueriesVisitor.h | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 3 -- .../Interpreters/InterpreterSystemQuery.cpp | 3 -- dbms/src/Storages/Kafka/StorageKafka.cpp | 9 ++-- dbms/src/Storages/Kafka/StorageKafka.h | 15 ++++--- .../src/Storages/LiveView/StorageLiveView.cpp | 9 ++-- dbms/src/Storages/LiveView/StorageLiveView.h | 3 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 7 ++- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 8 +++- .../MergeTree/registerStorageMergeTree.cpp | 4 +- dbms/src/Storages/StorageBuffer.cpp | 36 +++++++++------ dbms/src/Storages/StorageBuffer.h | 14 ++++-- dbms/src/Storages/StorageDictionary.cpp | 11 +++-- dbms/src/Storages/StorageDictionary.h | 3 +- dbms/src/Storages/StorageDistributed.cpp | 44 ++++++++++--------- dbms/src/Storages/StorageDistributed.h | 10 ++--- dbms/src/Storages/StorageFactory.cpp | 9 ++-- dbms/src/Storages/StorageFactory.h | 7 +-- dbms/src/Storages/StorageFile.cpp | 8 ++-- dbms/src/Storages/StorageFile.h | 3 +- dbms/src/Storages/StorageHDFS.cpp | 7 ++- dbms/src/Storages/StorageHDFS.h | 3 +- dbms/src/Storages/StorageInput.cpp | 2 +- dbms/src/Storages/StorageJoin.cpp | 8 ++-- dbms/src/Storages/StorageJoin.h | 3 +- dbms/src/Storages/StorageLog.cpp | 13 +++--- dbms/src/Storages/StorageLog.h | 3 +- dbms/src/Storages/StorageMaterializedView.cpp | 15 +++---- dbms/src/Storages/StorageMaterializedView.h | 3 +- dbms/src/Storages/StorageMemory.cpp | 6 +-- dbms/src/Storages/StorageMemory.h | 2 +- dbms/src/Storages/StorageMerge.cpp | 7 ++- dbms/src/Storages/StorageMerge.h | 3 +- dbms/src/Storages/StorageMergeTree.cpp | 29 ++++++++---- dbms/src/Storages/StorageMergeTree.h | 3 +- dbms/src/Storages/StorageMySQL.cpp | 8 ++-- dbms/src/Storages/StorageMySQL.h | 3 +- dbms/src/Storages/StorageNull.cpp | 2 +- dbms/src/Storages/StorageNull.h | 4 +- .../Storages/StorageReplicatedMergeTree.cpp | 42 ++++++++++++------ .../src/Storages/StorageReplicatedMergeTree.h | 2 +- dbms/src/Storages/StorageS3.cpp | 7 ++- dbms/src/Storages/StorageS3.h | 3 +- dbms/src/Storages/StorageSet.cpp | 12 +++-- dbms/src/Storages/StorageSet.h | 6 +-- dbms/src/Storages/StorageStripeLog.cpp | 16 +++---- dbms/src/Storages/StorageStripeLog.h | 3 +- dbms/src/Storages/StorageTinyLog.cpp | 15 ++++--- dbms/src/Storages/StorageTinyLog.h | 3 +- dbms/src/Storages/StorageURL.cpp | 11 +++-- dbms/src/Storages/StorageURL.h | 8 ++-- dbms/src/Storages/StorageValues.cpp | 4 +- dbms/src/Storages/StorageValues.h | 2 +- dbms/src/Storages/StorageView.cpp | 7 ++- dbms/src/Storages/StorageView.h | 3 +- dbms/src/Storages/StorageXDBC.cpp | 13 ++++-- dbms/src/Storages/StorageXDBC.h | 3 +- ..._transform_query_for_external_database.cpp | 2 +- dbms/src/Storages/tests/storage_log.cpp | 2 +- .../src/TableFunctions/ITableFunctionXDBC.cpp | 2 +- dbms/src/TableFunctions/TableFunctionFile.cpp | 2 +- dbms/src/TableFunctions/TableFunctionHDFS.cpp | 3 +- .../src/TableFunctions/TableFunctionMerge.cpp | 3 +- .../src/TableFunctions/TableFunctionMySQL.cpp | 3 +- .../TableFunctions/TableFunctionRemote.cpp | 4 +- dbms/src/TableFunctions/TableFunctionS3.cpp | 2 +- dbms/src/TableFunctions/TableFunctionURL.cpp | 2 +- .../TableFunctions/TableFunctionValues.cpp | 2 +- 76 files changed, 282 insertions(+), 262 deletions(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 76ea69cc737..82d6b8b28e1 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -992,7 +992,7 @@ bool TCPHandler::receiveData(bool scalar) if (!(storage = query_context->tryGetExternalTable(name))) { NamesAndTypesList columns = block.getNamesAndTypesList(); - storage = StorageMemory::create("_external", name, ColumnsDescription{columns}, ConstraintsDescription{}); + storage = StorageMemory::create(StorageID("_external", name), ColumnsDescription{columns}, ConstraintsDescription{}); storage->startup(); query_context->addExternalTable(name, storage); } diff --git a/dbms/src/Core/ExternalTable.cpp b/dbms/src/Core/ExternalTable.cpp index 8befb4e054c..e87bc3b513b 100644 --- a/dbms/src/Core/ExternalTable.cpp +++ b/dbms/src/Core/ExternalTable.cpp @@ -160,7 +160,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, /// Create table NamesAndTypesList columns = sample_block.getNamesAndTypesList(); - StoragePtr storage = StorageMemory::create("_external", data.second, ColumnsDescription{columns}, ConstraintsDescription{}); + StoragePtr storage = StorageMemory::create(StorageID("_external", data.second), ColumnsDescription{columns}, ConstraintsDescription{}); storage->startup(); context.addExternalTable(data.second, storage); BlockOutputStreamPtr output = storage->write(ASTPtr(), context); diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index d7fb7bad343..a630757b9a5 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -223,7 +223,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n /// InterpreterSelectQuery will do processing of alias columns. Context local_context = *views_context; local_context.addViewSource( - StorageValues::create(storage->getDatabaseName(), storage->getTableName(), storage->getColumns(), + StorageValues::create(StorageID(storage->getDatabaseName(), storage->getTableName()), storage->getColumns(), //FIXME block)); select.emplace(view.query, local_context, SelectQueryOptions()); in = std::make_shared(select->execute().in); diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index 771a9f5924a..ec0714ff5fa 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -52,7 +52,7 @@ Tables DatabaseDictionary::listTables(const Context & context, const FilterByNam auto dict_name = dict_ptr->getName(); const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - tables[dict_name] = StorageDictionary::create(getDatabaseName(), dict_name, ColumnsDescription{columns}, context, true, dict_name); + tables[dict_name] = StorageDictionary::create(StorageID(getDatabaseName(), dict_name), ColumnsDescription{columns}, context, true, dict_name); } } return tables; @@ -74,7 +74,7 @@ StoragePtr DatabaseDictionary::tryGetTable( { const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - return StorageDictionary::create(getDatabaseName(), table_name, ColumnsDescription{columns}, context, true, table_name); + return StorageDictionary::create(StorageID(getDatabaseName(), table_name), ColumnsDescription{columns}, context, true, table_name); } return {}; diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index b4933040d99..b7e32ad9bb0 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -235,7 +235,7 @@ DatabaseMySQL::MySQLStorageInfo DatabaseMySQL::createStorageInfo( const String & table_name, const NamesAndTypesList & columns_name_and_type, const UInt64 & table_modification_time) const { const auto & mysql_table = StorageMySQL::create( - database_name, table_name, std::move(mysql_pool), mysql_database_name, table_name, + StorageID(database_name, table_name), std::move(mysql_pool), mysql_database_name, table_name, false, "", ColumnsDescription{columns_name_and_type}, ConstraintsDescription{}, global_context); const auto & create_table_query = std::make_shared(); diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index ed7453af663..5e19bfb631c 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -74,9 +74,12 @@ std::pair createTableFromAST( ast_create_query.table, StorageFactory::instance().get( ast_create_query, - table_data_path_relative, ast_create_query.table, database_name, context, context.getGlobalContext(), - columns, constraints, - true, has_force_restore_data_flag) + table_data_path_relative, + context, + context.getGlobalContext(), + columns, + constraints, + has_force_restore_data_flag) }; } diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index b536d8a21b1..51e263db3ae 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -194,7 +194,7 @@ StoragePtr DatabaseWithDictionaries::getDictionaryStorage(const Context & contex { const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - return StorageDictionary::create(database_name, table_name, ColumnsDescription{columns}, context, true, dict_name); + return StorageDictionary::create(StorageID(database_name, table_name), ColumnsDescription{columns}, context, true, dict_name); } return nullptr; } diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h index 6c380162af4..d49f4c8f4e5 100644 --- a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -95,7 +95,7 @@ public: Block sample = interpreter->getSampleBlock(); NamesAndTypesList columns = sample.getNamesAndTypesList(); - StoragePtr external_storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns}, ConstraintsDescription{}); + StoragePtr external_storage = StorageMemory::create(StorageID("_external", external_table_name), ColumnsDescription{columns}, ConstraintsDescription{}); external_storage->startup(); /** We replace the subquery with the name of the temporary table. diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 74c85c7fce9..42d6290c525 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -672,13 +672,10 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, { res = StorageFactory::instance().get(create, data_path, - table_name, - database_name, context, context.getGlobalContext(), properties.columns, properties.constraints, - create.attach, false); } diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 7a866c525b8..c1e462f93b0 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -291,13 +291,10 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam StoragePtr table = StorageFactory::instance().get(create, data_path, - table_name, - database_name, system_context, system_context.getGlobalContext(), columns, constraints, - create.attach, false); database->createTable(system_context, table_name, table, create_ast); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index d15ace31426..db95769227d 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -74,8 +74,7 @@ namespace } StorageKafka::StorageKafka( - const std::string & table_name_, - const std::string & database_name_, + const StorageID & table_id_, Context & context_, const ColumnsDescription & columns_, const String & brokers_, @@ -88,7 +87,7 @@ StorageKafka::StorageKafka( UInt64 max_block_size_, size_t skip_broken_, bool intermediate_commit_) - : IStorage({database_name_, table_name_}, + : IStorage(table_id_, ColumnsDescription({{"_topic", std::make_shared()}, {"_key", std::make_shared()}, {"_offset", std::make_shared()}, @@ -103,7 +102,7 @@ StorageKafka::StorageKafka( , schema_name(global_context.getMacros()->expand(schema_name_)) , num_consumers(num_consumers_) , max_block_size(max_block_size_) - , log(&Logger::get("StorageKafka (" + table_name_ + ")")) + , log(&Logger::get("StorageKafka (" + table_id_.table_name + ")")) , semaphore(0, num_consumers_) , skip_broken(skip_broken_) , intermediate_commit(intermediate_commit_) @@ -642,7 +641,7 @@ void registerStorageKafka(StorageFactory & factory) } return StorageKafka::create( - args.table_name, args.database_name, args.context, args.columns, + args.table_id, args.context, args.columns, brokers, group, topics, format, row_delimiter, schema, num_consumers, max_block_size, skip_broken, intermediate_commit); }); } diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index 2b66a6f6248..61960ceb674 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -64,13 +64,18 @@ public: protected: StorageKafka( - const std::string & table_name_, - const std::string & database_name_, + const StorageID & table_id_, Context & context_, const ColumnsDescription & columns_, - const String & brokers_, const String & group_, const Names & topics_, - const String & format_name_, char row_delimiter_, const String & schema_name_, - size_t num_consumers_, UInt64 max_block_size_, size_t skip_broken, + const String & brokers_, + const String & group_, + const Names & topics_, + const String & format_name_, + char row_delimiter_, + const String & schema_name_, + size_t num_consumers_, + UInt64 max_block_size_, + size_t skip_broken, bool intermediate_commit_); private: diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 7e6e5edfa58..c041aeef9d4 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -193,12 +193,11 @@ void StorageLiveView::writeIntoLiveView( StorageLiveView::StorageLiveView( - const String & table_name_, - const String & database_name_, + const StorageID & table_id_, Context & local_context, const ASTCreateQuery & query, const ColumnsDescription & columns_) - : IStorage({database_name_, table_name_}), global_context(local_context.getGlobalContext()) + : IStorage(table_id_), global_context(local_context.getGlobalContext()) { setColumns(columns_); @@ -224,7 +223,7 @@ StorageLiveView::StorageLiveView( global_context.addDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name_, table_name_)); //FIXME + DatabaseAndTableName(table_id_.database_name, table_id_.table_name)); //FIXME is_temporary = query.temporary; temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds(); @@ -599,7 +598,7 @@ void registerStorageLiveView(StorageFactory & factory) if (!args.attach && !args.local_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); - return StorageLiveView::create(args.table_name, args.database_name, args.local_context, args.query, args.columns); + return StorageLiveView::create(args.table_id, args.local_context, args.query, args.columns); }); } diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 54722a22ff9..63a923df2cc 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -173,8 +173,7 @@ private: UInt64 temporary_live_view_timeout; StorageLiveView( - 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/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 492ac4b8cd4..4072ca706ed 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -97,8 +97,7 @@ namespace ErrorCodes MergeTreeData::MergeTreeData( - const String & database_, - const String & table_, + const StorageID & table_id_, const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, @@ -115,7 +114,7 @@ MergeTreeData::MergeTreeData( bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_) - : IStorage({database_, table_}) + : IStorage(table_id_) , global_context(context_) , merging_params(merging_params_) , partition_by_ast(partition_by_ast_) @@ -124,7 +123,7 @@ MergeTreeData::MergeTreeData( , require_part_metadata(require_part_metadata_) , relative_data_path(relative_data_path_) , broken_part_callback(broken_part_callback_) - , log_name(database_ + "." + table_) + , log_name(table_id_.getNameForLogs()) , log(&Logger::get(log_name)) , storage_settings(std::move(storage_settings_)) , storage_policy(context_.getStoragePolicy(getSettings()->storage_policy)) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 50779a734b3..c4bcd54f4ad 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -331,7 +331,7 @@ public: /// /// require_part_metadata - should checksums.txt and columns.txt exist in the part directory. /// attach - whether the existing table is attached or the new table is created. - MergeTreeData(const String & database_, const String & table_, + MergeTreeData(const StorageID & table_id_, const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 195fa899175..e04749e86cc 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -49,8 +49,12 @@ public: protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) - : IStorage({part_->storage.getDatabaseName(), part_->storage.getTableName() + " (part " + part_->name + ")"}, - part_->storage.getVirtuals()), part(part_) + : IStorage({ + part_->storage.getDatabaseName(), + part_->storage.getTableName() + " (part " + part_->name + ")" + } + , part_->storage.getVirtuals()) + , part(part_) { setColumns(part_->storage.getColumns()); setIndices(part_->storage.getIndices()); diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 633699d1529..46e55b75b4f 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -639,14 +639,14 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) return StorageReplicatedMergeTree::create( - zookeeper_path, replica_name, args.attach, args.database_name, args.table_name, args.relative_data_path, + zookeeper_path, replica_name, args.attach, args.table_id, args.relative_data_path, args.columns, indices_description, args.constraints, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, std::move(storage_settings), args.has_force_restore_data_flag); else return StorageMergeTree::create( - args.database_name, args.table_name, args.relative_data_path, args.columns, indices_description, + args.table_id, args.relative_data_path, args.columns, indices_description, args.constraints, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, std::move(storage_settings), args.has_force_restore_data_flag); diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index e01f1594b85..cbdd4236071 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -56,18 +56,27 @@ namespace ErrorCodes } -StorageBuffer::StorageBuffer(const std::string & database_name_, const std::string & table_name_, - const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, +StorageBuffer::StorageBuffer( + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, Context & context_, - size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, - const String & destination_database_, const String & destination_table_, bool allow_materialized_) - : IStorage({database_name_, table_name_}), - global_context(context_), - num_shards(num_shards_), buffers(num_shards_), - min_thresholds(min_thresholds_), max_thresholds(max_thresholds_), - destination_database(destination_database_), destination_table(destination_table_), - no_destination(destination_database.empty() && destination_table.empty()), - allow_materialized(allow_materialized_), log(&Logger::get("StorageBuffer (" + table_name_ + ")")) + size_t num_shards_, + const Thresholds & min_thresholds_, + const Thresholds & max_thresholds_, + const String & destination_database_, + const String & destination_table_, + bool allow_materialized_) + : IStorage(table_id_) + , global_context(context_) + , num_shards(num_shards_), buffers(num_shards_) + , min_thresholds(min_thresholds_) + , max_thresholds(max_thresholds_) + , destination_database(destination_database_) + , destination_table(destination_table_) + , no_destination(destination_database.empty() && destination_table.empty()) + , allow_materialized(allow_materialized_) + , log(&Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) { setColumns(columns_); setConstraints(constraints_); @@ -750,8 +759,9 @@ void registerStorageBuffer(StorageFactory & factory) UInt64 max_bytes = applyVisitor(FieldVisitorConvertToNumber(), engine_args[8]->as().value); return StorageBuffer::create( - args.database_name, - args.table_name, args.columns, args.constraints, + args.table_id, + args.columns, + args.constraints, args.context, num_buckets, StorageBuffer::Thresholds{min_time, min_rows, min_bytes}, diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index ea08a9bd61a..9c27412e849 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -135,11 +135,17 @@ protected: /** num_shards - the level of internal parallelism (the number of independent buffers) * The buffer is flushed if all minimum thresholds or at least one of the maximum thresholds are exceeded. */ - StorageBuffer(const std::string & database_name_, const std::string & table_name_, - const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, + StorageBuffer( + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, Context & context_, - size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, - const String & destination_database_, const String & destination_table_, bool allow_materialized_); + size_t num_shards_, + const Thresholds & min_thresholds_, + const Thresholds & max_thresholds_, + const String & destination_database_, + const String & destination_table_, + bool allow_materialized_); }; } diff --git a/dbms/src/Storages/StorageDictionary.cpp b/dbms/src/Storages/StorageDictionary.cpp index 1bfa0e86b14..51914b53ae6 100644 --- a/dbms/src/Storages/StorageDictionary.cpp +++ b/dbms/src/Storages/StorageDictionary.cpp @@ -24,15 +24,14 @@ namespace ErrorCodes StorageDictionary::StorageDictionary( - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const Context & context, bool attach, const String & dictionary_name_) - : IStorage({database_name_, table_name_}), - dictionary_name(dictionary_name_), - logger(&Poco::Logger::get("StorageDictionary")) + : IStorage(table_id_) + , dictionary_name(dictionary_name_) + , logger(&Poco::Logger::get("StorageDictionary")) { setColumns(columns_); @@ -107,7 +106,7 @@ void registerStorageDictionary(StorageFactory & factory) String dictionary_name = args.engine_args[0]->as().value.safeGet(); return StorageDictionary::create( - args.database_name, args.table_name, args.columns, args.context, args.attach, dictionary_name); + args.table_id, args.columns, args.context, args.attach, dictionary_name); }); } diff --git a/dbms/src/Storages/StorageDictionary.h b/dbms/src/Storages/StorageDictionary.h index 57f8a3d2907..1e741a05094 100644 --- a/dbms/src/Storages/StorageDictionary.h +++ b/dbms/src/Storages/StorageDictionary.h @@ -64,8 +64,7 @@ private: protected: StorageDictionary( - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const Context & context, bool attach, diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 81f60340ad7..a47afda5ecf 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -205,8 +205,7 @@ static ExpressionActionsPtr buildShardingKeyExpression(const ASTPtr & sharding_k } StorageDistributed::StorageDistributed( - const String & database_name_, - const String & table_name_, + const StorageID & id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & remote_database_, @@ -216,12 +215,18 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & relative_data_path_, bool attach_) - : IStorage({database_name_, table_name_}, ColumnsDescription({ - {"_shard_num", std::make_shared()}, - }, true)), - remote_database(remote_database_), remote_table(remote_table_), - global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_), - path(relative_data_path_.empty() ? "" : (context_.getPath() + relative_data_path_)) + : IStorage(id_, + ColumnsDescription( + { + {"_shard_num", std::make_shared()}, + }, + true)) + , remote_database(remote_database_) + , remote_table(remote_table_) + , global_context(context_) + , cluster_name(global_context.getMacros()->expand(cluster_name_)) + , has_sharding_key(sharding_key_) + , path(relative_data_path_.empty() ? "" : (context_.getPath() + relative_data_path_)) { setColumns(columns_); setConstraints(constraints_); @@ -236,15 +241,14 @@ StorageDistributed::StorageDistributed( if (!attach_ && !cluster_name.empty()) { size_t num_local_shards = global_context.getCluster(cluster_name)->getLocalShardCount(); - if (num_local_shards && remote_database == database_name_ && remote_table == table_name_) - throw Exception("Distributed table " + table_name_ + " looks at itself", ErrorCodes::INFINITE_LOOP); + if (num_local_shards && remote_database == id_.database_name && remote_table == id_.table_name) + throw Exception("Distributed table " + id_.table_name + " looks at itself", ErrorCodes::INFINITE_LOOP); } } StorageDistributed::StorageDistributed( - const String & database_name_, - const String & table_name_, + const StorageID & id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ASTPtr remote_table_function_ptr_, @@ -253,34 +257,34 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & relative_data_path_, bool attach) - : StorageDistributed(database_name_, table_name_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, relative_data_path_, attach) + : StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, relative_data_path_, attach) { - remote_table_function_ptr = remote_table_function_ptr_; + remote_table_function_ptr = std::move(remote_table_function_ptr_); } StoragePtr StorageDistributed::createWithOwnCluster( - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const String & remote_database_, /// database on remote servers. const String & remote_table_, /// The name of the table on the remote servers. ClusterPtr owned_cluster_, const Context & context_) { - auto res = create(String{}, table_name_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false); - res->owned_cluster = owned_cluster_; + auto res = create(table_id_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false); + res->owned_cluster = std::move(owned_cluster_); return res; } StoragePtr StorageDistributed::createWithOwnCluster( - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, ASTPtr & remote_table_function_ptr_, ClusterPtr & owned_cluster_, const Context & context_) { - auto res = create(String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); + auto res = create(table_id_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); res->owned_cluster = owned_cluster_; return res; } @@ -647,7 +651,7 @@ void registerStorageDistributed(StorageFactory & factory) } return StorageDistributed::create( - args.database_name, args.table_name, args.columns, args.constraints, + args.table_id, args.columns, args.constraints, remote_database, remote_table, cluster_name, args.context, sharding_key, args.relative_data_path, args.attach); diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 7b408b361e0..acd66cf106f 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -37,7 +37,7 @@ public: ~StorageDistributed() override; static StoragePtr createWithOwnCluster( - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const String & remote_database_, /// database on remote servers. const String & remote_table_, /// The name of the table on the remote servers. @@ -45,7 +45,7 @@ public: const Context & context_); static StoragePtr createWithOwnCluster( - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, ASTPtr & remote_table_function_ptr_, /// Table function ptr. ClusterPtr & owned_cluster_, @@ -159,8 +159,7 @@ public: protected: StorageDistributed( - const String & database_name_, - const String & table_name_, + const StorageID & id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & remote_database_, @@ -172,8 +171,7 @@ protected: bool attach_); StorageDistributed( - const String & database_name, - const String & table_name_, + const StorageID & id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ASTPtr remote_table_function_ptr_, diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 59828932eb2..b63e0b9e7c3 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -41,13 +42,10 @@ void StorageFactory::registerStorage(const std::string & name, Creator creator) StoragePtr StorageFactory::get( const ASTCreateQuery & query, const String & relative_data_path, - const String & table_name, - const String & database_name, // TODO remove Context & local_context, Context & context, const ColumnsDescription & columns, const ConstraintsDescription & constraints, - bool attach, bool has_force_restore_data_flag) const { String name; @@ -151,13 +149,12 @@ StoragePtr StorageFactory::get( .storage_def = storage_def, .query = query, .relative_data_path = relative_data_path, - .table_name = table_name, - .database_name = database_name, //TODO remove + .table_id = StorageID(query.database, query.table, query.uuid), .local_context = local_context, .context = context, .columns = columns, .constraints = constraints, - .attach = attach, + .attach = query.attach, .has_force_restore_data_flag = has_force_restore_data_flag }; diff --git a/dbms/src/Storages/StorageFactory.h b/dbms/src/Storages/StorageFactory.h index 945bfaee0a4..8921fe4addd 100644 --- a/dbms/src/Storages/StorageFactory.h +++ b/dbms/src/Storages/StorageFactory.h @@ -14,6 +14,7 @@ namespace DB class Context; class ASTCreateQuery; class ASTStorage; +struct StorageID; /** Allows to create a table by the name and parameters of the engine. @@ -35,8 +36,7 @@ public: /// Path to table data. /// Relative to from server config (possibly of some of some for *MergeTree) const String & relative_data_path; - const String & table_name; - const String & database_name; //TODO remove + const StorageID & table_id; Context & local_context; Context & context; const ColumnsDescription & columns; @@ -50,13 +50,10 @@ public: StoragePtr get( const ASTCreateQuery & query, const String & relative_data_path, - const String & table_name, - const String & database_name, Context & local_context, Context & context, const ColumnsDescription & columns, const ConstraintsDescription & constraints, - bool attach, bool has_force_restore_data_flag) const; /// Register a table engine by its name. diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index c2ef924f2a4..5659984dc0a 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -159,8 +159,10 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu } StorageFile::StorageFile(CommonArguments args) - : IStorage({args.database_name, args.table_name}), format_name(args.format_name), - context_global(args.context), compression_method(args.compression_method) + : IStorage(args.table_id) + , format_name(args.format_name) + , context_global(args.context) + , compression_method(args.compression_method) { setColumns(args.columns); setConstraints(args.constraints); @@ -366,7 +368,7 @@ void registerStorageFile(StorageFactory & factory) String format_name = engine_args[0]->as().value.safeGet(); String compression_method; - StorageFile::CommonArguments common_args{args.database_name, args.table_name, format_name, + StorageFile::CommonArguments common_args{args.table_id, format_name, args.columns, args.constraints, args.context, compression_method}; if (engine_args.size() == 1) /// Table in database diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index d7c1ea4d5d1..4112f303e68 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -42,8 +42,7 @@ public: struct CommonArguments { - const std::string & database_name; - const std::string & table_name; + const StorageID & table_id; const std::string & format_name; const ColumnsDescription & columns; const ConstraintsDescription & constraints; diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 95c0617f015..9468c8747ea 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -33,14 +33,13 @@ namespace ErrorCodes } StorageHDFS::StorageHDFS(const String & uri_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, Context & context_, const String & compression_method_ = "") - : IStorage({database_name_, table_name_}) + : IStorage(table_id_) , uri(uri_) , format_name(format_name_) , context(context_) @@ -249,7 +248,7 @@ void registerStorageHDFS(StorageFactory & factory) compression_method = engine_args[2]->as().value.safeGet(); } else compression_method = "auto"; - return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context, compression_method); + return StorageHDFS::create(url, args.table_id, format_name, args.columns, args.constraints, args.context, compression_method); }); } diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index 1ce2163099a..42f1891597e 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -30,8 +30,7 @@ public: protected: StorageHDFS(const String & uri_, - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, diff --git a/dbms/src/Storages/StorageInput.cpp b/dbms/src/Storages/StorageInput.cpp index eece306a20a..4f41ba20dfa 100644 --- a/dbms/src/Storages/StorageInput.cpp +++ b/dbms/src/Storages/StorageInput.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes extern const int INVALID_USAGE_OF_INPUT; } -StorageInput::StorageInput(const String &table_name_, const ColumnsDescription & columns_) +StorageInput::StorageInput(const String & table_name_, const ColumnsDescription & columns_) : IStorage({"", table_name_}, columns_) { setColumns(columns_); diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index efe332ef9a1..a9eabb78874 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -29,8 +29,7 @@ namespace ErrorCodes StorageJoin::StorageJoin( const String & relative_path_, - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const Names & key_names_, bool use_nulls_, SizeLimits limits_, @@ -40,7 +39,7 @@ StorageJoin::StorageJoin( const ConstraintsDescription & constraints_, bool overwrite, const Context & context_) - : StorageSetOrJoinBase{relative_path_, database_name_, table_name_, columns_, constraints_, context_} + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_} , key_names(key_names_) , use_nulls(use_nulls_) , limits(limits_) @@ -164,8 +163,7 @@ void registerStorageJoin(StorageFactory & factory) return StorageJoin::create( args.relative_data_path, - args.database_name, - args.table_name, + args.table_id, key_names, join_use_nulls, SizeLimits{max_rows_in_join, max_bytes_in_join, join_overflow_mode}, diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index dd888e53cb1..f340480d88d 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -61,8 +61,7 @@ private: protected: StorageJoin( const String & relative_path_, - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const Names & key_names_, bool use_nulls_, SizeLimits limits_, diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 7a948eda200..6dc544e2c00 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -419,15 +419,16 @@ void LogBlockOutputStream::writeMarks(MarksForColumns && marks) StorageLog::StorageLog( const std::string & relative_path_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, size_t max_compress_block_size_, const Context & context_) - : IStorage({database_name_, table_name_}), base_path(context_.getPath()), path(base_path + relative_path_), - max_compress_block_size(max_compress_block_size_), - file_checker(path + "sizes.json") + : IStorage(table_id_) + , base_path(context_.getPath()) + , path(base_path + relative_path_) + , max_compress_block_size(max_compress_block_size_) + , file_checker(path + "sizes.json") { setColumns(columns_); setConstraints(constraints_); @@ -648,7 +649,7 @@ void registerStorageLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageLog::create( - args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, + args.relative_data_path, args.table_id, args.columns, args.constraints, args.context.getSettings().max_compress_block_size, args.context); }); } diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index 6cea51f96e4..4d223312832 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -53,8 +53,7 @@ protected: */ StorageLog( const std::string & relative_path_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, size_t max_compress_block_size_, diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 0ffe9c92cb4..38ed5200b65 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -93,13 +93,12 @@ static void checkAllowedQueries(const ASTSelectQuery & query) StorageMaterializedView::StorageMaterializedView( - const String & table_name_, - const String & database_name_, + const StorageID & table_id_, Context & local_context, const ASTCreateQuery & query, const ColumnsDescription & columns_, bool attach_) - : IStorage({database_name_, table_name_}), global_context(local_context.getGlobalContext()) + : IStorage(table_id_), global_context(local_context.getGlobalContext()) { setColumns(columns_); @@ -125,7 +124,7 @@ StorageMaterializedView::StorageMaterializedView( if (!select_table_name.empty()) global_context.addDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name_, table_name_)); //FIXME + DatabaseAndTableName(table_id_.database_name, table_id_.table_name)); //FIXME // If the destination table is not set, use inner table if (!query.to_table.empty()) @@ -135,8 +134,8 @@ StorageMaterializedView::StorageMaterializedView( } else { - target_database_name = database_name_; - target_table_name = generateInnerTableName(table_name_); + target_database_name = table_id_.database_name; + target_table_name = generateInnerTableName(table_id_.table_name); has_inner_table = true; } @@ -167,7 +166,7 @@ StorageMaterializedView::StorageMaterializedView( if (!select_table_name.empty()) global_context.removeDependency( DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(database_name_, table_name_)); //FIXME + DatabaseAndTableName(table_id_.database_name, table_id_.table_name)); //FIXME throw; } @@ -388,7 +387,7 @@ void registerStorageMaterializedView(StorageFactory & factory) { /// Pass local_context here to convey setting for inner table return StorageMaterializedView::create( - args.table_name, args.database_name, args.local_context, args.query, + args.table_id, args.local_context, args.query, args.columns, args.attach); }); } diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index e8e3272a3c9..a86f49cd0f6 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -78,8 +78,7 @@ private: protected: StorageMaterializedView( - 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/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 0489f7bb115..b2cb5682f6d 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -74,8 +74,8 @@ private: }; -StorageMemory::StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) - : IStorage({database_name_, table_name_}) +StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) + : IStorage(table_id_) { setColumns(std::move(columns_description_)); setConstraints(std::move(constraints_)); @@ -145,7 +145,7 @@ void registerStorageMemory(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageMemory::create(args.database_name, args.table_name, args.columns, args.constraints); + return StorageMemory::create(args.table_id, args.columns, args.constraints); }); } diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index d6eb40ef7d0..1e66b17606b 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -49,7 +49,7 @@ private: std::mutex mutex; protected: - StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_, ConstraintsDescription constraints_); + StorageMemory(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_); }; } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 14179773fda..8c955e27ace 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -46,13 +46,12 @@ namespace ErrorCodes StorageMerge::StorageMerge( - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const String & source_database_, const String & table_name_regexp_, const Context & context_) - : IStorage({database_name_, table_name_}, ColumnsDescription({{"_table", std::make_shared()}}, true)) + : IStorage(table_id_, ColumnsDescription({{"_table", std::make_shared()}}, true)) , source_database(source_database_) , table_name_regexp(table_name_regexp_) , global_context(context_) @@ -494,7 +493,7 @@ void registerStorageMerge(StorageFactory & factory) String table_name_regexp = engine_args[1]->as().value.safeGet(); return StorageMerge::create( - args.database_name, args.table_name, args.columns, + args.table_id, args.columns, source_database, table_name_regexp, args.context); }); } diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index b14e8972dff..b212af55009 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -65,8 +65,7 @@ private: protected: StorageMerge( - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const String & source_database_, const String & table_name_regexp_, diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 29623232b3f..14189f6ef6f 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -53,8 +53,7 @@ namespace ActionLocks StorageMergeTree::StorageMergeTree( - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, @@ -70,13 +69,25 @@ StorageMergeTree::StorageMergeTree( const MergingParams & merging_params_, std::unique_ptr storage_settings_, bool has_force_restore_data_flag) - : MergeTreeData(database_name_, table_name_, relative_data_path_, - columns_, indices_, constraints_, - context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, - sample_by_ast_, ttl_table_ast_, merging_params_, - std::move(storage_settings_), false, attach), - reader(*this), writer(*this), - merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()) + : MergeTreeData(table_id_, + relative_data_path_, + columns_, + indices_, + constraints_, + context_, + date_column_name, + partition_by_ast_, + order_by_ast_, + primary_key_ast_, + sample_by_ast_, + ttl_table_ast_, + merging_params_, + std::move(storage_settings_), + false, /// require_part_metadata + attach) + , reader(*this) + , writer(*this) + , merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()) { loadDataParts(has_force_restore_data_flag); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index b907622513d..bda6a9a84c5 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -147,8 +147,7 @@ protected: * See MergeTreeData constructor for comments on parameters. */ StorageMergeTree( - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, diff --git a/dbms/src/Storages/StorageMySQL.cpp b/dbms/src/Storages/StorageMySQL.cpp index b89a8a8a98a..9e4199c3a8a 100644 --- a/dbms/src/Storages/StorageMySQL.cpp +++ b/dbms/src/Storages/StorageMySQL.cpp @@ -37,8 +37,7 @@ String backQuoteMySQL(const String & x) } StorageMySQL::StorageMySQL( - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, mysqlxx::Pool && pool_, const std::string & remote_database_name_, const std::string & remote_table_name_, @@ -47,7 +46,7 @@ StorageMySQL::StorageMySQL( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_) - : IStorage({database_name_, table_name_}) + : IStorage(table_id_) , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) , replace_query{replace_query_} @@ -234,8 +233,7 @@ void registerStorageMySQL(StorageFactory & factory) ErrorCodes::BAD_ARGUMENTS); return StorageMySQL::create( - args.database_name, - args.table_name, + args.table_id, std::move(pool), remote_database, remote_table, diff --git a/dbms/src/Storages/StorageMySQL.h b/dbms/src/Storages/StorageMySQL.h index af486cbd964..574221377dc 100644 --- a/dbms/src/Storages/StorageMySQL.h +++ b/dbms/src/Storages/StorageMySQL.h @@ -21,8 +21,7 @@ class StorageMySQL : public ext::shared_ptr_helper, public IStorag friend struct ext::shared_ptr_helper; public: StorageMySQL( - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, mysqlxx::Pool && pool_, const std::string & remote_database_name_, const std::string & remote_table_name_, diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 67d2ea0e48e..5979a00497b 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -26,7 +26,7 @@ void registerStorageNull(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageNull::create(args.database_name, args.table_name, args.columns, args.constraints); + return StorageNull::create(args.table_id, args.columns, args.constraints); }); } diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index f52780c9993..ec0503decfb 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -42,8 +42,8 @@ public: private: protected: - StorageNull(String database_name_, String table_name_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) - : IStorage({database_name_, table_name_}) + StorageNull(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_) + : IStorage(table_id_) { setColumns(std::move(columns_description_)); setConstraints(std::move(constraints_)); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 05870c39eef..ea437a73b88 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -190,8 +190,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const String & zookeeper_path_, const String & replica_name_, bool attach, - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, @@ -206,17 +205,34 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const MergingParams & merging_params_, std::unique_ptr settings_, bool has_force_restore_data_flag) - : MergeTreeData(database_name_, table_name_, relative_data_path_, - columns_, indices_, constraints_, - context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, - sample_by_ast_, ttl_table_ast_, merging_params_, - std::move(settings_), true, attach, - [this] (const std::string & name) { enqueuePartForCheck(name); }), - zookeeper_path(global_context.getMacros()->expand(zookeeper_path_, database_name_, table_name_)), - replica_name(global_context.getMacros()->expand(replica_name_, database_name_, table_name_)), - reader(*this), writer(*this), merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()), - queue(*this), fetcher(*this), cleanup_thread(*this), alter_thread(*this), - part_check_thread(*this), restarting_thread(*this) + : MergeTreeData(table_id_, + relative_data_path_, + columns_, + indices_, + constraints_, + context_, + date_column_name, + partition_by_ast_, + order_by_ast_, + primary_key_ast_, + sample_by_ast_, + ttl_table_ast_, + merging_params_, + std::move(settings_), + true, /// require_part_metadata + attach, + [this] (const std::string & name) { enqueuePartForCheck(name); }) + , zookeeper_path(global_context.getMacros()->expand(zookeeper_path_, table_id_.database_name, table_id_.table_name)) + , replica_name(global_context.getMacros()->expand(replica_name_, table_id_.database_name, table_id_.table_name)) + , reader(*this) + , writer(*this) + , merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()) + , queue(*this) + , fetcher(*this) + , cleanup_thread(*this) + , alter_thread(*this) + , part_check_thread(*this) + , restarting_thread(*this) { if (!zookeeper_path.empty() && zookeeper_path.back() == '/') zookeeper_path.resize(zookeeper_path.size() - 1); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 95e5a9a25fd..e153f188d85 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -537,7 +537,7 @@ protected: const String & zookeeper_path_, const String & replica_name_, bool attach, - const String & database_name_, const String & name_, + const StorageID & table_id_, const String & relative_data_path_, const ColumnsDescription & columns_, const IndicesDescription & indices_, diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 91ae5eaebd7..5021645b617 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -124,15 +124,14 @@ namespace StorageS3::StorageS3( const Poco::URI & uri_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const String & format_name_, UInt64 min_upload_part_size_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, Context & context_, const String & compression_method_ = "") - : IStorage({database_name_, table_name_}, columns_) + : IStorage({table_id_}, columns_) , uri(uri_) , context_global(context_) , format_name(format_name_) @@ -204,7 +203,7 @@ void registerStorageS3(StorageFactory & factory) compression_method = engine_args[2]->as().value.safeGet(); } else compression_method = "auto"; - return StorageS3::create(uri, args.database_name, args.table_name, format_name, min_upload_part_size, args.columns, args.constraints, args.context); + return StorageS3::create(uri, args.table_id, format_name, min_upload_part_size, args.columns, args.constraints, args.context); }); } } diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index 093f247ab48..361a43b720c 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -18,8 +18,7 @@ class StorageS3 : public ext::shared_ptr_helper, public IStorage public: StorageS3( const Poco::URI & uri_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const String & format_name_, UInt64 min_upload_part_size_, const ColumnsDescription & columns_, diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index 81267cbc4f6..30f20b7df57 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -89,12 +89,11 @@ BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageSetOrJoinBase::StorageSetOrJoinBase( const String & relative_path_, - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_) - : IStorage({database_name_, table_name_}) + : IStorage(table_id_) { setColumns(columns_); setConstraints(constraints_); @@ -110,12 +109,11 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( StorageSet::StorageSet( const String & relative_path_, - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_) - : StorageSetOrJoinBase{relative_path_, database_name_, table_name_, columns_, constraints_, context_}, + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_}, set(std::make_shared(SizeLimits(), false)) { Block header = getSampleBlock(); @@ -222,7 +220,7 @@ void registerStorageSet(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageSet::create(args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, args.context); + return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context); }); } diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index 8934b16cd7d..41e2e9ccdd6 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -28,8 +28,7 @@ public: protected: StorageSetOrJoinBase( const String & relative_path_, - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_); @@ -80,8 +79,7 @@ private: protected: StorageSet( const String & relative_path_, - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_); diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index e44fe9b505f..114a097a337 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -195,18 +195,18 @@ private: StorageStripeLog::StorageStripeLog( const std::string & relative_path_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool attach, size_t max_compress_block_size_, const Context & context_) - : IStorage({database_name_, table_name_}), - base_path(context_.getPath()), path(base_path + relative_path_), - max_compress_block_size(max_compress_block_size_), - file_checker(path + "sizes.json"), - log(&Logger::get("StorageStripeLog")) + : IStorage(table_id_) + , base_path(context_.getPath()) + , path(base_path + relative_path_) + , max_compress_block_size(max_compress_block_size_) + , file_checker(path + "sizes.json") + , log(&Logger::get("StorageStripeLog")) { setColumns(columns_); setConstraints(constraints_); @@ -315,7 +315,7 @@ void registerStorageStripeLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageStripeLog::create( - args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, + args.relative_data_path, args.table_id, args.columns, args.constraints, args.attach, args.context.getSettings().max_compress_block_size, args.context); }); } diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 5d31c739896..0b863c0c4dd 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -69,8 +69,7 @@ private: protected: StorageStripeLog( const std::string & relative_path_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool attach, diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 090809009b4..aac7ca4768a 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -324,17 +324,18 @@ void TinyLogBlockOutputStream::write(const Block & block) StorageTinyLog::StorageTinyLog( const std::string & relative_path_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool attach, size_t max_compress_block_size_, const Context & context_) - : IStorage({database_name_, table_name_}), base_path(context_.getPath()), path(base_path + relative_path_), - max_compress_block_size(max_compress_block_size_), - file_checker(path + "sizes.json"), - log(&Logger::get("StorageTinyLog")) + : IStorage(table_id_) + , base_path(context_.getPath()) + , path(base_path + relative_path_) + , max_compress_block_size(max_compress_block_size_) + , file_checker(path + "sizes.json") + , log(&Logger::get("StorageTinyLog")) { setColumns(columns_); setConstraints(constraints_); @@ -448,7 +449,7 @@ void registerStorageTinyLog(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageTinyLog::create( - args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, + args.relative_data_path, args.table_id, args.columns, args.constraints, args.attach, args.context.getSettings().max_compress_block_size, args.context); }); } diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 17b5e6e94cc..1e40efc1183 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -72,8 +72,7 @@ private: protected: StorageTinyLog( const std::string & relative_path_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool attach, diff --git a/dbms/src/Storages/StorageURL.cpp b/dbms/src/Storages/StorageURL.cpp index 583eac646bd..c797aeb0a52 100644 --- a/dbms/src/Storages/StorageURL.cpp +++ b/dbms/src/Storages/StorageURL.cpp @@ -29,13 +29,16 @@ namespace ErrorCodes IStorageURLBase::IStorageURLBase( const Poco::URI & uri_, const Context & context_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & compression_method_) - : IStorage({database_name_, table_name_}), uri(uri_), context_global(context_), compression_method(compression_method_), format_name(format_name_) + : IStorage(table_id_) + , uri(uri_) + , context_global(context_) + , compression_method(compression_method_) + , format_name(format_name_) { setColumns(columns_); setConstraints(constraints_); @@ -227,7 +230,7 @@ void registerStorageURL(StorageFactory & factory) return StorageURL::create( uri, - args.database_name, args.table_name, + args.table_id, format_name, args.columns, args.constraints, args.context, compression_method); diff --git a/dbms/src/Storages/StorageURL.h b/dbms/src/Storages/StorageURL.h index 4f752ea4ec4..b0fec1527c1 100644 --- a/dbms/src/Storages/StorageURL.h +++ b/dbms/src/Storages/StorageURL.h @@ -30,8 +30,7 @@ protected: IStorageURLBase( const Poco::URI & uri_, const Context & context_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & id_, const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -70,14 +69,13 @@ class StorageURL : public ext::shared_ptr_helper, public IStorageURL public: StorageURL( const Poco::URI & uri_, - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, Context & context_, const String & compression_method_) - : IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_, constraints_, compression_method_) + : IStorageURLBase(uri_, context_, table_id_, format_name_, columns_, constraints_, compression_method_) { } diff --git a/dbms/src/Storages/StorageValues.cpp b/dbms/src/Storages/StorageValues.cpp index ec9cf82a7e7..080343a0956 100644 --- a/dbms/src/Storages/StorageValues.cpp +++ b/dbms/src/Storages/StorageValues.cpp @@ -7,8 +7,8 @@ namespace DB { -StorageValues::StorageValues(const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const Block & res_block_) - : IStorage({database_name_, table_name_}), res_block(res_block_) +StorageValues::StorageValues(const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_) + : IStorage(table_id_), res_block(res_block_) { setColumns(columns_); } diff --git a/dbms/src/Storages/StorageValues.h b/dbms/src/Storages/StorageValues.h index 67858516213..14f8e2cb4e3 100644 --- a/dbms/src/Storages/StorageValues.h +++ b/dbms/src/Storages/StorageValues.h @@ -27,7 +27,7 @@ private: Block res_block; protected: - StorageValues(const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_, const Block & res_block_); + StorageValues(const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_); }; } diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index 6289db95726..d083e0966a1 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -27,11 +27,10 @@ namespace ErrorCodes StorageView::StorageView( - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const ASTCreateQuery & query, const ColumnsDescription & columns_) - : IStorage({database_name_, table_name_}) + : IStorage(table_id_) { setColumns(columns_); @@ -106,7 +105,7 @@ void registerStorageView(StorageFactory & factory) if (args.query.storage) throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY); - return StorageView::create(args.database_name, args.table_name, args.query, args.columns); + return StorageView::create(args.table_id, args.query, args.columns); }); } diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index 709082b58ef..3a78c474456 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -35,8 +35,7 @@ private: protected: StorageView( - const String & database_name_, - const String & table_name_, + const StorageID & table_id_, const ASTCreateQuery & query, const ColumnsDescription & columns_); }; diff --git a/dbms/src/Storages/StorageXDBC.cpp b/dbms/src/Storages/StorageXDBC.cpp index bab751e4f36..8b13f60e6a4 100644 --- a/dbms/src/Storages/StorageXDBC.cpp +++ b/dbms/src/Storages/StorageXDBC.cpp @@ -24,15 +24,20 @@ namespace ErrorCodes StorageXDBC::StorageXDBC( - const std::string & database_name_, - const std::string & table_name_, + const StorageID & table_id_, const std::string & remote_database_name_, const std::string & remote_table_name_, const ColumnsDescription & columns_, const Context & context_, const BridgeHelperPtr bridge_helper_) /// Please add support for constraints as soon as StorageODBC or JDBC will support insertion. - : IStorageURLBase(Poco::URI(), context_, database_name_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_, ConstraintsDescription{}, "" /* CompressionMethod */) + : IStorageURLBase(Poco::URI(), + context_, + table_id_, + IXDBCBridgeHelper::DEFAULT_FORMAT, + columns_, + ConstraintsDescription{}, + "" /* CompressionMethod */) , bridge_helper(bridge_helper_) , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) @@ -120,7 +125,7 @@ namespace BridgeHelperPtr bridge_helper = std::make_shared>(args.context, args.context.getSettingsRef().http_receive_timeout.value, engine_args[0]->as().value.safeGet()); - return std::make_shared(args.database_name, args.table_name, + return std::make_shared(args.table_id, engine_args[1]->as().value.safeGet(), engine_args[2]->as().value.safeGet(), args.columns, diff --git a/dbms/src/Storages/StorageXDBC.h b/dbms/src/Storages/StorageXDBC.h index c658fc5c8a1..e7cf2932785 100644 --- a/dbms/src/Storages/StorageXDBC.h +++ b/dbms/src/Storages/StorageXDBC.h @@ -24,8 +24,7 @@ public: unsigned num_streams) override; - StorageXDBC(const std::string & database_name_, - const std::string & table_name_, + StorageXDBC(const StorageID & table_id_, const std::string & remote_database_name, const std::string & remote_table_name, const ColumnsDescription & columns_, diff --git a/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp index d615d1fb8ad..bbe7a8625ad 100644 --- a/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/dbms/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -30,7 +30,7 @@ struct State { registerFunctions(); DatabasePtr database = std::make_shared("test"); - database->attachTable("table", StorageMemory::create("test", "table", ColumnsDescription{columns}, ConstraintsDescription{})); + database->attachTable("table", StorageMemory::create(StorageID("test", "table"), ColumnsDescription{columns}, ConstraintsDescription{})); context.makeGlobalContext(); context.addDatabase("test", database); context.setCurrentDatabase("test"); diff --git a/dbms/src/Storages/tests/storage_log.cpp b/dbms/src/Storages/tests/storage_log.cpp index f06988504f7..cd5f1ff849c 100644 --- a/dbms/src/Storages/tests/storage_log.cpp +++ b/dbms/src/Storages/tests/storage_log.cpp @@ -29,7 +29,7 @@ try context.makeGlobalContext(); context.setPath("./"); - StoragePtr table = StorageLog::create("./", "test", "test", ColumnsDescription{names_and_types}, ConstraintsDescription{}, 1048576, context); + StoragePtr table = StorageLog::create("./", StorageID("test", "test"), ColumnsDescription{names_and_types}, ConstraintsDescription{}, 1048576, context); table->startup(); /// write into it diff --git a/dbms/src/TableFunctions/ITableFunctionXDBC.cpp b/dbms/src/TableFunctions/ITableFunctionXDBC.cpp index c90ed0f34fb..25e28abd58d 100644 --- a/dbms/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/dbms/src/TableFunctions/ITableFunctionXDBC.cpp @@ -81,7 +81,7 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co readStringBinary(columns_info, buf); NamesAndTypesList columns = NamesAndTypesList::parse(columns_info); - auto result = std::make_shared(getDatabaseName(), table_name, schema_name, remote_table_name, ColumnsDescription{columns}, context, helper); + auto result = std::make_shared(StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, ColumnsDescription{columns}, context, helper); if (!result) throw Exception("Failed to instantiate storage from table function " + getName(), ErrorCodes::UNKNOWN_EXCEPTION); diff --git a/dbms/src/TableFunctions/TableFunctionFile.cpp b/dbms/src/TableFunctions/TableFunctionFile.cpp index c10f4772033..1c8c010c59c 100644 --- a/dbms/src/TableFunctions/TableFunctionFile.cpp +++ b/dbms/src/TableFunctions/TableFunctionFile.cpp @@ -8,7 +8,7 @@ namespace DB StoragePtr TableFunctionFile::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const { - StorageFile::CommonArguments args{getDatabaseName(), table_name, format, columns,ConstraintsDescription{}, + StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, columns,ConstraintsDescription{}, global_context, compression_method}; return StorageFile::create(source, global_context.getUserFilesPath(), args); diff --git a/dbms/src/TableFunctions/TableFunctionHDFS.cpp b/dbms/src/TableFunctions/TableFunctionHDFS.cpp index 3e8a3d6b954..567ff7b25c7 100644 --- a/dbms/src/TableFunctions/TableFunctionHDFS.cpp +++ b/dbms/src/TableFunctions/TableFunctionHDFS.cpp @@ -12,8 +12,7 @@ StoragePtr TableFunctionHDFS::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const { return StorageHDFS::create(source, - getDatabaseName(), - table_name, + StorageID(getDatabaseName(), table_name), format, columns, ConstraintsDescription{}, diff --git a/dbms/src/TableFunctions/TableFunctionMerge.cpp b/dbms/src/TableFunctions/TableFunctionMerge.cpp index 0cae1cda987..694e219bc5a 100644 --- a/dbms/src/TableFunctions/TableFunctionMerge.cpp +++ b/dbms/src/TableFunctions/TableFunctionMerge.cpp @@ -70,8 +70,7 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Co String table_name_regexp = args[1]->as().value.safeGet(); auto res = StorageMerge::create( - getDatabaseName(), - table_name, + StorageID(getDatabaseName(), table_name), ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}, source_database, table_name_regexp, diff --git a/dbms/src/TableFunctions/TableFunctionMySQL.cpp b/dbms/src/TableFunctions/TableFunctionMySQL.cpp index 820a55c3a2c..5d8ab826ad6 100644 --- a/dbms/src/TableFunctions/TableFunctionMySQL.cpp +++ b/dbms/src/TableFunctions/TableFunctionMySQL.cpp @@ -119,8 +119,7 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co throw Exception("MySQL table " + backQuoteIfNeed(remote_database_name) + "." + backQuoteIfNeed(remote_table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); auto res = StorageMySQL::create( - getDatabaseName(), - table_name, + StorageID(getDatabaseName(), table_name), std::move(pool), remote_database_name, remote_table_name, diff --git a/dbms/src/TableFunctions/TableFunctionRemote.cpp b/dbms/src/TableFunctions/TableFunctionRemote.cpp index 9d0a8024c0e..0a87b73f877 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.cpp +++ b/dbms/src/TableFunctions/TableFunctionRemote.cpp @@ -162,13 +162,13 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C StoragePtr res = remote_table_function_ptr ? StorageDistributed::createWithOwnCluster( - table_name, + StorageID("", table_name), structure_remote_table, remote_table_function_ptr, cluster, context) : StorageDistributed::createWithOwnCluster( - table_name, + StorageID("", table_name), structure_remote_table, remote_database, remote_table, diff --git a/dbms/src/TableFunctions/TableFunctionS3.cpp b/dbms/src/TableFunctions/TableFunctionS3.cpp index a9ee5ebf691..a910be3279f 100644 --- a/dbms/src/TableFunctions/TableFunctionS3.cpp +++ b/dbms/src/TableFunctions/TableFunctionS3.cpp @@ -11,7 +11,7 @@ StoragePtr TableFunctionS3::getStorage( { Poco::URI uri(source); UInt64 min_upload_part_size = global_context.getSettingsRef().s3_min_upload_part_size; - return StorageS3::create(uri, getDatabaseName(), table_name, format, min_upload_part_size, columns, ConstraintsDescription{}, global_context, compression_method); + return StorageS3::create(uri, StorageID(getDatabaseName(), table_name), format, min_upload_part_size, columns, ConstraintsDescription{}, global_context, compression_method); } void registerTableFunctionS3(TableFunctionFactory & factory) diff --git a/dbms/src/TableFunctions/TableFunctionURL.cpp b/dbms/src/TableFunctions/TableFunctionURL.cpp index adb930efa8c..5a1a46edaee 100644 --- a/dbms/src/TableFunctions/TableFunctionURL.cpp +++ b/dbms/src/TableFunctions/TableFunctionURL.cpp @@ -11,7 +11,7 @@ StoragePtr TableFunctionURL::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const { Poco::URI uri(source); - return StorageURL::create(uri, getDatabaseName(), table_name, format, columns, ConstraintsDescription{}, global_context, compression_method); + return StorageURL::create(uri, StorageID(getDatabaseName(), table_name), format, columns, ConstraintsDescription{}, global_context, compression_method); } void registerTableFunctionURL(TableFunctionFactory & factory) diff --git a/dbms/src/TableFunctions/TableFunctionValues.cpp b/dbms/src/TableFunctions/TableFunctionValues.cpp index d4ca0ff4211..ccedb8a85d1 100644 --- a/dbms/src/TableFunctions/TableFunctionValues.cpp +++ b/dbms/src/TableFunctions/TableFunctionValues.cpp @@ -87,7 +87,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C Block res_block = sample_block.cloneWithColumns(std::move(res_columns)); - auto res = StorageValues::create(getDatabaseName(), table_name, columns, res_block); + auto res = StorageValues::create(StorageID(getDatabaseName(), table_name), columns, res_block); res->startup(); return res; } From 0bc2fabbdd3b126ec30a09126caa2dd708ef5b26 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 5 Dec 2019 14:42:13 +0300 Subject: [PATCH 020/743] refactor Context 1 --- .../PushingToViewsBlockOutputStream.cpp | 76 +++++++++---------- .../PushingToViewsBlockOutputStream.h | 6 +- dbms/src/Interpreters/Context.cpp | 40 +++++----- dbms/src/Interpreters/Context.h | 22 +++--- .../Interpreters/InterpreterCreateQuery.cpp | 15 ++-- .../src/Interpreters/InterpreterCreateQuery.h | 2 +- .../Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 32 ++++---- .../src/Interpreters/InterpreterSelectQuery.h | 4 + dbms/src/Storages/IStorage.h | 30 +------- dbms/src/Storages/Kafka/StorageKafka.cpp | 8 +- .../src/Storages/LiveView/StorageLiveView.cpp | 16 ++-- dbms/src/Storages/StorageID.h | 58 ++++++++++++++ dbms/src/Storages/StorageMaterializedView.cpp | 24 +++--- .../Storages/System/StorageSystemTables.cpp | 7 +- 15 files changed, 187 insertions(+), 155 deletions(-) create mode 100644 dbms/src/Storages/StorageID.h diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index a630757b9a5..c43f2e7bf1d 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -18,7 +18,7 @@ namespace DB { PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( - const String & database, const String & table, const StoragePtr & storage_, + const StoragePtr & storage_, const Context & context_, const ASTPtr & query_ptr_, bool no_destination) : storage(storage_), context(context_), query_ptr(query_ptr_) { @@ -32,47 +32,43 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks bool disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); - if (!table.empty()) + auto table_id = storage->getStorageID(); + Dependencies dependencies = context.getDependencies(table_id); + + /// We need special context for materialized views insertions + if (!dependencies.empty()) { - Dependencies dependencies = context.getDependencies(database, table); + views_context = std::make_unique(context); + // Do not deduplicate insertions into MV if the main insertion is Ok + if (disable_deduplication_for_children) + views_context->getSettingsRef().insert_deduplicate = false; + } - /// We need special context for materialized views insertions - if (!dependencies.empty()) + for (const auto & database_table : dependencies) + { + auto dependent_table = context.getTable(database_table.database_name, database_table.table_name); //FIXME + + ASTPtr query; + BlockOutputStreamPtr out; + + if (auto * materialized_view = dynamic_cast(dependent_table.get())) { - views_context = std::make_unique(context); - // Do not deduplicate insertions into MV if the main insertion is Ok - if (disable_deduplication_for_children) - views_context->getSettingsRef().insert_deduplicate = false; + StoragePtr inner_table = materialized_view->getTargetTable(); + query = materialized_view->getInnerQuery(); + std::unique_ptr insert = std::make_unique(); + insert->database = inner_table->getDatabaseName(); + insert->table = inner_table->getTableName(); + ASTPtr insert_query_ptr(insert.release()); + InterpreterInsertQuery interpreter(insert_query_ptr, *views_context); + BlockIO io = interpreter.execute(); + out = io.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()); - for (const auto & database_table : dependencies) - { - auto dependent_table = context.getTable(database_table.first, database_table.second); - - ASTPtr query; - BlockOutputStreamPtr out; - - if (auto * materialized_view = dynamic_cast(dependent_table.get())) - { - StoragePtr inner_table = materialized_view->getTargetTable(); - query = materialized_view->getInnerQuery(); - std::unique_ptr insert = std::make_unique(); - insert->database = inner_table->getDatabaseName(); - insert->table = inner_table->getTableName(); - ASTPtr insert_query_ptr(insert.release()); - InterpreterInsertQuery interpreter(insert_query_ptr, *views_context); - 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 - 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)}); - } + views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out)}); } /* Do not push to destination table if the flag is set */ @@ -161,7 +157,7 @@ void PushingToViewsBlockOutputStream::writePrefix() } catch (Exception & ex) { - ex.addMessage("while write prefix to view " + view.database + "." + view.table); + ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs()); throw; } } @@ -180,7 +176,7 @@ void PushingToViewsBlockOutputStream::writeSuffix() } catch (Exception & ex) { - ex.addMessage("while write prefix to view " + view.database + "." + view.table); + ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs()); throw; } } @@ -250,7 +246,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n } catch (Exception & ex) { - ex.addMessage("while pushing to view " + backQuoteIfNeed(view.database) + "." + backQuoteIfNeed(view.table)); + ex.addMessage("while pushing to view " + view.table_id.getNameForLogs()); throw; } } diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h index c9b0538e470..162c2e1b447 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -17,8 +17,7 @@ class ReplicatedMergeTreeBlockOutputStream; class PushingToViewsBlockOutputStream : public IBlockOutputStream { public: - PushingToViewsBlockOutputStream( - const String & database, const String & table, const StoragePtr & storage_, + PushingToViewsBlockOutputStream(const StoragePtr & storage_, const Context & context_, const ASTPtr & query_ptr_, bool no_destination = false); Block getHeader() const override; @@ -39,8 +38,7 @@ private: struct ViewInfo { ASTPtr query; - String database; - String table; + StorageID table_id; BlockOutputStreamPtr out; }; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index d5049a95d75..23c1570011f 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -599,7 +599,7 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } -bool Context::hasUserProperty(const String & database, const String & table, const String & name) const +bool Context::hasUserProperty(const StorageID & table_id, const String & name) const { auto lock = getLock(); @@ -609,22 +609,22 @@ bool Context::hasUserProperty(const String & database, const String & table, con const auto & props = shared->users_manager->getUser(client_info.current_user)->table_props; - auto db = props.find(database); + auto db = props.find(table_id.database_name); if (db == props.end()) return false; - auto table_props = db->second.find(table); + auto table_props = db->second.find(table_id.table_name); if (table_props == db->second.end()) return false; return !!table_props->second.count(name); } -const String & Context::getUserProperty(const String & database, const String & table, const String & name) const +const String & Context::getUserProperty(const StorageID & table_id, const String & name) const { auto lock = getLock(); const auto & props = shared->users_manager->getUser(client_info.current_user)->table_props; - return props.at(database).at(table).at(name); + return props.at(table_id.database_name).at(table_id.table_name).at(name); } void Context::calculateUserSettings() @@ -728,49 +728,51 @@ void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) c throw Exception("Access denied to database " + database_name + " for user " + client_info.current_user , ErrorCodes::DATABASE_ACCESS_DENIED); } -void Context::addDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where) +//FIXME use uuids if not empty + +void Context::addDependencyUnsafe(const StorageID & from, const StorageID & where) { - checkDatabaseAccessRightsImpl(from.first); - checkDatabaseAccessRightsImpl(where.first); + checkDatabaseAccessRightsImpl(from.database_name); + checkDatabaseAccessRightsImpl(where.database_name); shared->view_dependencies[from].insert(where); // Notify table of dependencies change - auto table = tryGetTable(from.first, from.second); + auto table = tryGetTable(from.database_name, from.table_name); if (table != nullptr) table->updateDependencies(); } -void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where) +void Context::addDependency(const StorageID & from, const StorageID & where) { auto lock = getLock(); addDependencyUnsafe(from, where); } -void Context::removeDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where) +void Context::removeDependencyUnsafe(const StorageID & from, const StorageID & where) { - checkDatabaseAccessRightsImpl(from.first); - checkDatabaseAccessRightsImpl(where.first); + checkDatabaseAccessRightsImpl(from.database_name); + checkDatabaseAccessRightsImpl(where.database_name); shared->view_dependencies[from].erase(where); // Notify table of dependencies change - auto table = tryGetTable(from.first, from.second); + auto table = tryGetTable(from.database_name, from.table_name); if (table != nullptr) table->updateDependencies(); } -void Context::removeDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where) +void Context::removeDependency(const StorageID & from, const StorageID & where) { auto lock = getLock(); removeDependencyUnsafe(from, where); } -Dependencies Context::getDependencies(const String & database_name, const String & table_name) const +Dependencies Context::getDependencies(const StorageID & from) const { auto lock = getLock(); - String db = resolveDatabase(database_name, current_database); + String db = resolveDatabase(from.database_name, current_database); - if (database_name.empty() && tryGetExternalTable(table_name)) + if (from.database_name.empty() && tryGetExternalTable(from.table_name)) { /// Table is temporary. Access granted. } @@ -779,7 +781,7 @@ Dependencies Context::getDependencies(const String & database_name, const String checkDatabaseAccessRightsImpl(db); } - ViewDependencies::const_iterator iter = shared->view_dependencies.find(DatabaseAndTableName(db, table_name)); + ViewDependencies::const_iterator iter = shared->view_dependencies.find(StorageID(db, from.table_name)); if (iter == shared->view_dependencies.end()) return {}; diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index f00fd8046be..fe3545e9730 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -77,6 +77,7 @@ using ActionLocksManagerPtr = std::shared_ptr; class ShellCommand; class ICompressionCodec; class SettingsConstraints; +struct StorageID; class IOutputFormat; using OutputFormatPtr = std::shared_ptr; @@ -89,11 +90,11 @@ class CompiledExpressionCache; /// (database name, table name) //FIXME replace with StorageID -using DatabaseAndTableName = std::pair; +//using DatabaseAndTableName = std::pair; /// Table -> set of table-views that make SELECT from it. -using ViewDependencies = std::map>; -using Dependencies = std::vector; +using ViewDependencies = std::map>; +using Dependencies = std::vector; using TableAndCreateAST = std::pair; using TableAndCreateASTs = std::map; @@ -208,8 +209,8 @@ public: ConfigurationPtr getUsersConfig(); // User property is a key-value pair from the configuration entry: users..databases... - bool hasUserProperty(const String & database, const String & table, const String & name) const; - const String & getUserProperty(const String & database, const String & table, const String & name) const; + bool hasUserProperty(const StorageID & table_id, const String & name) const; + const String & getUserProperty(const StorageID & table_id, const String & name) const; /// Must be called before getClientInfo. void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key); @@ -242,13 +243,13 @@ public: void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address); QuotaForIntervals & getQuota(); - void addDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where); - void removeDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where); - Dependencies getDependencies(const String & database_name, const String & table_name) const; + void addDependency(const StorageID & from, const StorageID & where); + void removeDependency(const StorageID & from, const StorageID & where); + Dependencies getDependencies(const StorageID & from) const; /// Functions where we can lock the context manually - void addDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where); - void removeDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where); + void addDependencyUnsafe(const StorageID & from, const StorageID & where); + void removeDependencyUnsafe(const StorageID & from, const StorageID & where); /// Checking the existence of the table/database. Database can be empty - in this case the current database is used. bool isTableExist(const String & database_name, const String & table_name) const; @@ -361,6 +362,7 @@ public: std::optional getTCPPortSecure() const; /// Get query for the CREATE table. + // TODO do we really need it here? ASTPtr getCreateTableQuery(const String & database_name, const String & table_name) const; ASTPtr getCreateExternalTableQuery(const String & table_name) const; ASTPtr getCreateDatabaseQuery(const String & database_name) const; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 42d6290c525..80ec0f24402 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -562,10 +562,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) 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); - String current_database = context.getCurrentDatabase(); - - String database_name = create.database.empty() ? current_database : create.database; - String table_name = create.table; + auto & database_name = create.database; + auto & table_name = create.table; // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) @@ -576,6 +574,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.attach = true; } + String current_database = context.getCurrentDatabase(); + if (create.database.empty()) + create.database = current_database; if (create.to_database.empty()) create.to_database = current_database; @@ -589,7 +590,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) TableProperties properties = setProperties(create); /// Actually creates table - bool created = doCreateTable(create, properties, database_name); + bool created = doCreateTable(create, properties); if (!created) return {}; @@ -597,14 +598,14 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, - const InterpreterCreateQuery::TableProperties & properties, - const String & database_name) + const InterpreterCreateQuery::TableProperties & properties) { std::unique_ptr guard; String data_path; DatabasePtr database; + const String & database_name = create.database; const String & table_name = create.table; bool need_add_to_database = !create.temporary || create.is_live_view; if (need_add_to_database) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index af868b016ac..b936d83d490 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -67,7 +67,7 @@ private: void checkAccess(const ASTCreateQuery & create); /// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false. - bool doCreateTable(/*const*/ ASTCreateQuery & create, const TableProperties & properties, const String & database_name); + bool doCreateTable(/*const*/ ASTCreateQuery & create, const TableProperties & properties); /// Inserts data in created table if it's CREATE ... SELECT BlockIO fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name); diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 8c95362a6f7..0d425b6b2e1 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -113,7 +113,7 @@ BlockIO InterpreterInsertQuery::execute() if (table->noPushingToViews() && !no_destination) out = table->write(query_ptr, context); else - out = std::make_shared(query.database, query.table, table, context, query_ptr, no_destination); + out = std::make_shared(table, context, query_ptr, no_destination); /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 0f3d5d82f96..efc4541c162 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -114,15 +114,12 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; } -namespace -{ - /// Assumes `storage` is set and the table filter (row-level security) is not empty. -String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr & storage, const Context & context, const Names & prerequisite_columns = {}) +String InterpreterSelectQuery::generateFilterActions(ExpressionActionsPtr & actions, const Names & prerequisite_columns) const { - const auto & db_name = storage->getDatabaseName(); - const auto & table_name = storage->getTableName(); - const auto & filter_str = context.getUserProperty(db_name, table_name, "filter"); + const auto & db_name = table_id.value().database_name; + const auto & table_name = table_id.value().table_name; + const auto & filter_str = context->getUserProperty(table_id.value(), "filter"); /// TODO: implement some AST builders for this kind of stuff ASTPtr query_ast = std::make_shared(); @@ -155,17 +152,15 @@ String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr & table_expr->children.push_back(table_expr->database_and_table_name); /// Using separate expression analyzer to prevent any possible alias injection - auto syntax_result = SyntaxAnalyzer(context).analyze(query_ast, storage->getColumns().getAllPhysical()); - SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context); - ExpressionActionsChain new_chain(context); + auto syntax_result = SyntaxAnalyzer(*context).analyze(query_ast, storage->getColumns().getAllPhysical()); + SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context); + ExpressionActionsChain new_chain(*context); analyzer.appendSelect(new_chain, false); actions = new_chain.getLastActions(); return expr_list->children.at(0)->getColumnName(); } -} - InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, const Context & context_, @@ -316,7 +311,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( } if (storage) + { table_lock = storage->lockStructureForShare(false, context->getInitialQueryId()); + table_id = storage->getStorageID(); + } auto analyze = [&] () { @@ -378,10 +376,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( source_header = storage->getSampleBlockForColumns(required_columns); /// Fix source_header for filter actions. - if (context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (context->hasUserProperty(table_id.value(), "filter")) { filter_info = std::make_shared(); - filter_info->column_name = generateFilterActions(filter_info->actions, storage, *context, required_columns); + filter_info->column_name = generateFilterActions(filter_info->actions, required_columns); source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns()); } } @@ -491,7 +489,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() /// PREWHERE optimization. /// Turn off, if the table filter (row-level security) is applied. - if (storage && !context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (storage && !context->hasUserProperty(table_id.value(), "filter")) { query_analyzer->makeSetsForIndex(query.where()); query_analyzer->makeSetsForIndex(query.prewhere()); @@ -1426,11 +1424,11 @@ void InterpreterSelectQuery::executeFetchColumns( if (storage) { /// Append columns from the table filter to required - if (context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (context->hasUserProperty(table_id.value(), "filter")) { auto initial_required_columns = required_columns; ExpressionActionsPtr actions; - generateFilterActions(actions, storage, *context, initial_required_columns); + generateFilterActions(actions, initial_required_columns); auto required_columns_from_filter = actions->getRequiredColumns(); for (const auto & column : required_columns_from_filter) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 083a4ebe680..d2a8ba9398d 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -239,6 +240,8 @@ private: void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, std::unordered_map & subqueries_for_sets); void executeMergeSorted(QueryPipeline & pipeline, const SortDescription & sort_description, UInt64 limit); + String generateFilterActions(ExpressionActionsPtr & actions, const Names & prerequisite_columns = {}) const; + /// Add ConvertingBlockInputStream to specified header. void unifyStreams(Pipeline & pipeline, Block header); @@ -288,6 +291,7 @@ private: /// Table from where to read data, if not subquery. StoragePtr storage; + std::optional table_id; TableStructureReadLockHolder table_lock; /// Used when we read from prepared input, not table or subquery. diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index d4d757f0111..f069d50f8b3 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -65,35 +66,6 @@ struct ColumnSize } }; -struct StorageID -{ - String database_name; - String table_name; - String uuid; - - StorageID() = delete; - StorageID(const String & database, const String & table, const String & uuid_ = {}) - : database_name(database), table_name(table), uuid(uuid_) {} - - String getFullTableName() const - { - return (database_name.empty() ? "" : database_name + ".") + table_name; - } - - String getNameForLogs() const - { - return "`" + getFullTableName() + "` (UUID = " + uuid +")"; - } - - String getId() const - { - //if (uuid.empty()) - return getFullTableName(); - //else - // return uuid; - } -}; - /** Storage. Describes the table. Responsible for * - storage of the table data; * - the definition in which files (or not in files) the data is stored; diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index db95769227d..a51f26fa1aa 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -292,14 +292,14 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & conf) bool StorageKafka::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached - auto dependencies = global_context.getDependencies(table_id.database_name, table_id.database_name); //FIXME replace with id + auto dependencies = global_context.getDependencies(table_id); if (dependencies.size() == 0) return true; // Check the dependencies are ready? for (const auto & db_tab : dependencies) { - auto table = global_context.tryGetTable(db_tab.first, db_tab.second); + auto table = global_context.tryGetTable(db_tab.database_name, db_tab.table_name); //FIXME if (!table) return false; @@ -309,7 +309,7 @@ bool StorageKafka::checkDependencies(const StorageID & table_id) return false; // Check all its dependencies - if (!checkDependencies(StorageID(db_tab.first, db_tab.second))) //FIXME replace with id + if (!checkDependencies(StorageID(db_tab.database_name, db_tab.table_name))) return false; } @@ -322,7 +322,7 @@ void StorageKafka::threadFunc() { auto table_id = getStorageID(); // Check if at least one direct dependency is attached - auto dependencies = global_context.getDependencies(table_id.database_name, table_id.database_name); //FIXME replace with id + auto dependencies = global_context.getDependencies(table_id); // Keep streaming as long as there are attached views and streaming is not cancelled while (!stream_cancelled && num_created_consumers > 0 && dependencies.size() > 0) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index c041aeef9d4..987996f9cc7 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -222,8 +222,8 @@ StorageLiveView::StorageLiveView( } global_context.addDependency( - DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(table_id_.database_name, table_id_.table_name)); //FIXME + StorageID(select_database_name, select_table_name), + table_id_); //FIXME is_temporary = query.temporary; temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds(); @@ -338,11 +338,11 @@ bool StorageLiveView::getNewBlocks() void StorageLiveView::checkTableCanBeDropped() const { auto table_id = getStorageID(); - Dependencies dependencies = global_context.getDependencies(table_id.database_name, table_id.table_name); //FIXME + 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); } } @@ -365,7 +365,7 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co return; if (storage->hasUsers()) return; - if (!storage->global_context.getDependencies(table_id.database_name, table_id.table_name).empty()) //FIXME + if (!storage->global_context.getDependencies(table_id).empty()) continue; drop_table = true; } @@ -469,8 +469,8 @@ void StorageLiveView::drop(TableStructureWriteLockHolder &) { auto table_id = getStorageID(); global_context.removeDependency( - DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(table_id.database_name, table_id.table_name)); //FIXME + StorageID(select_database_name, select_table_name), + table_id); //FIXME std::lock_guard lock(mutex); is_dropped = true; diff --git a/dbms/src/Storages/StorageID.h b/dbms/src/Storages/StorageID.h new file mode 100644 index 00000000000..cdce84729ff --- /dev/null +++ b/dbms/src/Storages/StorageID.h @@ -0,0 +1,58 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +struct StorageID +{ + String database_name; + String table_name; + String uuid; + + StorageID() = delete; + + StorageID(const String & database, const String & table, const String & uuid_ = {}) + : database_name(database), table_name(table), uuid(uuid_) + { + assert_not_empty(); + } + + String getFullTableName() const + { + return (database_name.empty() ? "" : database_name + ".") + table_name; + } + + String getNameForLogs() const + { + return (database_name.empty() ? "" : backQuoteIfNeed(database_name) + ".") + backQuoteIfNeed(table_name) + " (UUID " + uuid + ")"; + } + + String getId() const + { + //if (uuid.empty()) + return getFullTableName(); + //else + // return uuid; + } + + bool operator<(const StorageID & rhs) const + { + return std::tie(uuid, database_name, table_name) < std::tie(rhs.uuid, rhs.database_name, rhs.table_name); + } + + void assert_not_empty() const + { + if (database_name.empty() && table_name.empty()) + throw Exception("empty table name", ErrorCodes::LOGICAL_ERROR); + } +}; + +} diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 38ed5200b65..9ba30a7d108 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -123,8 +123,8 @@ StorageMaterializedView::StorageMaterializedView( if (!select_table_name.empty()) global_context.addDependency( - DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(table_id_.database_name, table_id_.table_name)); //FIXME + StorageID(select_database_name, select_table_name), + table_id_); //FIXME // If the destination table is not set, use inner table if (!query.to_table.empty()) @@ -165,8 +165,8 @@ StorageMaterializedView::StorageMaterializedView( /// In case of any error we should remove dependency to the view. if (!select_table_name.empty()) global_context.removeDependency( - DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(table_id_.database_name, table_id_.table_name)); //FIXME + StorageID(select_database_name, select_table_name), + table_id_); //FIXME throw; } @@ -234,8 +234,8 @@ void StorageMaterializedView::drop(TableStructureWriteLockHolder &) { auto table_id = getStorageID(); global_context.removeDependency( - DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(table_id.database_name, table_id.table_name)); //FIXME + StorageID(select_database_name, select_table_name), + table_id); //FIXME if (has_inner_table && tryGetTargetTable()) executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_database_name, target_table_name); @@ -314,14 +314,14 @@ void StorageMaterializedView::renameInMemory(const String & new_database_name, c auto table_id = getStorageID(&name_lock); global_context.removeDependencyUnsafe( - DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(table_id.database_name, table_id.table_name)); + StorageID(select_database_name, select_table_name), + table_id); IStorage::renameInMemory(new_database_name, new_table_name, &name_lock); global_context.addDependencyUnsafe( - DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(new_database_name, new_table_name)); + StorageID(select_database_name, select_table_name), + table_id); } void StorageMaterializedView::shutdown() @@ -329,8 +329,8 @@ void StorageMaterializedView::shutdown() auto table_id = getStorageID(); /// Make sure the dependency is removed after DETACH TABLE global_context.removeDependency( - DatabaseAndTableName(select_database_name, select_table_name), - DatabaseAndTableName(table_id.database_name, table_id.table_name)); //FIXME + StorageID(select_database_name, select_table_name), + table_id); //FIXME } StoragePtr StorageMaterializedView::getTargetTable() const diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 364b9dbad0d..a12105add08 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -98,6 +98,7 @@ public: protected: Block readImpl() override { + //FIXME add uuids if (done) return {}; @@ -265,14 +266,14 @@ protected: Array dependencies_database_name_array; if (columns_mask[src_index] || columns_mask[src_index + 1]) { - const auto dependencies = context.getDependencies(database_name, table_name); + const auto dependencies = context.getDependencies(StorageID(database_name, table_name)); dependencies_table_name_array.reserve(dependencies.size()); dependencies_database_name_array.reserve(dependencies.size()); for (const auto & dependency : dependencies) { - dependencies_table_name_array.push_back(dependency.second); - dependencies_database_name_array.push_back(dependency.first); + dependencies_table_name_array.push_back(dependency.table_name); + dependencies_database_name_array.push_back(dependency.database_name); } } From 03b1a576bad4d0cef5b389effe473128797995a1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 5 Dec 2019 16:44:22 +0300 Subject: [PATCH 021/743] refactor Context 2 --- dbms/src/Databases/DatabaseDictionary.cpp | 2 +- dbms/src/Databases/DatabaseDictionary.h | 2 +- dbms/src/Databases/DatabaseMemory.cpp | 18 +++++------ dbms/src/Databases/DatabaseMemory.h | 2 +- dbms/src/Databases/DatabaseMySQL.cpp | 2 +- dbms/src/Databases/DatabaseMySQL.h | 2 +- dbms/src/Databases/DatabaseOnDisk.cpp | 2 +- dbms/src/Databases/DatabaseOnDisk.h | 2 +- dbms/src/Databases/IDatabase.h | 2 +- dbms/src/Interpreters/Context.cpp | 31 ------------------- dbms/src/Interpreters/Context.h | 5 +-- .../Interpreters/InterpreterCreateQuery.cpp | 6 ++-- .../InterpreterShowCreateQuery.cpp | 6 ++-- .../Interpreters/InterpreterSystemQuery.cpp | 2 +- 14 files changed, 23 insertions(+), 61 deletions(-) diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index ec0714ff5fa..ce69f917458 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -119,7 +119,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context, return ast; } -ASTPtr DatabaseDictionary::getCreateDatabaseQuery(const Context & /*context*/) const +ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const { String query; { diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index cd5dde3177c..3155e12b862 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -41,7 +41,7 @@ public: bool empty(const Context & context) const override; - ASTPtr getCreateDatabaseQuery(const Context & context) const override; + ASTPtr getCreateDatabaseQuery() const override; void shutdown() override; diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index 96d8d7e72aa..996d5ca7c84 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -1,18 +1,12 @@ #include #include #include +#include namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_GET_CREATE_TABLE_QUERY; - extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY; - extern const int UNSUPPORTED_METHOD; -} - DatabaseMemory::DatabaseMemory(const String & name_) : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")") {} @@ -33,11 +27,13 @@ void DatabaseMemory::removeTable( detachTable(table_name); } -ASTPtr DatabaseMemory::getCreateDatabaseQuery( - const Context &) const +ASTPtr DatabaseMemory::getCreateDatabaseQuery() const { - //FIXME - throw Exception("There is no CREATE DATABASE query for DatabaseMemory", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY); + auto create_query = std::make_shared(); + create_query->database = database_name; + create_query->set(create_query->storage, std::make_shared()); + create_query->storage->set(create_query->storage->engine, makeASTFunction(getEngineName())); + return create_query; } } diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index c5239fc88ee..5609e6053ce 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -31,7 +31,7 @@ public: const Context & context, const String & table_name) override; - ASTPtr getCreateDatabaseQuery(const Context & context) const override; + ASTPtr getCreateDatabaseQuery() const override; }; } diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index b7e32ad9bb0..a6de9d39717 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -124,7 +124,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam return time_t(local_tables_cache[table_name].modification_time); } -ASTPtr DatabaseMySQL::getCreateDatabaseQuery(const Context &) const +ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const { const auto & create_query = std::make_shared(); create_query->database = database_name; diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index 0776d9cd9c4..b7c6d5fac65 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -30,7 +30,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override; - ASTPtr getCreateDatabaseQuery(const Context & context) const override; + ASTPtr getCreateDatabaseQuery() const override; bool isTableExist(const Context & context, const String & name) const override; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 5e19bfb631c..700a7ea7b71 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -296,7 +296,7 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const Context & context, const St return ast; } -ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const Context & /*context*/) const +ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const { ASTPtr ast; diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index f7a56641fa0..c05c60b9823 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -48,7 +48,7 @@ public: IDatabase & to_database, const String & to_table_name) override; - ASTPtr getCreateDatabaseQuery(const Context & context) const override; + ASTPtr getCreateDatabaseQuery() const override; void drop(const Context & context) override; diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 9a04b7a8257..4cc30877835 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -263,7 +263,7 @@ public: } /// Get the CREATE DATABASE query for current database. - virtual ASTPtr getCreateDatabaseQuery(const Context & context) const = 0; + virtual ASTPtr getCreateDatabaseQuery() const = 0; /// Get name of database. String getDatabaseName() const { return database_name; } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 23c1570011f..266e932ca17 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1084,27 +1084,6 @@ DatabasePtr Context::detachDatabase(const String & database_name) } -ASTPtr Context::getCreateTableQuery(const String & database_name, const String & table_name) const -{ - auto lock = getLock(); - - String db = resolveDatabase(database_name, current_database); - assertDatabaseExists(db); - - return shared->databases[db]->getCreateTableQuery(*this, table_name); -} - - -ASTPtr Context::getCreateDictionaryQuery(const String & database_name, const String & dictionary_name) const -{ - auto lock = getLock(); - - String db = resolveDatabase(database_name, current_database); - assertDatabaseExists(db); - - return shared->databases[db]->getCreateDictionaryQuery(*this, dictionary_name); -} - ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const { TableAndCreateASTs::const_iterator jt = external_tables.find(table_name); @@ -1114,16 +1093,6 @@ ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const return jt->second.second; } -ASTPtr Context::getCreateDatabaseQuery(const String & database_name) const -{ - auto lock = getLock(); - - String db = resolveDatabase(database_name, current_database); - assertDatabaseExists(db); - - return shared->databases[db]->getCreateDatabaseQuery(*this); -} - Settings Context::getSettings() const { return settings; diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index fe3545e9730..e77b8995439 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -148,6 +148,7 @@ private: String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification. /// Thus, used in HTTP interface. If not specified - then some globally default format is used. + // TODO maybe replace with DatabaseMemory? TableAndCreateASTs external_tables; /// Temporary tables. Scalars scalars; StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views @@ -362,11 +363,7 @@ public: std::optional getTCPPortSecure() const; /// Get query for the CREATE table. - // TODO do we really need it here? - ASTPtr getCreateTableQuery(const String & database_name, const String & table_name) const; ASTPtr getCreateExternalTableQuery(const String & table_name) const; - ASTPtr getCreateDatabaseQuery(const String & database_name) const; - ASTPtr getCreateDictionaryQuery(const String & database_name, const String & dictionary_name) const; const DatabasePtr getDatabase(const String & database_name) const; DatabasePtr getDatabase(const String & database_name); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 80ec0f24402..d58aa1530d3 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -525,7 +525,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const String as_database_name = create.as_database.empty() ? context.getCurrentDatabase() : create.as_database; String as_table_name = create.as_table; - ASTPtr as_create_ptr = context.getCreateTableQuery(as_database_name, as_table_name); + ASTPtr as_create_ptr = context.getDatabase(as_database_name)->getCreateTableQuery(context, as_table_name); const auto & as_create = as_create_ptr->as(); if (as_create.is_view) @@ -569,7 +569,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.attach && !create.storage && !create.columns_list) { // Table SQL definition is available even if the table is detached - auto query = context.getCreateTableQuery(database_name, table_name); + auto query = context.getDatabase(database_name)->getCreateTableQuery(context, table_name); create = query->as(); // Copy the saved create query, but use ATTACH instead of CREATE create.attach = true; } @@ -744,7 +744,7 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create) if (create.attach) { - auto query = context.getCreateDictionaryQuery(database_name, dictionary_name); + auto query = context.getDatabase(database_name)->getCreateDictionaryQuery(context, dictionary_name); create = query->as(); create.attach = true; } diff --git a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp index f1c0d8937ed..01a15bcd7cd 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -51,19 +51,19 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (show_query->temporary) create_query = context.getCreateExternalTableQuery(show_query->table); else - create_query = context.getCreateTableQuery(show_query->database, show_query->table); + create_query = context.getDatabase(show_query->database)->getCreateTableQuery(context, show_query->table); } else if (show_query = query_ptr->as(); show_query) { if (show_query->temporary) throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR); - create_query = context.getCreateDatabaseQuery(show_query->database); + create_query = context.getDatabase(show_query->database)->getCreateDatabaseQuery(); } else if (show_query = query_ptr->as(); show_query) { if (show_query->temporary) throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR); - create_query = context.getCreateDictionaryQuery(show_query->database, show_query->table); + create_query = context.getDatabase(show_query->database)->getCreateDictionaryQuery(context, show_query->table); } if (!create_query && show_query->temporary) diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index c1e462f93b0..50e69460d65 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -274,7 +274,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); - create_ast = system_context.getCreateTableQuery(database_name, table_name); + create_ast = database->getCreateTableQuery(system_context, table_name); database->detachTable(table_name); } From 0b70bffe36e8b7d57a917f3d921c2cd527d2a7be Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Dec 2019 22:48:16 +0300 Subject: [PATCH 022/743] use StorageID in views --- .../PushingToViewsBlockOutputStream.cpp | 4 +- dbms/src/Databases/DatabaseOnDisk.cpp | 1 - dbms/src/Interpreters/Context.cpp | 34 ++-- dbms/src/Interpreters/Context.h | 9 +- .../Interpreters/InterpreterCreateQuery.cpp | 8 +- dbms/src/Parsers/ASTAlterQuery.h | 1 + dbms/src/Parsers/ASTCreateQuery.cpp | 13 +- dbms/src/Parsers/ASTCreateQuery.h | 5 +- dbms/src/Parsers/ASTQueryWithTableAndOutput.h | 1 + dbms/src/Parsers/ExpressionElementParsers.cpp | 36 +++- dbms/src/Parsers/ExpressionElementParsers.h | 6 + dbms/src/Parsers/ParserCreateQuery.cpp | 125 +++---------- dbms/src/Storages/Kafka/StorageKafka.cpp | 4 +- .../src/Storages/LiveView/StorageLiveView.cpp | 70 ++------ dbms/src/Storages/LiveView/StorageLiveView.h | 6 +- dbms/src/Storages/StorageID.h | 38 +++- dbms/src/Storages/StorageMaterializedView.cpp | 170 +++++++----------- dbms/src/Storages/StorageMaterializedView.h | 9 +- 18 files changed, 229 insertions(+), 311 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index c43f2e7bf1d..1e197d4d182 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -46,7 +46,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( for (const auto & database_table : dependencies) { - auto dependent_table = context.getTable(database_table.database_name, database_table.table_name); //FIXME + auto dependent_table = context.getTable(database_table); ASTPtr query; BlockOutputStreamPtr out; @@ -219,7 +219,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n /// InterpreterSelectQuery will do processing of alias columns. Context local_context = *views_context; local_context.addViewSource( - StorageValues::create(StorageID(storage->getDatabaseName(), storage->getTableName()), storage->getColumns(), //FIXME + StorageValues::create(storage->getStorageID(), storage->getColumns(), block)); select.emplace(view.query, local_context, SelectQueryOptions()); in = std::make_shared(select->execute().in); diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 700a7ea7b71..44c6dddb886 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -29,7 +29,6 @@ namespace DB { static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; -static constexpr char const * TABLE_WITH_UUID_NAME_PLACEHOLDER = "_"; namespace ErrorCodes { diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 266e932ca17..227a6a6fa1d 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -728,7 +728,6 @@ void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) c throw Exception("Access denied to database " + database_name + " for user " + client_info.current_user , ErrorCodes::DATABASE_ACCESS_DENIED); } -//FIXME use uuids if not empty void Context::addDependencyUnsafe(const StorageID & from, const StorageID & where) { @@ -737,7 +736,7 @@ void Context::addDependencyUnsafe(const StorageID & from, const StorageID & wher shared->view_dependencies[from].insert(where); // Notify table of dependencies change - auto table = tryGetTable(from.database_name, from.table_name); + auto table = tryGetTable(from); if (table != nullptr) table->updateDependencies(); } @@ -755,7 +754,7 @@ void Context::removeDependencyUnsafe(const StorageID & from, const StorageID & w shared->view_dependencies[from].erase(where); // Notify table of dependencies change - auto table = tryGetTable(from.database_name, from.table_name); + auto table = tryGetTable(from); if (table != nullptr) table->updateDependencies(); } @@ -910,24 +909,32 @@ StoragePtr Context::tryGetExternalTable(const String & table_name) const return jt->second.first; } - StoragePtr Context::getTable(const String & database_name, const String & table_name) const +{ + return getTable(StorageID(database_name, table_name)); +} + +StoragePtr Context::getTable(const StorageID & table_id) const { Exception exc; - auto res = getTableImpl(database_name, table_name, &exc); + auto res = getTableImpl(table_id, &exc); if (!res) throw exc; return res; } - StoragePtr Context::tryGetTable(const String & database_name, const String & table_name) const { - return getTableImpl(database_name, table_name, nullptr); + return tryGetTable(StorageID(database_name, table_name)); +} + +StoragePtr Context::tryGetTable(const StorageID & table_id) const +{ + return getTableImpl(table_id, nullptr); } -StoragePtr Context::getTableImpl(const String & database_name, const String & table_name, Exception * exception) const +StoragePtr Context::getTableImpl(const StorageID & table_id, Exception * exception) const { String db; DatabasePtr database; @@ -935,14 +942,15 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta { auto lock = getLock(); - if (database_name.empty()) + if (table_id.database_name.empty()) { - StoragePtr res = tryGetExternalTable(table_name); + StoragePtr res = tryGetExternalTable(table_id.table_name); if (res) return res; } - db = resolveDatabase(database_name, current_database); + //FIXME what if table was moved to another database? + db = resolveDatabase(table_id.database_name, current_database); checkDatabaseAccessRightsImpl(db); Databases::const_iterator it = shared->databases.find(db); @@ -956,11 +964,11 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta database = it->second; } - auto table = database->tryGetTable(*this, table_name); + auto table = database->tryGetTable(*this, table_id.table_name); if (!table) { if (exception) - *exception = Exception("Table " + backQuoteIfNeed(db) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + *exception = Exception("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); return {}; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index e77b8995439..7571843abfd 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -88,10 +88,6 @@ class CompiledExpressionCache; #endif -/// (database name, table name) -//FIXME replace with StorageID -//using DatabaseAndTableName = std::pair; - /// Table -> set of table-views that make SELECT from it. using ViewDependencies = std::map>; using Dependencies = std::vector; @@ -125,6 +121,7 @@ using IHostContextPtr = std::shared_ptr; * * Everything is encapsulated for all sorts of checks and locks. */ +///TODO remove syntax sugar and legacy methods from Context (e.g. getInputFormat(...) which just returns object from factory) class Context { private: @@ -276,7 +273,9 @@ public: Tables getExternalTables() const; StoragePtr tryGetExternalTable(const String & table_name) const; StoragePtr getTable(const String & database_name, const String & table_name) const; + StoragePtr getTable(const StorageID & table_id) const; StoragePtr tryGetTable(const String & database_name, const String & table_name) const; + StoragePtr tryGetTable(const StorageID & table_id) const; void addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast = {}); void addScalar(const String & name, const Block & block); bool hasScalar(const String & name) const; @@ -578,7 +577,7 @@ private: EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const; - StoragePtr getTableImpl(const String & database_name, const String & table_name, Exception * exception) const; + StoragePtr getTableImpl(const StorageID & table_id, Exception * exception) const; SessionKey getSessionKey(const String & session_id) const; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index d58aa1530d3..658f7ff80c1 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -548,8 +548,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.cluster.empty()) { NameSet databases{create.database}; - if (!create.to_table.empty()) - databases.emplace(create.to_database); + if (!create.to_table_id.empty()) + databases.emplace(create.to_table_id.database_name); /// NOTE: if it's CREATE query and create.database is DatabaseAtomic, different UUIDs will be generated on all servers. /// However, it allows to use UUID as replica name. @@ -577,8 +577,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String current_database = context.getCurrentDatabase(); if (create.database.empty()) create.database = current_database; - if (create.to_database.empty()) - create.to_database = current_database; + if (!create.to_table_id.empty() && create.to_table_id.database_name.empty()) + create.to_table_id.database_name = current_database; if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view)) { diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 162b9518824..e2b8eb20e35 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -147,6 +147,7 @@ public: String with_name; /// REPLACE(ATTACH) PARTITION partition FROM db.table + //FIXME use StorageID String from_database; String from_table; /// To distinguish REPLACE and ATTACH PARTITION partition FROM db.table diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index 32ac43129f8..62a44914f12 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -233,9 +233,11 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat << what << " " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") - << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table) - << (!uuid.empty() ? " UUID " + quoteString(uuid) : ""); - formatOnCluster(settings); + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + if (!uuid.empty()) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "") + << quoteString(uuid); + formatOnCluster(settings); } else { @@ -250,11 +252,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); as_table_function->formatImpl(settings, state, frame); } - if (!to_table.empty()) + if (!to_table_id.empty()) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "") - << (!to_database.empty() ? backQuoteIfNeed(to_database) + "." : "") << backQuoteIfNeed(to_table); + << (!to_table_id.database_name.empty() ? backQuoteIfNeed(to_table_id.database_name) + "." : "") + << backQuoteIfNeed(to_table_id.table_name); } if (!as_table.empty()) diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 9e88e51bdd6..87097e36882 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -65,15 +66,13 @@ public: ASTColumns * columns_list = nullptr; ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of dictionary ASTExpressionList * tables = nullptr; - String to_database; /// For CREATE MATERIALIZED VIEW mv TO table. - String to_table; + StorageID to_table_id; /// For CREATE MATERIALIZED VIEW mv TO table. ASTStorage * storage = nullptr; String as_database; String as_table; ASTPtr as_table_function; ASTSelectWithUnionQuery * select = nullptr; ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.) - String uuid; /// For ATTACH TABLE query when db engine is Atomic /** Get the text that identifies this element. */ String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + database) + delim + table; } diff --git a/dbms/src/Parsers/ASTQueryWithTableAndOutput.h b/dbms/src/Parsers/ASTQueryWithTableAndOutput.h index 594876ace7b..5068db1edb7 100644 --- a/dbms/src/Parsers/ASTQueryWithTableAndOutput.h +++ b/dbms/src/Parsers/ASTQueryWithTableAndOutput.h @@ -15,6 +15,7 @@ class ASTQueryWithTableAndOutput : public ASTQueryWithOutput public: String database; String table; + String uuid; bool temporary{false}; protected: diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index c26f9363797..5c6f8ca8593 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -29,7 +29,7 @@ #include #include -#include +#include #include "ASTColumnsMatcher.h" @@ -197,6 +197,40 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex } +bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected) +{ + ParserKeyword s_uuid("UUID"); + ParserIdentifier name_p; + ParserStringLiteral uuid_p; + ParserToken s_dot(TokenType::Dot); + + ASTPtr database; + ASTPtr table; + ASTPtr uuid; + + 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 (s_uuid.ignore(pos, expected)) + { + if (!uuid_p.parse(pos, uuid, expected)) + return false; + } + + tryGetIdentifierNameInto(database, res.database_name); + tryGetIdentifierNameInto(table, res.table_name); + res.uuid = uuid ? uuid->as()->value.get() : ""; + return true; +} + + bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index 63ed1348b13..35e8948938d 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -56,6 +56,12 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); }; +struct StorageID; +/// Table name, possibly with database name and UUID as string literal +/// [db_name.]table_name [UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx'] +//TODO replace with class +bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected); + /// Just * class ParserAsterisk : public IParserBase { diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index aa0af4ea739..6a61728a7a6 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -340,7 +341,6 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_table("TABLE"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserKeyword s_uuid("UUID"); ParserKeyword s_on("ON"); ParserKeyword s_as("AS"); ParserToken s_dot(TokenType::Dot); @@ -352,14 +352,9 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserSelectWithUnionQuery select_p; ParserFunction table_function_p; ParserNameList names_p; - ParserStringLiteral uuid_p; - ASTPtr database; - ASTPtr table; - ASTPtr uuid; + StorageID table_id; ASTPtr columns_list; - ASTPtr to_database; - ASTPtr to_table; ASTPtr storage; ASTPtr as_database; ASTPtr as_table; @@ -389,22 +384,9 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (s_if_not_exists.ignore(pos, expected)) if_not_exists = true; - if (!name_p.parse(pos, table, expected)) + if (!parseStorageID(pos, table_id, expected)) return false; - if (s_dot.ignore(pos, expected)) - { - database = table; - if (!name_p.parse(pos, table, expected)) - return false; - } - - if (attach && s_uuid.ignore(pos, expected)) - { - if (!uuid_p.parse(pos, uuid, expected)) - return false; - } - if (s_on.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) @@ -421,8 +403,9 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->if_not_exists = if_not_exists; query->cluster = cluster_str; - tryGetIdentifierNameInto(database, query->database); - tryGetIdentifierNameInto(table, query->table); + query->database = table_id.database_name; + query->table = table_id.table_name; + query->uuid = table_id.uuid; return true; } @@ -479,15 +462,11 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->if_not_exists = if_not_exists; query->temporary = is_temporary; - tryGetIdentifierNameInto(database, query->database); - tryGetIdentifierNameInto(table, query->table); - if (uuid) - query->uuid = uuid->as()->value.get(); + query->database = table_id.database_name; + query->table = table_id.table_name; + query->uuid = table_id.uuid; query->cluster = cluster_str; - tryGetIdentifierNameInto(to_database, query->to_database); - tryGetIdentifierNameInto(to_table, query->to_table); - query->set(query->columns_list, columns_list); query->set(query->storage, storage); @@ -504,7 +483,6 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserKeyword s_uuid("UUID"); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_live("LIVE"); @@ -515,14 +493,10 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserIdentifier name_p; ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; - ParserStringLiteral uuid_p; - ASTPtr database; - ASTPtr table; - ASTPtr uuid; + StorageID table_id; + StorageID to_table_id; ASTPtr columns_list; - ASTPtr to_database; - ASTPtr to_table; ASTPtr storage; ASTPtr as_database; ASTPtr as_table; @@ -555,22 +529,9 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (s_if_not_exists.ignore(pos, expected)) if_not_exists = true; - if (!name_p.parse(pos, table, expected)) + if (!parseStorageID(pos, table_id, expected)) return false; - if (s_dot.ignore(pos, expected)) - { - database = table; - if (!name_p.parse(pos, table, expected)) - return false; - } - - if (attach && s_uuid.ignore(pos, expected)) - { - if (!uuid_p.parse(pos, uuid, expected)) - return false; - } - if (ParserKeyword{"ON"}.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) @@ -580,15 +541,8 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e // TO [db.]table if (ParserKeyword{"TO"}.ignore(pos, expected)) { - if (!name_p.parse(pos, to_table, expected)) + if (!parseStorageID(pos, to_table_id, 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. @@ -617,14 +571,12 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->is_live_view = true; query->temporary = is_temporary; - tryGetIdentifierNameInto(database, query->database); - tryGetIdentifierNameInto(table, query->table); + query->database = table_id.database_name; + query->table = table_id.table_name; + query->uuid = table_id.uuid; query->cluster = cluster_str; - tryGetIdentifierNameInto(to_database, query->to_database); - tryGetIdentifierNameInto(to_table, query->to_table); - if (uuid) - query->uuid = uuid->as()->value.get(); + query->to_table_id = to_table_id; query->set(query->columns_list, columns_list); @@ -713,12 +665,9 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserNameList names_p; ParserStringLiteral uuid_p; - ASTPtr database; - ASTPtr table; - ASTPtr uuid; + StorageID table_id; + StorageID to_table_id; ASTPtr columns_list; - ASTPtr to_database; - ASTPtr to_table; ASTPtr storage; ASTPtr as_database; ASTPtr as_table; @@ -759,22 +708,9 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!replace_view && s_if_not_exists.ignore(pos, expected)) if_not_exists = true; - if (!name_p.parse(pos, table, expected)) + if (!parseStorageID(pos, table_id, expected)) return false; - if (s_dot.ignore(pos, expected)) - { - database = table; - if (!name_p.parse(pos, table, expected)) - return false; - } - - if (attach && s_uuid.ignore(pos, expected)) - { - if (!uuid_p.parse(pos, uuid, expected)) - return false; - } - if (ParserKeyword{"ON"}.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) @@ -784,15 +720,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec // TO [db.]table if (ParserKeyword{"TO"}.ignore(pos, expected)) { - if (!name_p.parse(pos, to_table, expected)) + if (!parseStorageID(pos, to_table_id, 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. @@ -805,7 +734,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } - if (is_materialized_view && !to_table) + if (is_materialized_view && to_table_id.empty()) { /// Internal ENGINE for MATERIALIZED VIEW must be specified. if (!storage_p.parse(pos, storage, expected)) @@ -833,14 +762,12 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->is_populate = is_populate; query->replace_view = replace_view; - tryGetIdentifierNameInto(database, query->database); - tryGetIdentifierNameInto(table, query->table); - if (uuid) - query->uuid = uuid->as()->value.get(); + query->database = table_id.database_name; + query->table = table_id.table_name; + query->uuid = table_id.uuid; query->cluster = cluster_str; - tryGetIdentifierNameInto(to_database, query->to_database); - tryGetIdentifierNameInto(to_table, query->to_table); + query->to_table_id = to_table_id; query->set(query->columns_list, columns_list); query->set(query->storage, storage); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index a51f26fa1aa..d859bc3c7e6 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -299,7 +299,7 @@ bool StorageKafka::checkDependencies(const StorageID & table_id) // Check the dependencies are ready? for (const auto & db_tab : dependencies) { - auto table = global_context.tryGetTable(db_tab.database_name, db_tab.table_name); //FIXME + auto table = global_context.tryGetTable(db_tab); if (!table) return false; @@ -351,7 +351,7 @@ void StorageKafka::threadFunc() bool StorageKafka::streamToViews() { auto table_id = getStorageID(); - auto table = global_context.getTable(table_id.database_name, table_id.table_name); + auto table = global_context.getTable(table_id); if (!table) throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 987996f9cc7..fe1fb21cc4b 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -14,7 +14,6 @@ limitations under the License. */ #include #include #include -#include #include #include #include @@ -33,10 +32,9 @@ limitations under the License. */ #include #include #include +#include #include -#include -#include #include #include @@ -52,42 +50,6 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } -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 LIVE VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_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 StorageLiveView." - " Could not retrieve table name from select query.", - DB::ErrorCodes::LOGICAL_ERROR); -} - void StorageLiveView::writeIntoLiveView( StorageLiveView & live_view, @@ -145,7 +107,7 @@ void StorageLiveView::writeIntoLiveView( if (!is_block_processed) { - auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); + auto parent_storage = context.getTable(live_view.getSelectTableID()); BlockInputStreams streams = {std::make_shared(block)}; auto proxy_storage = std::make_shared(parent_storage, std::move(streams), QueryProcessingStage::FetchColumns); InterpreterSelectQuery select_block(live_view.getInnerQuery(), @@ -177,7 +139,7 @@ void StorageLiveView::writeIntoLiveView( } } - auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); + auto parent_storage = context.getTable(live_view.getSelectTableID()); auto proxy_storage = std::make_shared(parent_storage, std::move(from), QueryProcessingStage::WithMergeableState); InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete); BlockInputStreamPtr data = std::make_shared(select.execute().in); @@ -205,25 +167,19 @@ StorageLiveView::StorageLiveView( 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 LIVE VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); 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); + select_table_id = extractDependentTableFromSelectQuery(select_query, global_context, true); /// If the table is not specified - use the table `system.one` - if (select_table_name.empty()) - { - select_database_name = "system"; - select_table_name = "one"; - } + //FIXME why? + if (select_table_id.empty()) + select_table_id = StorageID("system", "one"); - global_context.addDependency( - StorageID(select_database_name, select_table_name), - table_id_); //FIXME + global_context.addDependency(select_table_id, table_id_); is_temporary = query.temporary; temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds(); @@ -255,7 +211,7 @@ Block StorageLiveView::getHeader() const if (!sample_block) { - auto storage = global_context.getTable(select_database_name, select_table_name); + auto storage = global_context.getTable(select_table_id); sample_block = InterpreterSelectQuery(inner_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::Complete)).getSampleBlock(); sample_block.insert({DataTypeUInt64().createColumnConst( @@ -290,7 +246,7 @@ bool StorageLiveView::getNewBlocks() mergeable_blocks = std::make_shared>(); mergeable_blocks->push_back(new_mergeable_blocks); BlockInputStreamPtr from = std::make_shared(std::make_shared(new_mergeable_blocks), mergeable_stream->getHeader()); - auto proxy_storage = ProxyStorage::createProxyStorage(global_context.getTable(select_database_name, select_table_name), {from}, QueryProcessingStage::WithMergeableState); + auto proxy_storage = ProxyStorage::createProxyStorage(global_context.getTable(select_table_id), {from}, QueryProcessingStage::WithMergeableState); InterpreterSelectQuery select(inner_query->clone(), global_context, proxy_storage, SelectQueryOptions(QueryProcessingStage::Complete)); BlockInputStreamPtr data = std::make_shared(select.execute().in); @@ -375,7 +331,7 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co if (drop_table) { - if (storage->global_context.tryGetTable(table_id.database_name, table_id.table_name)) //FIXME + if (storage->global_context.tryGetTable(table_id)) { try { @@ -468,9 +424,7 @@ StorageLiveView::~StorageLiveView() void StorageLiveView::drop(TableStructureWriteLockHolder &) { auto table_id = getStorageID(); - global_context.removeDependency( - StorageID(select_database_name, select_table_name), - table_id); //FIXME + global_context.removeDependency(select_table_id, table_id); std::lock_guard lock(mutex); is_dropped = true; diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 63a923df2cc..9c495c4b1d4 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -41,8 +41,7 @@ friend class LiveViewBlockOutputStream; public: ~StorageLiveView() override; String getName() const override { return "LiveView"; } - String getSelectDatabaseName() const { return select_database_name; } - String getSelectTableName() const { return select_table_name; } + StorageID getSelectTableID() const { return select_table_id; } NameAndTypePair getColumn(const String & column_name) const override; bool hasColumn(const String & column_name) const override; @@ -140,8 +139,7 @@ public: const Context & context); private: - String select_database_name; - String select_table_name; + StorageID select_table_id; ASTPtr inner_query; Context & global_context; bool is_temporary = false; diff --git a/dbms/src/Storages/StorageID.h b/dbms/src/Storages/StorageID.h index cdce84729ff..23ff37e0e11 100644 --- a/dbms/src/Storages/StorageID.h +++ b/dbms/src/Storages/StorageID.h @@ -11,28 +11,36 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static constexpr char const * TABLE_WITH_UUID_NAME_PLACEHOLDER = "_"; + struct StorageID { String database_name; String table_name; String uuid; - StorageID() = delete; + //StorageID() = delete; + StorageID() = default; + + //TODO StorageID(const ASTPtr & query_with_one_table, const Context & context) to get db and table names (and maybe uuid) from query + //But there are a lot of different ASTs with db and table name + //And it looks like it depends on https://github.com/ClickHouse/ClickHouse/pull/7774 StorageID(const String & database, const String & table, const String & uuid_ = {}) : database_name(database), table_name(table), uuid(uuid_) { - assert_not_empty(); } String getFullTableName() const { + assert_valid(); return (database_name.empty() ? "" : database_name + ".") + table_name; } String getNameForLogs() const { - return (database_name.empty() ? "" : backQuoteIfNeed(database_name) + ".") + backQuoteIfNeed(table_name) + " (UUID " + uuid + ")"; + assert_valid(); + return (database_name.empty() ? "" : backQuoteIfNeed(database_name) + ".") + backQuoteIfNeed(table_name) + (uuid.empty() ? "" : " (UUID " + uuid + ")"); } String getId() const @@ -45,13 +53,31 @@ struct StorageID bool operator<(const StorageID & rhs) const { - return std::tie(uuid, database_name, table_name) < std::tie(rhs.uuid, rhs.database_name, rhs.table_name); + assert_valid(); + /// It's needed for ViewDependencies + if (uuid.empty() && rhs.uuid.empty()) + /// If both IDs don't have UUID, compare them like pair of strings + return std::tie(database_name, table_name) < std::tie(rhs.database_name, rhs.table_name); + else if (!uuid.empty() && !rhs.uuid.empty()) + /// If both IDs have UUID, compare UUIDs and ignore database and table name + return uuid < rhs.uuid; + else + /// All IDs without UUID are less, then all IDs with UUID + return uuid.empty(); } - void assert_not_empty() const + bool empty() const { - if (database_name.empty() && table_name.empty()) + return table_name.empty() || (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && uuid.empty()); + } + + void assert_valid() const + { + if (empty()) throw Exception("empty table name", ErrorCodes::LOGICAL_ERROR); + if (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && uuid.empty() && !database_name.empty()) + throw Exception("unexpected database name", ErrorCodes::LOGICAL_ERROR); + } }; diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 9ba30a7d108..b2e4df4cf7d 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int INCORRECT_QUERY; extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW; + extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW; } static inline String generateInnerTableName(const String & table_name) @@ -34,39 +35,37 @@ static inline String generateInnerTableName(const String & table_name) return ".inner." + table_name; } -static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) +StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, Context & context, bool is_live_view /*= false*/, bool need_visitor /*= true*/) { + if (need_visitor) + { + AddDefaultDatabaseVisitor visitor(context.getCurrentDatabase(), nullptr); + visitor.visit(query); + } auto db_and_table = getDatabaseAndTable(query, 0); ASTPtr subquery = extractTableExpression(query, 0); if (!db_and_table && !subquery) - return; + return {}; //FIXME in which cases we cannot get table name? 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; + //TODO uuid + return StorageID(db_and_table->database, db_and_table->table/*, db_and_table->uuid*/); } else if (auto * ast_select = subquery->as()) { if (ast_select->list_of_selects->children.size() != 1) - throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW); + throw Exception(String("UNION is not supported for ") + (is_live_view ? "LIVE VIEW" : "MATERIALIZED VIEW"), + is_live_view ? ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW : ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW); auto & inner_query = ast_select->list_of_selects->children.at(0); - extractDependentTable(inner_query->as(), select_database_name, select_table_name); + return extractDependentTableFromSelectQuery(inner_query->as(), context, is_live_view, false); } else - throw Exception("Logical error while creating StorageMaterializedView." - " Could not retrieve table name from select query.", + throw Exception(String("Logical error while creating Storage") + (is_live_view ? "Live" : "Materialized") + + "View. Could not retrieve table name from select query.", DB::ErrorCodes::LOGICAL_ERROR); } @@ -105,47 +104,36 @@ StorageMaterializedView::StorageMaterializedView( if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); - if (!query.storage && query.to_table.empty()) + /// If the destination table is not set, use inner table + has_inner_table = query.to_table_id.empty(); + if (has_inner_table && !query.storage) throw Exception( "You must specify where to save results of a MaterializedView query: either ENGINE or an existing table in a TO clause", 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 MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW); inner_query = query.select->list_of_selects->children.at(0); auto & select_query = inner_query->as(); - extractDependentTable(select_query, select_database_name, select_table_name); + select_table_id = extractDependentTableFromSelectQuery(select_query, local_context); checkAllowedQueries(select_query); - if (!select_table_name.empty()) - global_context.addDependency( - StorageID(select_database_name, select_table_name), - table_id_); //FIXME - - // If the destination table is not set, use inner table - if (!query.to_table.empty()) + if (!has_inner_table) + target_table_id = query.to_table_id; + else if (attach_) { - target_database_name = query.to_database; - target_table_name = query.to_table; + /// If there is an ATTACH request, then the internal table must already be created. + //TODO use uuid + target_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name)); } else - { - target_database_name = table_id_.database_name; - target_table_name = generateInnerTableName(table_id_.table_name); - has_inner_table = true; - } - - /// If there is an ATTACH request, then the internal table must already be connected. - if (!attach_ && has_inner_table) { /// We will create a query to create an internal table. auto manual_create_query = std::make_shared(); - manual_create_query->database = target_database_name; - manual_create_query->table = target_table_name; + manual_create_query->database = table_id_.database_name; + manual_create_query->table = generateInnerTableName(table_id_.table_name); auto new_columns_list = std::make_shared(); new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr()); @@ -153,24 +141,15 @@ StorageMaterializedView::StorageMaterializedView( manual_create_query->set(manual_create_query->columns_list, new_columns_list); manual_create_query->set(manual_create_query->storage, query.storage->ptr()); - /// Execute the query. - try - { - InterpreterCreateQuery create_interpreter(manual_create_query, local_context); - create_interpreter.setInternal(true); - create_interpreter.execute(); - } - catch (...) - { - /// In case of any error we should remove dependency to the view. - if (!select_table_name.empty()) - global_context.removeDependency( - StorageID(select_database_name, select_table_name), - table_id_); //FIXME + InterpreterCreateQuery create_interpreter(manual_create_query, local_context); + create_interpreter.setInternal(true); + create_interpreter.execute(); - throw; - } + target_table_id = global_context.getTable(manual_create_query->database, manual_create_query->table)->getStorageID(); } + + if (!select_table_id.empty()) + global_context.addDependency(select_table_id, table_id_); } NameAndTypePair StorageMaterializedView::getColumn(const String & column_name) const @@ -214,14 +193,14 @@ BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const } -static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, const String & target_database_name, const String & target_table_name) +static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, const StorageID & target_table_id) { - if (global_context.tryGetTable(target_database_name, target_table_name)) + 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_database_name; - drop_query->table = target_table_name; + 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); @@ -233,25 +212,23 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, void StorageMaterializedView::drop(TableStructureWriteLockHolder &) { auto table_id = getStorageID(); - global_context.removeDependency( - StorageID(select_database_name, select_table_name), - table_id); //FIXME + global_context.removeDependency(select_table_id, table_id); if (has_inner_table && tryGetTargetTable()) - executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_database_name, target_table_name); + executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_table_id); } void StorageMaterializedView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { if (has_inner_table) - executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, target_database_name, target_table_name); + executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, target_table_id); } void StorageMaterializedView::checkStatementCanBeForwarded() const { if (!has_inner_table) throw Exception( - "MATERIALIZED VIEW targets existing table " + target_database_name + "." + target_table_name + ". " + "MATERIALIZED VIEW targets existing table " + target_table_id.getNameForLogs() + ". " + "Execute the statement directly on it.", ErrorCodes::INCORRECT_QUERY); } @@ -273,74 +250,59 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co getTargetTable()->mutate(commands, context); } -static void executeRenameQuery(Context & global_context, const String & database_name, const String & table_original_name, const String & new_table_name) -{ - if (global_context.tryGetTable(database_name, table_original_name)) - { - auto rename = std::make_shared(); - - ASTRenameQuery::Table from; - from.database = database_name; - from.table = table_original_name; - - ASTRenameQuery::Table to; - to.database = database_name; - to.table = new_table_name; - - ASTRenameQuery::Element elem; - elem.from = from; - elem.to = to; - - rename->elements.emplace_back(elem); - - InterpreterRenameQuery(rename, global_context).execute(); - } -} - - void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name, std::unique_lock *) { //FIXME if (has_inner_table && tryGetTargetTable()) { - String new_target_table_name = generateInnerTableName(new_table_name); - executeRenameQuery(global_context, target_database_name, target_table_name, new_target_table_name); - target_table_name = new_target_table_name; + auto new_target_table_name = generateInnerTableName(new_table_name); + auto rename = std::make_shared(); + + ASTRenameQuery::Table from; + from.database = target_table_id.database_name; + from.table = target_table_id.table_name; + + ASTRenameQuery::Table to; + to.database = target_table_id.database_name; + to.table = new_target_table_name; + + ASTRenameQuery::Element elem; + elem.from = from; + elem.to = to; + rename->elements.emplace_back(elem); + + InterpreterRenameQuery(rename, global_context).execute(); + target_table_id.table_name = new_target_table_name; } auto lock = global_context.getLock(); std::unique_lock name_lock; auto table_id = getStorageID(&name_lock); - global_context.removeDependencyUnsafe( - StorageID(select_database_name, select_table_name), - table_id); + global_context.removeDependencyUnsafe(select_table_id, table_id); IStorage::renameInMemory(new_database_name, new_table_name, &name_lock); - global_context.addDependencyUnsafe( - StorageID(select_database_name, select_table_name), - table_id); + auto new_table_id = getStorageID(&name_lock); + global_context.addDependencyUnsafe(select_table_id, new_table_id); } void StorageMaterializedView::shutdown() { auto table_id = getStorageID(); /// Make sure the dependency is removed after DETACH TABLE - global_context.removeDependency( - StorageID(select_database_name, select_table_name), - table_id); //FIXME + global_context.removeDependency(select_table_id, table_id); } StoragePtr StorageMaterializedView::getTargetTable() const { - return global_context.getTable(target_database_name, target_table_name); + return global_context.getTable(target_table_id); } StoragePtr StorageMaterializedView::tryGetTargetTable() const { - return global_context.tryGetTable(target_database_name, target_table_name); + return global_context.tryGetTable(target_table_id); } Strings StorageMaterializedView::getDataPaths() const diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index a86f49cd0f6..19055d41011 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -9,6 +9,9 @@ namespace DB { +StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, Context & context, bool is_live_view = false, bool need_visitor = true); + + class StorageMaterializedView : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; @@ -66,10 +69,8 @@ public: Strings getDataPaths() const override; private: - String select_database_name; - String select_table_name; - String target_database_name; - String target_table_name; + StorageID select_table_id; + StorageID target_table_id; ASTPtr inner_query; Context & global_context; bool has_inner_table = false; From 3a4ee4a07c62bac4d36c4684f9cd4c5b31d0b289 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Dec 2019 23:47:05 +0300 Subject: [PATCH 023/743] remove separate getTableName() and getDatabaseName() --- dbms/src/Core/iostream_debug_helpers.cpp | 3 ++- .../PushingToViewsBlockOutputStream.cpp | 5 +++-- dbms/src/Interpreters/InterpreterDropQuery.cpp | 7 ++++--- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 4 +++- dbms/src/Interpreters/PartLog.cpp | 7 ++++--- .../Distributed/DistributedBlockOutputStream.cpp | 4 ++-- dbms/src/Storages/IStorage.cpp | 13 ++++++------- dbms/src/Storages/IStorage.h | 12 ------------ dbms/src/Storages/LiveView/ProxyStorage.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 4 +--- .../MergeTree/StorageFromMergeTreeDataPart.h | 12 +++++++----- dbms/src/Storages/StorageBuffer.cpp | 5 ++--- dbms/src/Storages/StorageDistributed.cpp | 5 ++--- dbms/src/Storages/StorageMerge.cpp | 15 ++++++++++----- dbms/src/Storages/StorageMergeTree.cpp | 7 +++---- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 11 +++++------ .../src/Storages/System/StorageSystemGraphite.cpp | 10 ++++++---- 17 files changed, 61 insertions(+), 65 deletions(-) diff --git a/dbms/src/Core/iostream_debug_helpers.cpp b/dbms/src/Core/iostream_debug_helpers.cpp index eea8694dfb0..07a478961a1 100644 --- a/dbms/src/Core/iostream_debug_helpers.cpp +++ b/dbms/src/Core/iostream_debug_helpers.cpp @@ -47,7 +47,8 @@ std::ostream & operator<<(std::ostream & stream, const IDataType & what) std::ostream & operator<<(std::ostream & stream, const IStorage & what) { - stream << "IStorage(name = " << what.getName() << ", tableName = " << what.getTableName() << ") {" + auto table_id = what.getStorageID(); + stream << "IStorage(name = " << what.getName() << ", tableName = " << table_id.table_name << ") {" << what.getColumns().getAllPhysical().toString() << "}"; return stream; } diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 1e197d4d182..971c7a28d49 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -54,10 +54,11 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( if (auto * materialized_view = dynamic_cast(dependent_table.get())) { StoragePtr inner_table = materialized_view->getTargetTable(); + auto inner_table_id = inner_table->getStorageID(); query = materialized_view->getInnerQuery(); std::unique_ptr insert = std::make_unique(); - insert->database = inner_table->getDatabaseName(); - insert->table = inner_table->getTableName(); + insert->database = inner_table_id.database_name; + insert->table = inner_table_id.table_name; //FIXME add uuid ASTPtr insert_query_ptr(insert.release()); InterpreterInsertQuery interpreter(insert_query_ptr, *views_context); BlockIO io = interpreter.execute(); diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 2f7240b5ddc..b9bb2aef4dc 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -79,13 +79,14 @@ BlockIO InterpreterDropQuery::executeToTable( if (database_and_table.first && database_and_table.second) { + auto table_id = database_and_table.second->getStorageID(); if (kind == ASTDropQuery::Kind::Detach) { database_and_table.second->shutdown(); /// If table was already dropped by anyone, an exception will be thrown auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); /// Drop table from memory, don't touch data and metadata - database_and_table.first->detachTable(database_and_table.second->getTableName()); + database_and_table.first->detachTable(table_id.table_name); } else if (kind == ASTDropQuery::Kind::Truncate) { @@ -107,7 +108,7 @@ BlockIO InterpreterDropQuery::executeToTable( const std::string metadata_file_without_extension = database_and_table.first->getMetadataPath() - + escapeForFileName(database_and_table.second->getTableName()); + + escapeForFileName(table_id.table_name); const auto prev_metadata_name = metadata_file_without_extension + ".sql"; const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop"; @@ -132,7 +133,7 @@ BlockIO InterpreterDropQuery::executeToTable( String table_data_path = database_and_table.first->getDataPath(table_name); /// Delete table metadata and table itself from memory - database_and_table.first->removeTable(context, database_and_table.second->getTableName()); + database_and_table.first->removeTable(context, table_id.table_name); database_and_table.second->is_dropped = true; /// If it is not virtual database like Dictionary then drop remaining data dir diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index efc4541c162..5216aaeab34 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -295,7 +295,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (auto view_source = context->getViewSource()) { auto & storage_values = static_cast(*view_source); - if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name) + auto tmp_table_id = storage_values.getStorageID(); + //FIXME maybe add uuid? + if (tmp_table_id.database_name == database_name && tmp_table_id.table_name == table_name) { /// Read from view source. storage = context->getViewSource(); diff --git a/dbms/src/Interpreters/PartLog.cpp b/dbms/src/Interpreters/PartLog.cpp index c860f3212c7..66315dc4eec 100644 --- a/dbms/src/Interpreters/PartLog.cpp +++ b/dbms/src/Interpreters/PartLog.cpp @@ -112,7 +112,8 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP try { - part_log = current_context.getPartLog(parts.front()->storage.getDatabaseName()); // assume parts belong to the same table + auto table_id = parts.front()->storage.getStorageID(); + part_log = current_context.getPartLog(table_id.database_name); // assume parts belong to the same table if (!part_log) return false; @@ -124,8 +125,8 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP elem.event_time = time(nullptr); elem.duration_ms = elapsed_ns / 1000000; - elem.database_name = part->storage.getDatabaseName(); - elem.table_name = part->storage.getTableName(); + elem.database_name = table_id.database_name; + elem.table_name = table_id.table_name; elem.partition_id = part->info.partition_id; elem.part_name = part->name; elem.path_on_disk = part->getFullPath(); diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 61cb10cc38e..1fd5d1c5311 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -90,7 +90,7 @@ void DistributedBlockOutputStream::write(const Block & block) if (ordinary_block.has(col.name)) { ordinary_block.erase(col.name); - LOG_DEBUG(log, storage.getTableName() + LOG_DEBUG(log, storage.getStorageID().getNameForLogs() << ": column " + col.name + " will be removed, " << "because it is MATERIALIZED"); } @@ -515,7 +515,7 @@ void DistributedBlockOutputStream::writeAsyncImpl(const Block & block, const siz else { if (shard_info.dir_name_for_internal_replication.empty()) - throw Exception("Directory name for async inserts is empty, table " + storage.getTableName(), ErrorCodes::LOGICAL_ERROR); + throw Exception("Directory name for async inserts is empty, table " + storage.getStorageID().getNameForLogs(), ErrorCodes::LOGICAL_ERROR); writeToShard(block, {shard_info.dir_name_for_internal_replication}); } diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 4ab4e149cdd..a3f97337835 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -176,7 +176,7 @@ void IStorage::check(const Names & column_names, bool include_virtuals) const { if (columns_map.end() == columns_map.find(name)) throw Exception( - "There is no column with name " + backQuote(name) + " in table " + getTableName() + ". There are columns: " + list_of_columns, + "There is no column with name " + backQuote(name) + " in table " + getStorageID().getNameForLogs() + ". There are columns: " + list_of_columns, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); if (unique_names.end() != unique_names.find(name)) @@ -344,7 +344,7 @@ TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_ void IStorage::lockNewDataStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id) { if (!lock_holder.alter_intention_lock) - throw Exception("Alter intention lock for table " + getTableName() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id); } @@ -352,7 +352,7 @@ void IStorage::lockNewDataStructureExclusively(TableStructureWriteLockHolder & l void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id) { if (!lock_holder.alter_intention_lock) - throw Exception("Alter intention lock for table " + getTableName() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); if (!lock_holder.new_data_structure_lock) lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id); @@ -405,15 +405,14 @@ void IStorage::alter( if (params.isMutable()) throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); - const String database_name = getDatabaseName(); - const String table_name = getTableName(); + auto table_id = getStorageID(); if (params.isSettingsAlter()) { SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); //FIXME } else { @@ -422,7 +421,7 @@ void IStorage::alter( auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME setColumns(std::move(new_columns)); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index f069d50f8b3..ec84547061e 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -90,18 +90,6 @@ public: /// The name of the table. StorageID getStorageID(std::unique_lock * lock = nullptr) const; - // FIXME remove those methods - std::string getTableName() const - { - std::lock_guard lock(id_mutex); - return id.table_name; - } - std::string getDatabaseName() const - { - std::lock_guard lock(id_mutex); - return id.database_name; - } - /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } diff --git a/dbms/src/Storages/LiveView/ProxyStorage.h b/dbms/src/Storages/LiveView/ProxyStorage.h index df8aade6540..ea5d32f4264 100644 --- a/dbms/src/Storages/LiveView/ProxyStorage.h +++ b/dbms/src/Storages/LiveView/ProxyStorage.h @@ -9,7 +9,7 @@ class ProxyStorage : public IStorage { public: ProxyStorage(StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) - : IStorage({"", storage_->getTableName()}), storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {} + : IStorage({"", storage_->getStorageID().table_name}), storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {} public: std::string getName() const override { return "ProxyStorage(" + storage->getName() + ")"; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 4072ca706ed..d9db7755a05 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -3355,9 +3355,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & String part_absolute_path = Poco::Path(part->getFullPath()).absolute().toString(); String backup_part_absolute_path = backup_path - + "data/" - + escapeForFileName(getDatabaseName()) + "/" - + escapeForFileName(getTableName()) + "/" + + relative_data_path + part->relative_path; localBackup(part_absolute_path, backup_part_absolute_path); part->is_frozen.store(true, std::memory_order_relaxed); diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index e04749e86cc..1326c61b1f6 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -49,11 +49,7 @@ public: protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) - : IStorage({ - part_->storage.getDatabaseName(), - part_->storage.getTableName() + " (part " + part_->name + ")" - } - , part_->storage.getVirtuals()) + : IStorage(getIDFromPart(part_), part_->storage.getVirtuals()) , part(part_) { setColumns(part_->storage.getColumns()); @@ -62,6 +58,12 @@ protected: private: MergeTreeData::DataPartPtr part; + + static StorageID getIDFromPart(const MergeTreeData::DataPartPtr & part_) + { + auto table_id = part_->storage.getStorageID(); + return StorageID(table_id.database_name, table_id.table_name + " (part " + part_->name + ")"); + } }; } diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index cbdd4236071..9bff477b762 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -710,8 +710,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - const String database_name_ = getDatabaseName(); - const String table_name_ = getTableName(); + auto table_id = getStorageID(); /// So that no blocks of the old structure remain. optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context); @@ -720,7 +719,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, new_constraints, {}); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index a47afda5ecf..4af368cbd43 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -402,14 +402,13 @@ void StorageDistributed::alter( { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); + auto table_id = getStorageID(); auto new_columns = getColumns(); auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {}); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 8c955e27ace..427af8c6bd0 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -263,7 +263,10 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer SelectQueryInfo modified_query_info = query_info; modified_query_info.query = query_info.query->clone(); - VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", storage ? storage->getTableName() : ""); + StorageID table_id; + if (storage) + table_id = storage->getStorageID(); + VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_id.table_name); if (!storage) return BlockInputStreams{ @@ -283,7 +286,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer } else if (processed_stage > storage->getQueryProcessingStage(modified_context)) { - modified_query_info.query->as()->replaceDatabaseAndTable(source_database, storage->getTableName()); + modified_query_info.query->as()->replaceDatabaseAndTable(source_database, table_id.table_name); /// Maximum permissible parallelism is streams_num modified_context.getSettingsRef().max_threads = UInt64(streams_num); @@ -314,7 +317,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer { if (has_table_virtual_column) source_stream = std::make_shared>( - source_stream, std::make_shared(), storage->getTableName(), "_table"); + source_stream, std::make_shared(), table_id.table_name, "_table"); /// Subordinary tables could have different but convertible types, like numeric types of different width. /// We must return streams with structure equals to structure of Merge table. @@ -363,7 +366,8 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr if (storage.get() != this) { selected_tables.emplace_back(storage, get_lock ? storage->lockStructureForShare(false, query_id) : TableStructureReadLockHolder{}); - virtual_column->insert(storage->getTableName()); + auto table_id = storage->getStorageID(); + virtual_column->insert(table_id.table_name); } iterator->next(); @@ -376,7 +380,8 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr auto values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_table"); /// Remove unused tables from the list - selected_tables.remove_if([&] (const auto & elem) { return values.find(elem.first->getTableName()) == values.end(); }); + //FIXME table name can be changed, use StorageID + selected_tables.remove_if([&] (const auto & elem) { return values.find(elem.first->getStorageID().table_name) == values.end(); }); } return selected_tables; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 14189f6ef6f..c9f685e75e2 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -260,8 +260,7 @@ void StorageMergeTree::alter( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); + auto table_id = getStorageID(); if (!params.isMutable()) { @@ -279,7 +278,7 @@ void StorageMergeTree::alter( changeSettings(new_changes, table_lock_holder); IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, settings_modifier); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, settings_modifier); setColumns(std::move(new_columns)); return; } @@ -326,7 +325,7 @@ void StorageMergeTree::alter( changeSettings(new_changes, table_lock_holder); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, storage_modifier); /// Reinitialize primary key because primary key column types might have changed. diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ea437a73b88..761c05cbc73 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3173,8 +3173,7 @@ void StorageReplicatedMergeTree::alter( LOG_DEBUG(log, "Doing ALTER"); - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); + auto table_id = getStorageID(); if (params.isSettingsAlter()) { @@ -3187,8 +3186,8 @@ void StorageReplicatedMergeTree::alter( changeSettings(new_changes, table_lock_holder); IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - global_context.getDatabase(current_database_name)->alterTable( - query_context, current_table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); + global_context.getDatabase(table_id.database_name)->alterTable( + query_context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); return; } @@ -3268,8 +3267,8 @@ void StorageReplicatedMergeTree::alter( changeSettings(new_changes, table_lock_holder); - global_context.getDatabase(current_database_name)->alterTable( - query_context, current_table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); + global_context.getDatabase(table_id.database_name)->alterTable( + query_context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); } diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index 36839e06196..c622cfbe235 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -48,20 +48,22 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context & { const String & config_name = table_data->merging_params.graphite_params.config_name; + auto table_id = table_data->getStorageID(); if (!graphite_configs.count(config_name)) { + //TODO add uuid Config new_config = { table_data->merging_params.graphite_params, - { table_data->getDatabaseName() }, - { table_data->getTableName() }, + { table_id.database_name }, + { table_id.database_name }, }; graphite_configs.emplace(config_name, new_config); } else { - graphite_configs[config_name].databases.emplace_back(table_data->getDatabaseName()); - graphite_configs[config_name].tables.emplace_back(table_data->getTableName()); + graphite_configs[config_name].databases.emplace_back(table_id.database_name); + graphite_configs[config_name].tables.emplace_back(table_id.database_name); } } } From 0c9ce7e15a9a91761ebe8286fda2159158434456 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Dec 2019 15:30:31 +0300 Subject: [PATCH 024/743] fixes --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 2 +- dbms/src/Interpreters/loadMetadata.cpp | 2 +- dbms/src/Storages/IStorage.cpp | 8 ++++---- dbms/src/Storages/IStorage.h | 4 ++-- dbms/src/Storages/StorageDistributed.cpp | 6 +++--- dbms/src/Storages/StorageFile.cpp | 6 +++--- dbms/src/Storages/System/StorageSystemGraphite.cpp | 4 ++-- dbms/tests/integration/test_filesystem_layout/test.py | 2 +- dbms/tests/integration/test_partition/test.py | 2 +- ...00226_zookeeper_deduplication_and_unexpected_parts.sql | 2 +- .../00611_zookeeper_different_checksums_formats.sql | 6 +++--- .../1_stateful/00065_loyalty_with_storage_join.sql | 2 +- 12 files changed, 23 insertions(+), 23 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 658f7ff80c1..2d27ff3a8b2 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -575,7 +575,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } String current_database = context.getCurrentDatabase(); - if (create.database.empty()) + if (!create.temporary && create.database.empty()) create.database = current_database; if (!create.to_table_id.empty() && create.to_table_id.database_name.empty()) create.to_table_id.database_name = current_database; diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 77c2eb7e08f..3bad70c36c4 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -109,7 +109,7 @@ void loadMetadata(Context & context, const String & default_database_name) } if (!default_database_name.empty() && !databases.count(default_database_name)) - databases.emplace(default_database_name, path + "/metadata/" + escapeForFileName(default_database_name)); + databases.emplace(default_database_name, path + "/" + escapeForFileName(default_database_name)); for (const auto & [name, db_path] : databases) loadDatabase(context, name, db_path, has_force_restore_data_flag); diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index a3f97337835..d8b76531677 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; } -IStorage::IStorage(StorageID id_, ColumnsDescription virtuals_) : id(std::move(id_)), virtuals(std::move(virtuals_)) +IStorage::IStorage(StorageID storage_id_, ColumnsDescription virtuals_) : storage_id(std::move(storage_id_)), virtuals(std::move(virtuals_)) { } @@ -469,7 +469,7 @@ StorageID IStorage::getStorageID(std::unique_lock * id_lock) const lock = std::unique_lock(id_mutex); else if (!*id_lock) *id_lock = std::unique_lock(id_mutex); - return id; + return storage_id; } void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name, @@ -480,8 +480,8 @@ void IStorage::renameInMemory(const String & new_database_name, const String & n lock = std::unique_lock(id_mutex); else if (!*id_lock) *id_lock = std::unique_lock(id_mutex); - id.database_name = new_database_name; - id.table_name = new_table_name; + storage_id.database_name = new_database_name; + storage_id.table_name = new_table_name; } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index ec84547061e..b707217fbbd 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -77,7 +77,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo { public: IStorage() = delete; - explicit IStorage(StorageID id_) : id(std::move(id_)) {} + explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)) {} IStorage(StorageID id_, ColumnsDescription virtuals_); virtual ~IStorage() = default; @@ -167,7 +167,7 @@ protected: /// still thread-unsafe part. IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; private: - StorageID id; + StorageID storage_id; mutable std::mutex id_mutex; ColumnsDescription columns; /// combined real and virtual columns const ColumnsDescription virtuals = {}; diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 4af368cbd43..a625b2203c5 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -345,18 +345,18 @@ BlockInputStreams StorageDistributed::read( if (has_sharding_key) { auto smaller_cluster = skipUnusedShards(cluster, query_info); - auto storage_id = getStorageID(); + auto table_id = getStorageID(); if (smaller_cluster) { cluster = smaller_cluster; - LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " + LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": " "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): " " " << makeFormattedListOfShards(cluster)); } else { - LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " + LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": " "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster"); } } diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index b3a6219099b..7b3e97720af 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -138,15 +138,15 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us : StorageFile(args) { is_db_table = false; - std::string db_dir_path_abs = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); + std::string user_files_path_abs = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); Poco::Path poco_path = Poco::Path(table_path_); if (poco_path.isRelative()) - poco_path = Poco::Path(user_files_path, poco_path); + poco_path = Poco::Path(user_files_path_abs, poco_path); const std::string path = poco_path.absolute().toString(); paths = listFilesWithRegexpMatching("/", path); for (const auto & cur_path : paths) - checkCreationIsAllowed(context_global, user_files_path, cur_path); + checkCreationIsAllowed(context_global, user_files_path_abs, cur_path); } StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args) diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index c622cfbe235..0af21f4dbc2 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -56,14 +56,14 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context & { table_data->merging_params.graphite_params, { table_id.database_name }, - { table_id.database_name }, + { table_id.table_name }, }; graphite_configs.emplace(config_name, new_config); } else { graphite_configs[config_name].databases.emplace_back(table_id.database_name); - graphite_configs[config_name].tables.emplace_back(table_id.database_name); + graphite_configs[config_name].tables.emplace_back(table_id.table_name); } } } diff --git a/dbms/tests/integration/test_filesystem_layout/test.py b/dbms/tests/integration/test_filesystem_layout/test.py index cb41085a857..da20e9091f5 100644 --- a/dbms/tests/integration/test_filesystem_layout/test.py +++ b/dbms/tests/integration/test_filesystem_layout/test.py @@ -16,7 +16,7 @@ def started_cluster(): def test_file_path_escaping(started_cluster): - node.query('CREATE DATABASE IF NOT EXISTS test') + node.query('CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary') node.query(''' CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32) ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id`; diff --git a/dbms/tests/integration/test_partition/test.py b/dbms/tests/integration/test_partition/test.py index 3365343b6fb..120f83f232c 100644 --- a/dbms/tests/integration/test_partition/test.py +++ b/dbms/tests/integration/test_partition/test.py @@ -14,7 +14,7 @@ path_to_data = '/var/lib/clickhouse/' def started_cluster(): try: cluster.start() - q('CREATE DATABASE test') + q('CREATE DATABASE test ENGINE = Ordinary') yield cluster diff --git a/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index 6b29d0a8cd3..1abe7d54610 100644 --- a/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1); SELECT * FROM deduplication; DETACH TABLE deduplication; -ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); +ATTACH TABLE deduplication;-- (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); SELECT * FROM deduplication; diff --git a/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql b/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql index 3fbb0d400f3..4f0dfb54906 100644 --- a/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql +++ b/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS table_old; DROP TABLE IF EXISTS table_new; -CREATE TABLE table_old (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=0; +CREATE TABLE table_old UUID '00000611-abcd-4000-8000-123456789ab6c' (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=0; CREATE TABLE table_new (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'new') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; SET insert_quorum=2; @@ -17,8 +17,8 @@ SELECT * FROM table_new ORDER BY k; SELECT 'DETACH'; DETACH TABLE table_old; -ATTACH TABLE table_old (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; +ATTACH TABLE table_old UUID '00000611-abcd-4000-8000-123456789ab6c' (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; SELECT * FROM table_old ORDER BY k; DROP TABLE IF EXISTS table_old; -DROP TABLE IF EXISTS table_new; \ No newline at end of file +DROP TABLE IF EXISTS table_new; diff --git a/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index d3e73faa7be..b03013c11fe 100644 --- a/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -24,7 +24,7 @@ GROUP BY loyalty ORDER BY loyalty ASC; DETACH TABLE join; -ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); +ATTACH TABLE join;-- (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); SELECT loyalty, From 3b710feeb6149c3b6a6fd1e30f2250df8a3fb2cf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Dec 2019 15:30:31 +0300 Subject: [PATCH 025/743] fixes --- dbms/src/Storages/IStorage.cpp | 8 ++++---- dbms/src/Storages/IStorage.h | 4 ++-- dbms/src/Storages/StorageDistributed.cpp | 6 +++--- dbms/src/Storages/System/StorageSystemGraphite.cpp | 4 ++-- dbms/tests/integration/test_filesystem_layout/test.py | 2 +- dbms/tests/integration/test_partition/test.py | 2 +- ...00226_zookeeper_deduplication_and_unexpected_parts.sql | 2 +- .../00611_zookeeper_different_checksums_formats.sql | 6 +++--- .../1_stateful/00065_loyalty_with_storage_join.sql | 2 +- 9 files changed, 18 insertions(+), 18 deletions(-) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index cc572226a2d..1a1da6d6f4a 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; } -IStorage::IStorage(StorageID id_, ColumnsDescription virtuals_) : id(std::move(id_)), virtuals(std::move(virtuals_)) +IStorage::IStorage(StorageID storage_id_, ColumnsDescription virtuals_) : storage_id(std::move(storage_id_)), virtuals(std::move(virtuals_)) { } @@ -469,7 +469,7 @@ StorageID IStorage::getStorageID(std::unique_lock * id_lock) const lock = std::unique_lock(id_mutex); else if (!*id_lock) *id_lock = std::unique_lock(id_mutex); - return id; + return storage_id; } void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name, @@ -480,8 +480,8 @@ void IStorage::renameInMemory(const String & new_database_name, const String & n lock = std::unique_lock(id_mutex); else if (!*id_lock) *id_lock = std::unique_lock(id_mutex); - id.database_name = new_database_name; - id.table_name = new_table_name; + storage_id.database_name = new_database_name; + storage_id.table_name = new_table_name; } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 559c903822e..b77770d52c8 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -77,7 +77,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo { public: IStorage() = delete; - explicit IStorage(StorageID id_) : id(std::move(id_)) {} + explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)) {} IStorage(StorageID id_, ColumnsDescription virtuals_); virtual ~IStorage() = default; @@ -167,7 +167,7 @@ protected: /// still thread-unsafe part. IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; private: - StorageID id; + StorageID storage_id; mutable std::mutex id_mutex; ColumnsDescription columns; /// combined real and virtual columns const ColumnsDescription virtuals = {}; diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 4346dfa395d..4ced2694361 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -345,18 +345,18 @@ BlockInputStreams StorageDistributed::read( if (has_sharding_key) { auto smaller_cluster = skipUnusedShards(cluster, query_info); - auto storage_id = getStorageID(); + auto table_id = getStorageID(); if (smaller_cluster) { cluster = smaller_cluster; - LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " + LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": " "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): " " " << makeFormattedListOfShards(cluster)); } else { - LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " + LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": " "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster"); } } diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index c622cfbe235..0af21f4dbc2 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -56,14 +56,14 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context & { table_data->merging_params.graphite_params, { table_id.database_name }, - { table_id.database_name }, + { table_id.table_name }, }; graphite_configs.emplace(config_name, new_config); } else { graphite_configs[config_name].databases.emplace_back(table_id.database_name); - graphite_configs[config_name].tables.emplace_back(table_id.database_name); + graphite_configs[config_name].tables.emplace_back(table_id.table_name); } } } diff --git a/dbms/tests/integration/test_filesystem_layout/test.py b/dbms/tests/integration/test_filesystem_layout/test.py index 16d63cbf2b7..83389b3d9bd 100644 --- a/dbms/tests/integration/test_filesystem_layout/test.py +++ b/dbms/tests/integration/test_filesystem_layout/test.py @@ -16,7 +16,7 @@ def started_cluster(): def test_file_path_escaping(started_cluster): - node.query('CREATE DATABASE IF NOT EXISTS test') + node.query('CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary') node.query(''' CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32) ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id`; diff --git a/dbms/tests/integration/test_partition/test.py b/dbms/tests/integration/test_partition/test.py index 659b9a89069..21b1e55f666 100644 --- a/dbms/tests/integration/test_partition/test.py +++ b/dbms/tests/integration/test_partition/test.py @@ -14,7 +14,7 @@ path_to_data = '/var/lib/clickhouse/' def started_cluster(): try: cluster.start() - q('CREATE DATABASE test') + q('CREATE DATABASE test ENGINE = Ordinary') yield cluster diff --git a/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index 6b29d0a8cd3..1abe7d54610 100644 --- a/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1); SELECT * FROM deduplication; DETACH TABLE deduplication; -ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); +ATTACH TABLE deduplication;-- (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); SELECT * FROM deduplication; diff --git a/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql b/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql index 3fbb0d400f3..4f0dfb54906 100644 --- a/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql +++ b/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS table_old; DROP TABLE IF EXISTS table_new; -CREATE TABLE table_old (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=0; +CREATE TABLE table_old UUID '00000611-abcd-4000-8000-123456789ab6c' (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=0; CREATE TABLE table_new (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'new') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; SET insert_quorum=2; @@ -17,8 +17,8 @@ SELECT * FROM table_new ORDER BY k; SELECT 'DETACH'; DETACH TABLE table_old; -ATTACH TABLE table_old (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; +ATTACH TABLE table_old UUID '00000611-abcd-4000-8000-123456789ab6c' (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; SELECT * FROM table_old ORDER BY k; DROP TABLE IF EXISTS table_old; -DROP TABLE IF EXISTS table_new; \ No newline at end of file +DROP TABLE IF EXISTS table_new; diff --git a/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index d3e73faa7be..b03013c11fe 100644 --- a/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -24,7 +24,7 @@ GROUP BY loyalty ORDER BY loyalty ASC; DETACH TABLE join; -ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); +ATTACH TABLE join;-- (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); SELECT loyalty, From a930697cd7aa8866dad80d60c4b46da1b977162d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 26 Dec 2019 21:51:54 +0300 Subject: [PATCH 026/743] TMP --- dbms/src/Interpreters/Context.cpp | 1 - dbms/src/Interpreters/Context.h | 3 -- dbms/src/Parsers/ASTAlterQuery.h | 1 - dbms/src/Parsers/ASTCreateQuery.h | 1 - dbms/src/Parsers/ExpressionElementParsers.cpp | 36 +------------------ dbms/src/Parsers/ExpressionElementParsers.h | 6 ---- dbms/src/Parsers/ParserCreateQuery.cpp | 1 - dbms/src/Storages/IStorage.cpp | 4 +-- dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 3 +- 10 files changed, 5 insertions(+), 53 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 48c193a34e9..4ffc712f6f0 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -979,7 +979,6 @@ StoragePtr Context::getTableImpl(const StorageID & table_id, Exception * excepti return res; } - //FIXME what if table was moved to another database? db = resolveDatabase(table_id.database_name, current_database); checkDatabaseAccessRightsImpl(db); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 53d77adf94e..11af06f2b09 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -124,7 +124,6 @@ using IHostContextPtr = std::shared_ptr; * * Everything is encapsulated for all sorts of checks and locks. */ -///TODO remove syntax sugar and legacy methods from Context (e.g. getInputFormat(...) which just returns object from factory) class Context { private: @@ -249,8 +248,6 @@ public: ClientInfo & getClientInfo() { return client_info; } const ClientInfo & getClientInfo() const { return client_info; } - void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address); - void addDependency(const StorageID & from, const StorageID & where); void removeDependency(const StorageID & from, const StorageID & where); Dependencies getDependencies(const StorageID & from) const; diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 2262624a1a4..02e76d5555c 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -142,7 +142,6 @@ public: String with_name; /// REPLACE(ATTACH) PARTITION partition FROM db.table - //FIXME use StorageID String from_database; String from_table; /// To distinguish REPLACE and ATTACH PARTITION partition FROM db.table diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 3beab213161..41847c28b1e 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 4ab56d3e83a..eb77d77a5c8 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -30,7 +30,7 @@ #include #include -#include +#include #include "ASTColumnsMatcher.h" @@ -198,40 +198,6 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex } -bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected) -{ - ParserKeyword s_uuid("UUID"); - ParserIdentifier name_p; - ParserStringLiteral uuid_p; - ParserToken s_dot(TokenType::Dot); - - ASTPtr database; - ASTPtr table; - ASTPtr uuid; - - 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 (s_uuid.ignore(pos, expected)) - { - if (!uuid_p.parse(pos, uuid, expected)) - return false; - } - - tryGetIdentifierNameInto(database, res.database_name); - tryGetIdentifierNameInto(table, res.table_name); - res.uuid = uuid ? uuid->as()->value.get() : ""; - return true; -} - - bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index 9a1e090d8d6..fbcaeeb3d45 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -56,12 +56,6 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); }; -struct StorageID; -/// Table name, possibly with database name and UUID as string literal -/// [db_name.]table_name [UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx'] -//TODO replace with class -bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected); - /// Just * class ParserAsterisk : public IParserBase { diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index db2b42f5ea5..43e5d274994 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -13,7 +13,6 @@ #include #include #include -#include namespace DB diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 1a1da6d6f4a..3faee4d841a 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -412,7 +412,7 @@ void IStorage::alter( SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); //FIXME + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); } else { @@ -421,7 +421,7 @@ void IStorage::alter( auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index b77770d52c8..5fc164c77b2 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -315,7 +315,7 @@ public: /** * Just updates names of database and table without moving any data on disk - * Can be called only from DatabaseAtomic. + * Can be called directly only from DatabaseAtomic. */ virtual void renameInMemory(const String & new_database_name, const String & new_table_name, std::unique_lock * id_lock = nullptr); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 2a11e14a602..56cf62ce342 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -312,7 +312,7 @@ bool StorageKafka::checkDependencies(const StorageID & table_id) return false; // Check all its dependencies - if (!checkDependencies(StorageID(db_tab.database_name, db_tab.table_name))) + if (!checkDependencies(db_tab)) return false; } @@ -360,7 +360,6 @@ bool StorageKafka::streamToViews() // Create an INSERT query for streaming data auto insert = std::make_shared(); - //FIXME use uid if not empty insert->database = table_id.database_name; insert->table = table_id.table_name; From a108651d631700584405f2da837c124a8a236be4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 15 Jan 2020 16:17:52 +0300 Subject: [PATCH 027/743] Revert "TMP" This reverts commit a930697cd7aa8866dad80d60c4b46da1b977162d. --- dbms/src/Interpreters/Context.h | 2 ++ dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index ae53572a9e5..cf94def0412 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -250,6 +250,8 @@ public: ClientInfo & getClientInfo() { return client_info; } const ClientInfo & getClientInfo() const { return client_info; } + void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address); + void addDependency(const StorageID & from, const StorageID & where); void removeDependency(const StorageID & from, const StorageID & where); Dependencies getDependencies(const StorageID & from) const; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 5fc164c77b2..b77770d52c8 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -315,7 +315,7 @@ public: /** * Just updates names of database and table without moving any data on disk - * Can be called directly only from DatabaseAtomic. + * Can be called only from DatabaseAtomic. */ virtual void renameInMemory(const String & new_database_name, const String & new_table_name, std::unique_lock * id_lock = nullptr); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 350370d8dcf..7f3de348aa3 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -313,7 +313,7 @@ bool StorageKafka::checkDependencies(const StorageID & table_id) return false; // Check all its dependencies - if (!checkDependencies(db_tab)) + if (!checkDependencies(StorageID(db_tab.database_name, db_tab.table_name))) return false; } From e179500c54633d316a73e07be4403624359f38b9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Jan 2020 21:13:18 +0300 Subject: [PATCH 028/743] fix --- dbms/programs/server/Server.cpp | 9 --------- dbms/src/Databases/DatabaseAtomic.cpp | 9 +++++++-- dbms/src/Databases/DatabaseOrdinary.cpp | 2 +- dbms/src/Databases/DatabaseWithDictionaries.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 8 +++++--- dbms/src/Interpreters/InterpreterShowCreateQuery.cpp | 4 ++++ dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 7 files changed, 19 insertions(+), 17 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index c1a9be5d28d..8746f98ff6b 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -273,15 +273,6 @@ int Server::main(const std::vector & /*args*/) global_context->setPath(path); - /// Check that we have read and write access to all data paths - auto disk_selector = global_context->getDiskSelector(); - for (const auto & [name, disk] : disk_selector.getDisksMap()) - { - Poco::File disk_path(disk->getPath()); - if (!disk_path.canRead() || !disk_path.canWrite()) - throw Exception("There is no RW access to disk " + name + " (" + disk->getPath() + ")", ErrorCodes::PATH_ACCESS_DENIED); - } - StatusFile status{path + "status"}; SCOPE_EXIT({ diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index 2f3bf34101f..f66f1a3fcc2 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -27,14 +27,18 @@ String DatabaseAtomic::getTableDataPath(const String & table_name) const auto it = table_name_to_path.find(table_name); if (it == table_name_to_path.end()) throw Exception("Table " + table_name + " not found in database " + getDatabaseName(), ErrorCodes::UNKNOWN_TABLE); - return data_path + it->second; + assert(it->second != data_path && !it->second.empty()); + return it->second; } String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const { //stringToUUID(query.uuid); /// Check UUID is valid const size_t uuid_prefix_len = 3; - return data_path + toString(query.uuid).substr(0, uuid_prefix_len) + '/' + toString(query.uuid) + '/'; + auto tmp = data_path + toString(query.uuid).substr(0, uuid_prefix_len) + '/' + toString(query.uuid) + '/'; + assert(tmp != data_path && !tmp.empty()); + return tmp; + } void DatabaseAtomic::drop(const Context &) @@ -44,6 +48,7 @@ void DatabaseAtomic::drop(const Context &) void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) { + assert(relative_table_path != data_path && !relative_table_path.empty()); DatabaseWithDictionaries::attachTable(name, table, relative_table_path); std::lock_guard lock(mutex); table_name_to_path.emplace(std::make_pair(name, relative_table_path)); diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 90f16395e9c..5df90575146 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -62,7 +62,7 @@ namespace StoragePtr table; std::tie(table_name, table) = createTableFromAST(query, database_name, database.getTableDataPath(query), context, has_force_restore_data_flag); - database.attachTable(table_name, table); + database.attachTable(table_name, table, database.getTableDataPath(query)); } catch (Exception & e) { diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index f33af8c2692..4672d6d2d93 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -100,7 +100,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S out.sync(); out.close(); } - + bool succeeded = false; SCOPE_EXIT({ if (!succeeded) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 54157a7d7c2..190787acb01 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -745,7 +745,9 @@ void Context::addDependencyUnsafe(const StorageID & from, const StorageID & wher { checkDatabaseAccessRightsImpl(from.database_name); checkDatabaseAccessRightsImpl(where.database_name); - shared->view_dependencies[from].insert(where); + // FIXME when loading metadata storage may not know UUIDs of it's dependencies, because they are not loaded yet, + // so UUID of `from` is not used here. + shared->view_dependencies[{from.database_name, from.table_name}].insert(where); // Notify table of dependencies change auto table = tryGetTable(from); @@ -763,7 +765,7 @@ void Context::removeDependencyUnsafe(const StorageID & from, const StorageID & w { checkDatabaseAccessRightsImpl(from.database_name); checkDatabaseAccessRightsImpl(where.database_name); - shared->view_dependencies[from].erase(where); + shared->view_dependencies[{from.database_name, from.table_name}].erase(where); // Notify table of dependencies change auto table = tryGetTable(from); @@ -792,7 +794,7 @@ Dependencies Context::getDependencies(const StorageID & from) const checkDatabaseAccessRightsImpl(db); } - ViewDependencies::const_iterator iter = shared->view_dependencies.find(StorageID(db, from.table_name, from.uuid)); + ViewDependencies::const_iterator iter = shared->view_dependencies.find(StorageID(db, from.table_name)); if (iter == shared->view_dependencies.end()) return {}; diff --git a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp index ad92e27b39c..934af18c584 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -69,6 +69,10 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (!create_query && show_query && show_query->temporary) throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY); + //FIXME temporary print create query without UUID for tests (remove it) + auto & create = create_query->as(); + create.uuid = UUID{UInt128{0, 0}}; + std::stringstream stream; formatAST(*create_query, stream, false, true); String res = stream.str(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index a8b67840f10..98f1075cd57 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1334,7 +1334,7 @@ void MergeTreeData::rename( for (const auto & disk : disks) { auto new_table_path_parent = Poco::Path(new_table_path).makeParent().toString(); - disk->createDirectory(new_table_path_parent); + disk->createDirectories(new_table_path_parent); disk->moveDirectory(relative_data_path, new_table_path); } From b322aaf1b5075cb634b1eb8127698c1d9b3958d8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 17 Jan 2020 19:07:20 +0300 Subject: [PATCH 029/743] remove boost::uuid --- dbms/src/Core/UUID.h | 14 ++++++++++++++ dbms/src/Databases/DatabaseOnDisk.cpp | 8 ++++---- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 12 ++++++------ .../Interpreters/InterpreterShowCreateQuery.cpp | 2 +- dbms/src/Parsers/ASTCreateQuery.cpp | 2 +- dbms/src/Parsers/ExpressionElementParsers.cpp | 2 +- dbms/src/Storages/StorageID.h | 6 +++--- dbms/tests/integration/test_backup_restore/test.py | 2 +- 8 files changed, 31 insertions(+), 17 deletions(-) diff --git a/dbms/src/Core/UUID.h b/dbms/src/Core/UUID.h index 4f8fdced814..d5a6232a72b 100644 --- a/dbms/src/Core/UUID.h +++ b/dbms/src/Core/UUID.h @@ -2,10 +2,24 @@ #include #include +#include namespace DB { STRONG_TYPEDEF(UInt128, UUID) +namespace UUIDHelpers +{ + inline UUID generateV4() + { + UInt128 res{thread_local_rng(), thread_local_rng()}; + res.low = (res.low & 0xffffffffffff0fffull) | 0x0000000000004000ull; + res.high = (res.high & 0x3fffffffffffffffull) | 0x8000000000000000ull; + return UUID{res}; + } + + const UUID Nil = UUID(UInt128(0, 0)); +} + } diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index f4dd5ffddc7..3d6ed5c25ff 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -112,7 +112,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) create->format = nullptr; create->out_file = nullptr; - if (create->uuid != UUID(UInt128(0, 0))) + if (create->uuid != UUIDHelpers::Nil) create->table = TABLE_WITH_UUID_NAME_PLACEHOLDER; std::ostringstream statement_stream; @@ -245,9 +245,9 @@ void DatabaseOnDisk::renameTable( auto & create = ast->as(); create.table = to_table_name; if (from_ordinary_to_atomic) - create.uuid = parseFromString(boost::uuids::to_string(boost::uuids::random_generator()())); + create.uuid = UUIDHelpers::generateV4(); if (from_atomic_to_ordinary) - create.uuid = UUID(UInt128(0, 0)); + create.uuid = UUIDHelpers::Nil; /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. try @@ -427,7 +427,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str return nullptr; auto & create = ast->as(); - if (create.uuid != UUID(UInt128(0, 0))) + if (create.uuid != UUIDHelpers::Nil) { String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName(); if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 4cc5c5ea7f0..e7c54f5a6a4 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -620,19 +620,19 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, if (database->getEngineName() == "Atomic") { //TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/ - if (create.attach && create.uuid == UUID(UInt128(0, 0))) + if (create.attach && create.uuid == UUIDHelpers::Nil) throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY); - if (!create.attach && create.uuid == UUID(UInt128(0, 0))) - create.uuid = parseFromString(boost::uuids::to_string(boost::uuids::random_generator()())); + if (!create.attach && create.uuid == UUIDHelpers::Nil) + create.uuid = UUIDHelpers::generateV4(); } else { - if (create.uuid != UUID(UInt128(0, 0))) + if (create.uuid != UUIDHelpers::Nil) throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", ErrorCodes::INCORRECT_QUERY); } - if (!create.attach && create.uuid == UUID(UInt128(0, 0)) && database->getEngineName() == "Atomic") - create.uuid = parseFromString(boost::uuids::to_string(boost::uuids::random_generator()())); + if (!create.attach && create.uuid == UUIDHelpers::Nil && database->getEngineName() == "Atomic") + create.uuid = UUIDHelpers::generateV4(); data_path = database->getTableDataPath(create); diff --git a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp index 934af18c584..a896c2f0a9d 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -71,7 +71,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() //FIXME temporary print create query without UUID for tests (remove it) auto & create = create_query->as(); - create.uuid = UUID{UInt128{0, 0}}; + create.uuid = UUIDHelpers::Nil; std::stringstream stream; formatAST(*create_query, stream, false, true); diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index 0260051cee1..3bcce48a7a5 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -234,7 +234,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); - if (uuid != UUID(UInt128(0, 0))) + if (uuid != UUIDHelpers::Nil) settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "") << quoteString(toString(uuid)); formatOnCluster(settings); diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 0f3422e2ee5..1cc3c25d224 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -229,7 +229,7 @@ bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected) tryGetIdentifierNameInto(database, res.database_name); tryGetIdentifierNameInto(table, res.table_name); //FIXME - res.uuid = uuid ? parseFromString(uuid->as()->value.get()) : UUID(UInt128(0, 0)); + res.uuid = uuid ? parseFromString(uuid->as()->value.get()) : UUIDHelpers::Nil; return true; } diff --git a/dbms/src/Storages/StorageID.h b/dbms/src/Storages/StorageID.h index 2cc19818de5..e95b1da6299 100644 --- a/dbms/src/Storages/StorageID.h +++ b/dbms/src/Storages/StorageID.h @@ -19,10 +19,10 @@ struct StorageID { String database_name; String table_name; - UUID uuid = UUID{UInt128(0, 0)}; + UUID uuid = UUIDHelpers::Nil; - StorageID(const String & database, const String & table, UUID uuid_ = UUID{UInt128(0, 0)}) + StorageID(const String & database, const String & table, UUID uuid_ = UUIDHelpers::Nil) : database_name(database), table_name(table), uuid(uuid_) { assertNotEmpty(); @@ -60,7 +60,7 @@ struct StorageID bool hasUUID() const { - return uuid != UUID{UInt128(0, 0)}; + return uuid != UUIDHelpers::Nil; } bool operator<(const StorageID & rhs) const diff --git a/dbms/tests/integration/test_backup_restore/test.py b/dbms/tests/integration/test_backup_restore/test.py index c63e056414a..46d687f7019 100644 --- a/dbms/tests/integration/test_backup_restore/test.py +++ b/dbms/tests/integration/test_backup_restore/test.py @@ -15,7 +15,7 @@ path_to_data = '/var/lib/clickhouse/' def started_cluster(): try: cluster.start() - q('CREATE DATABASE test') + q('CREATE DATABASE test ENGINE = Ordinary') yield cluster From 34e733f06d19b1b4302c2bffc27c4f0f4b707e19 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Jan 2020 14:30:11 +0300 Subject: [PATCH 030/743] basic async DROP --- dbms/src/Databases/DatabaseAtomic.cpp | 109 +++++++++++++++++- dbms/src/Databases/DatabaseAtomic.h | 27 ++++- dbms/src/Databases/DatabaseLazy.cpp | 4 +- dbms/src/Databases/DatabaseLazy.h | 2 +- dbms/src/Databases/DatabaseMemory.cpp | 18 ++- dbms/src/Databases/DatabaseMemory.h | 2 +- dbms/src/Databases/DatabaseMySQL.cpp | 7 +- dbms/src/Databases/DatabaseMySQL.h | 3 +- dbms/src/Databases/DatabaseOnDisk.cpp | 82 +++++++------ dbms/src/Databases/DatabaseOnDisk.h | 7 +- dbms/src/Databases/IDatabase.h | 4 +- .../Formats/ParsedTemplateFormatString.cpp | 26 ++++- .../src/Interpreters/InterpreterDropQuery.cpp | 41 +------ dbms/src/Storages/IStorage.h | 3 +- .../src/Storages/LiveView/StorageLiveView.cpp | 2 +- dbms/src/Storages/LiveView/StorageLiveView.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 3 +- dbms/src/Storages/StorageDistributed.h | 2 - dbms/src/Storages/StorageMaterializedView.cpp | 2 +- dbms/src/Storages/StorageMaterializedView.h | 2 +- dbms/src/Storages/StorageMemory.cpp | 2 +- dbms/src/Storages/StorageMemory.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageMergeTree.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 2 +- .../src/Storages/StorageReplicatedMergeTree.h | 2 +- dbms/src/Storages/StorageTinyLog.cpp | 7 +- dbms/src/Storages/StorageTinyLog.h | 2 +- 28 files changed, 257 insertions(+), 112 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index f66f1a3fcc2..f09ae2d7d9e 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -14,11 +14,13 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; } -DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Context & context_) +DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_) : DatabaseOrdinary(name_, metadata_path_, context_) { data_path = "store/"; - log = &Logger::get("DatabaseAtomic (" + name_ + ")"); + auto log_name = "DatabaseAtomic (" + name_ + ")"; + log = &Logger::get(log_name); + drop_task = context_.getSchedulePool().createTask(log_name, [this](){ this->dropTableDataTask(); }); } String DatabaseAtomic::getTableDataPath(const String & table_name) const @@ -44,6 +46,7 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const void DatabaseAtomic::drop(const Context &) { Poco::File(getMetadataPath()).remove(false); + //TODO check all tables dropped } void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) @@ -63,6 +66,39 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) return DatabaseWithDictionaries::detachTable(name); } +void DatabaseAtomic::dropTable(const Context & context, const String & table_name) +{ + String table_metadata_path = getObjectMetadataPath(table_name); + String table_metadata_path_drop = table_metadata_path + drop_suffix; + String table_data_path_relative = getTableDataPath(table_name); + assert(!table_data_path_relative.empty()); + + StoragePtr table = detachTable(table_name); + try + { + // FIXME + // 1. CREATE table_name: + table_name.sql + // 2. DROP table_name: table_name.sql -> table_name.sql.tmp_drop + // 3. CREATE table_name: + table_name.sql + // 4. DROP table_name: table_name.sql -> table_name.sql.tmp_drop overwrites table_name.sql.tmp_drop ? + Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); + + { + LOG_INFO(log, "Mark table " + table->getStorageID().getNameForLogs() + " to drop."); + std::lock_guard lock(tables_to_drop_mutex); + time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + tables_to_drop.push_back({table, context.getPath() + table_data_path_relative, current_time}); + } + } + catch (...) + { + LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); + attachTable(table_name, table, table_data_path_relative); + Poco::File(table_metadata_path_drop).renameTo(table_metadata_path); + throw; + } +} + void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, const String & to_table_name) { @@ -89,6 +125,75 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n Poco::File(getObjectMetadataPath(table_name)).renameTo(to_database.getObjectMetadataPath(to_table_name)); } +void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) +{ + DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag); + drop_task->activateAndSchedule(); +} + +void DatabaseAtomic::shutdown() +{ + drop_task->deactivate(); + DatabaseWithDictionaries::shutdown(); + //TODO try drop tables +} + +void DatabaseAtomic::dropTableDataTask() +{ + LOG_INFO(log, String("Wake up ") + __PRETTY_FUNCTION__); + TableToDrop table; + try + { + std::lock_guard lock(tables_to_drop_mutex); + LOG_INFO(log, "There are " + std::to_string(tables_to_drop.size()) + " tables to drop"); + time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + auto it = std::find_if(tables_to_drop.begin(), tables_to_drop.end(), [current_time, this](const TableToDrop & elem) + { + LOG_INFO(log, "Check table " + elem.table->getStorageID().getNameForLogs() + ": " + + "refcount = " + std::to_string(elem.table.unique()) + ", " + + "time elapsed = " + std::to_string(current_time - elem.drop_time)); + return elem.table.unique() && elem.drop_time + drop_delay_s < current_time; + }); + if (it != tables_to_drop.end()) + { + table = std::move(*it); + LOG_INFO(log, "Will try drop " + table.table->getStorageID().getNameForLogs()); + tables_to_drop.erase(it); + } + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + if (table.table) + { + try + { + LOG_INFO(log, "Trying to drop table " + table.table->getStorageID().getNameForLogs()); + table.table->drop(); + table.table->is_dropped = true; + Poco::File table_data_dir{table.data_path}; + if (table_data_dir.exists()) + table_data_dir.remove(true); + + String metadata_tmp_drop = getObjectMetadataPath(table.table->getStorageID().getTableName()) + drop_suffix; + Poco::File(metadata_tmp_drop).remove(); + } + catch (...) + { + tryLogCurrentException(log, "Cannot drop table " + table.table->getStorageID().getNameForLogs() + + ". Will retry later."); + { + std::lock_guard lock(tables_to_drop_mutex); + tables_to_drop.emplace_back(std::move(table)); + } + } + } + + drop_task->scheduleAfter(reschedule_time_ms); +} + } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index 42357e3b0ee..6e80df57eaa 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -11,7 +12,7 @@ class DatabaseAtomic : public DatabaseOrdinary { public: - DatabaseAtomic(String name_, String metadata_path_, const Context & context_); + DatabaseAtomic(String name_, String metadata_path_, Context & context_); String getEngineName() const override { return "Atomic"; } @@ -21,6 +22,9 @@ public: IDatabase & to_database, const String & to_table_name) override; + //void removeTable(const Context & context, const String & table_name) override; + void dropTable(const Context & context, const String & table_name) override; + void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path = {}) override; StoragePtr detachTable(const String & name) override; @@ -29,9 +33,30 @@ public: void drop(const Context & /*context*/) override; + void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; + void shutdown() override; + private: + void dropTableDataTask(); + +private: + static constexpr time_t drop_delay_s = 10; + static constexpr size_t reschedule_time_ms = 5000; + //TODO store path in DatabaseWithOwnTables::tables std::map table_name_to_path; + struct TableToDrop + { + StoragePtr table; + String data_path; + time_t drop_time; + //time_t last_attempt_time; + }; + using TablesToDrop = std::list; + TablesToDrop tables_to_drop; + std::mutex tables_to_drop_mutex; + + BackgroundSchedulePoolTaskHolder drop_task; }; diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 4a931694af0..933514b57e5 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -65,12 +65,12 @@ void DatabaseLazy::createTable( it->second.metadata_modification_time = DatabaseOnDisk::getObjectMetadataModificationTime(table_name); } -void DatabaseLazy::removeTable( +void DatabaseLazy::dropTable( const Context & context, const String & table_name) { SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::removeTable(context, table_name); + DatabaseOnDisk::dropTable(context, table_name); } void DatabaseLazy::renameTable( diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index 8fd04fea7c8..effda57a4ba 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -32,7 +32,7 @@ public: const StoragePtr & table, const ASTPtr & query) override; - void removeTable( + void dropTable( const Context & context, const String & table_name) override; diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index 28a6a8a7af0..765d80c9fee 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include namespace DB @@ -21,11 +23,23 @@ void DatabaseMemory::createTable( attachTable(table_name, table); } -void DatabaseMemory::removeTable( +void DatabaseMemory::dropTable( const Context & /*context*/, const String & table_name) { - detachTable(table_name); + auto table = detachTable(table_name); + try + { + table->drop(); + Poco::File table_data_dir{getTableDataPath(table_name)}; + if (table_data_dir.exists()) + table_data_dir.remove(true); + } + catch (...) + { + attachTable(table_name, table); + } + table->is_dropped = true; } ASTPtr DatabaseMemory::getCreateDatabaseQuery(const Context & /*context*/) const diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 01789c11901..27abf6e87be 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -29,7 +29,7 @@ public: const StoragePtr & table, const ASTPtr & query) override; - void removeTable( + void dropTable( const Context & context, const String & table_name) override; diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index 504909c6f1b..4295afcb8a9 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -431,7 +431,7 @@ void DatabaseMySQL::loadStoredObjects(Context &, bool) } } -void DatabaseMySQL::removeTable(const Context &, const String & table_name) +void DatabaseMySQL::dropTable(const Context &, const String & table_name) { std::lock_guard lock{mutex}; @@ -445,7 +445,8 @@ void DatabaseMySQL::removeTable(const Context &, const String & table_name) throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " does not exists remove tables, it is bug.", ErrorCodes::LOGICAL_ERROR); - if (!local_tables_cache.count(table_name)) + auto table_iter = local_tables_cache.find(table_name); + if (table_iter == local_tables_cache.end()) throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -453,6 +454,7 @@ void DatabaseMySQL::removeTable(const Context &, const String & table_name) try { + table_iter->second.second->drop(); remove_flag.createFile(); } catch (...) @@ -460,6 +462,7 @@ void DatabaseMySQL::removeTable(const Context &, const String & table_name) remove_or_detach_tables.erase(table_name); throw; } + table_iter->second.second->is_dropped = true; } DatabaseMySQL::~DatabaseMySQL() diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index fe4aa6d55ae..01885be8d70 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -1,6 +1,7 @@ #pragma once #include "config_core.h" +#define USE_MYSQL 1 #if USE_MYSQL #include @@ -52,7 +53,7 @@ public: StoragePtr detachTable(const String & table_name) override; - void removeTable(const Context &, const String & table_name) override; + void dropTable(const Context &, const String & table_name) override; void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path = {}) override; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 3d6ed5c25ff..c9da90a3fd0 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -159,7 +159,7 @@ void DatabaseOnDisk::createTable( throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); String table_metadata_path = getObjectMetadataPath(table_name); - String table_metadata_tmp_path = table_metadata_path + ".tmp"; + String table_metadata_tmp_path = table_metadata_path + create_suffix; String statement; { @@ -190,30 +190,36 @@ void DatabaseOnDisk::createTable( } } -void DatabaseOnDisk::removeTable(const Context & /* context */, const String & table_name) +void DatabaseOnDisk::dropTable(const Context & context, const String & table_name) { - String table_data = getTableDataPath(table_name); - StoragePtr res = detachTable(table_name); String table_metadata_path = getObjectMetadataPath(table_name); + String table_metadata_path_drop = table_metadata_path + drop_suffix; + String table_data_path_relative = getTableDataPath(table_name); + assert(!table_data_path_relative.empty()); + StoragePtr table = detachTable(table_name); + bool renamed = false; try { - Poco::File(table_metadata_path).remove(); + Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); + renamed = true; + table->drop(); + table->is_dropped = true; + + Poco::File table_data_dir{context.getPath() + table_data_path_relative}; + if (table_data_dir.exists()) + table_data_dir.remove(true); } catch (...) { - try - { - Poco::File(table_metadata_path + ".tmp_drop").remove(); - return; - } - catch (...) - { - LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); - } - attachTable(table_name, res, data_path); + LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); + attachTable(table_name, table, table_data_path_relative); + if (renamed) + Poco::File(table_metadata_path_drop).renameTo(table_metadata_path); throw; } + + Poco::File(table_metadata_path_drop).remove(); } void DatabaseOnDisk::renameTable( @@ -234,41 +240,43 @@ void DatabaseOnDisk::renameTable( throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); } - StoragePtr table = tryGetTable(context, table_name); - - if (!table) - throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - - auto table_lock = table->lockExclusively(context.getCurrentQueryId()); - - ASTPtr ast = parseQueryFromMetadata(context, getObjectMetadataPath(table_name)); - auto & create = ast->as(); - create.table = to_table_name; - if (from_ordinary_to_atomic) - create.uuid = UUIDHelpers::generateV4(); - if (from_atomic_to_ordinary) - create.uuid = UUIDHelpers::Nil; - - /// Notify the table that it is renamed. If the table does not support renaming, exception is thrown. + auto table_data_relative_path = getTableDataPath(table_name); + TableStructureWriteLockHolder table_lock; + String table_metadata_path; + ASTPtr attach_query; + StoragePtr table = detachTable(table_name); try { - table->rename(to_database.getTableDataPath(create), - to_database.getDatabaseName(), - to_table_name, table_lock); + table_lock = table->lockExclusively(context.getCurrentQueryId()); + + table_metadata_path = getObjectMetadataPath(table_name); + attach_query = parseQueryFromMetadata(context, table_metadata_path); + auto & create = attach_query->as(); + create.table = to_table_name; + if (from_ordinary_to_atomic) + create.uuid = UUIDHelpers::generateV4(); + if (from_atomic_to_ordinary) + create.uuid = UUIDHelpers::Nil; + + /// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID + table->rename(to_database.getTableDataPath(create), to_database.getDatabaseName(), to_table_name, table_lock); } catch (const Exception &) { + attachTable(table_name, table, table_data_relative_path); throw; } catch (const Poco::Exception & e) { + attachTable(table_name, table, table_data_relative_path); /// Better diagnostics. throw Exception{Exception::CreateFromPoco, e}; } - /// NOTE Non-atomic. - to_database.createTable(context, to_table_name, table, ast); - removeTable(context, table_name); + /// Now table data are moved to new database, so we must add metadata and attach table to new database + to_database.createTable(context, to_table_name, table, attach_query); + + Poco::File(table_metadata_path).remove(); } ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 4025fabaef0..602a435c633 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -38,7 +38,7 @@ public: const StoragePtr & table, const ASTPtr & query) override; - void removeTable( + void dropTable( const Context & context, const String & table_name) override; @@ -62,6 +62,9 @@ public: String getMetadataPath() const override { return metadata_path; } protected: + static constexpr const char * create_suffix = ".tmp"; + static constexpr const char * drop_suffix = ".tmp_drop"; + using IteratingFunction = std::function; void iterateMetadataFiles(const Context & context, const IteratingFunction & iterating_function) const; @@ -73,6 +76,8 @@ protected: ASTPtr parseQueryFromMetadata(const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false) const; ASTPtr getCreateQueryFromMetadata(const Context & context, const String & metadata_path, bool throw_on_error) const; + //bool detachTableAndRemoveMetadata(const String & table_name); + //void replaceMetadata(const ASTPtr & create, ); const String metadata_path; /*const*/ String data_path; diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 159bf45ce61..de320cdba7c 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -170,8 +170,8 @@ public: throw Exception("There is no CREATE DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } - /// Delete the table from the database. Delete the metadata. - virtual void removeTable( + /// Delete the table from the database, drop table and delete the metadata. + virtual void dropTable( const Context & /*context*/, const String & /*name*/) { diff --git a/dbms/src/Formats/ParsedTemplateFormatString.cpp b/dbms/src/Formats/ParsedTemplateFormatString.cpp index 981d43089a2..fe5810d8a0b 100644 --- a/dbms/src/Formats/ParsedTemplateFormatString.cpp +++ b/dbms/src/Formats/ParsedTemplateFormatString.cpp @@ -16,11 +16,11 @@ namespace ErrorCodes ParsedTemplateFormatString::ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name) { + ReadBufferFromFile schema_file(schema.absoluteSchemaPath(), 4096); + String format_string; + readStringUntilEOF(format_string, schema_file); try { - ReadBufferFromFile schema_file(schema.absoluteSchemaPath(), 4096); - String format_string; - readStringUntilEOF(format_string, schema_file); parse(format_string, idx_by_name); } catch (DB::Exception & e) @@ -76,7 +76,14 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum case Column: column_names.emplace_back(); - pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back()); + try + { + pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back()); + } + catch (const DB::Exception & e) + { + throwInvalidFormat(e.message(), columnsCount()); + } if (*pos == ':') state = Format; @@ -99,7 +106,16 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum errno = 0; column_idx = strtoull(column_names.back().c_str(), &col_idx_end, 10); if (col_idx_end != column_names.back().c_str() + column_names.back().size() || errno) - column_idx = idx_by_name(column_names.back()); + { + try + { + column_idx = idx_by_name(column_names.back()); + } + catch (const DB::Exception & e) + { + throwInvalidFormat(e.message(), columnsCount()); + } + } } format_idx_to_column_idx.emplace_back(column_idx); break; diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index b1995869baf..ad04dd4f41c 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -106,43 +107,7 @@ BlockIO InterpreterDropQuery::executeToTable( auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); - const std::string metadata_file_without_extension = - database_and_table.first->getMetadataPath() - + escapeForFileName(table_id.table_name); - - const auto prev_metadata_name = metadata_file_without_extension + ".sql"; - const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop"; - - /// Try to rename metadata file and delete the data - //TODO move this logic to DatabaseOnDisk - try - { - /// There some kind of tables that have no metadata - ignore renaming - if (Poco::File(prev_metadata_name).exists()) - Poco::File(prev_metadata_name).renameTo(drop_metadata_name); - /// Delete table data - database_and_table.second->drop(table_lock); - } - catch (...) - { - if (Poco::File(drop_metadata_name).exists()) - Poco::File(drop_metadata_name).renameTo(prev_metadata_name); - throw; - } - - String table_data_path_relative = database_and_table.first->getTableDataPath(table_name); - - /// Delete table metadata and table itself from memory - database_and_table.first->removeTable(context, table_id.table_name); - database_and_table.second->is_dropped = true; - - /// If it is not virtual database like Dictionary then drop remaining data dir - if (!table_data_path_relative.empty()) - { - String table_data_path = context.getPath() + table_data_path_relative; - if (Poco::File(table_data_path).exists()) - Poco::File(table_data_path).remove(true); - } + database_and_table.first->dropTable(context, table_name); } } @@ -217,7 +182,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); /// Delete table data - table->drop(table_lock); + table->drop(); table->is_dropped = true; } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index dcb805b141e..a5655d65903 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -291,9 +291,10 @@ public: /** Delete the table data. Called before deleting the directory with the data. * The method can be called only after detaching table from Context (when no queries are performed with table). * The table is not usable during and after call to this method. + * If some queries may still use the table, then it must be called under exclusive lock. * If you do not need any action other than deleting the directory with data, you can leave this method blank. */ - virtual void drop(TableStructureWriteLockHolder &) {} + virtual void drop() {} /** Clear the table data and leave it empty. * Must be called under lockForAlter. diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 6eca05d2c48..ca804b28b4a 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -487,7 +487,7 @@ StorageLiveView::~StorageLiveView() } } -void StorageLiveView::drop(TableStructureWriteLockHolder &) +void StorageLiveView::drop() { auto table_id = getStorageID(); global_context.removeDependency(select_table_id, table_id); diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 77992b97e19..ba8e56317c8 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -124,7 +124,7 @@ public: } void checkTableCanBeDropped() const override; - void drop(TableStructureWriteLockHolder &) override; + void drop() override; void startup() override; void shutdown() override; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index c47afbf407e..a060813f351 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1381,7 +1381,8 @@ void MergeTreeData::dropAllData() auto full_paths = getDataPaths(); for (auto && full_data_path : full_paths) - Poco::File(full_data_path).remove(true); + if (Poco::File(full_data_path).exists()) + Poco::File(full_data_path).remove(true); LOG_TRACE(log, "dropAllData: done."); } diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index c1e3c639c1f..7a3ecdc377b 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -75,8 +75,6 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void drop(TableStructureWriteLockHolder &) override {} - /// Removes temporary data in local filesystem. void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 2bb79dcaa1d..10dd3704680 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -209,7 +209,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, } -void StorageMaterializedView::drop(TableStructureWriteLockHolder &) +void StorageMaterializedView::drop() { auto table_id = getStorageID(); if (!select_table_id.empty()) diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index f9a57783871..3af066b9619 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -31,7 +31,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void drop(TableStructureWriteLockHolder &) override; + void drop() override; void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index b2cb5682f6d..fbef2c75331 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -123,7 +123,7 @@ BlockOutputStreamPtr StorageMemory::write( } -void StorageMemory::drop(TableStructureWriteLockHolder &) +void StorageMemory::drop() { std::lock_guard lock(mutex); data.clear(); diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 1e66b17606b..fd21cf356ff 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -38,7 +38,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void drop(TableStructureWriteLockHolder &) override; + void drop() override; void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index f5279ebef92..ccc04de7ee6 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -170,7 +170,7 @@ void StorageMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition) global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size); } -void StorageMergeTree::drop(TableStructureWriteLockHolder &) +void StorageMergeTree::drop() { shutdown(); dropAllData(); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 765b43ed90e..43d38c40de2 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -62,7 +62,7 @@ public: CancellationCode killMutation(const String & mutation_id) override; - void drop(TableStructureWriteLockHolder &) override; + void drop() override; void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; void alter(const AlterCommands & commands, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 2fdd7daa684..96428573ddb 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3798,7 +3798,7 @@ void StorageReplicatedMergeTree::checkPartitionCanBeDropped(const ASTPtr & parti } -void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &) +void StorageReplicatedMergeTree::drop() { { auto zookeeper = tryGetZooKeeper(); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 0fff99b00f3..bbe12a05071 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -113,7 +113,7 @@ public: /** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper. */ - void drop(TableStructureWriteLockHolder &) override; + void drop() override; void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index aeb90399816..2b9ac0df296 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -425,10 +425,13 @@ void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWri addFiles(column.name, *column.type); } -void StorageTinyLog::drop(TableStructureWriteLockHolder &) +void StorageTinyLog::drop() { + // TODO do we really need another rwlock here if drop() is called under exclusive table lock? std::unique_lock lock(rwlock); - disk->removeRecursive(table_path); + // TODO may be move it to DatabaseOnDisk + if (disk->exists(table_path)) + disk->removeRecursive(table_path); files.clear(); } diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index b9a45a9f271..ed75849d843 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -46,7 +46,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void drop(TableStructureWriteLockHolder &) override; + void drop() override; protected: StorageTinyLog( From e4a889c7f43e9e42aad17d192ec3a62878917d95 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Jan 2020 22:10:09 +0300 Subject: [PATCH 031/743] retry DROP after restart --- dbms/src/Databases/DatabaseAtomic.cpp | 24 +++++++++- dbms/src/Databases/DatabaseOnDisk.cpp | 63 +++++++++++++++---------- dbms/src/Databases/DatabaseOnDisk.h | 6 ++- dbms/src/Databases/DatabaseOrdinary.cpp | 1 + dbms/src/Databases/DatabaseOrdinary.h | 2 +- 5 files changed, 68 insertions(+), 28 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index f09ae2d7d9e..a047456e5d5 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -80,7 +81,7 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam // 1. CREATE table_name: + table_name.sql // 2. DROP table_name: table_name.sql -> table_name.sql.tmp_drop // 3. CREATE table_name: + table_name.sql - // 4. DROP table_name: table_name.sql -> table_name.sql.tmp_drop overwrites table_name.sql.tmp_drop ? + // 4. DROP table_name: table_name.sql -> table_name.sql.tmp_drop overwrites table_name.sql.tmp_drop Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); { @@ -127,6 +128,27 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) { + iterateMetadataFiles(context, [](const String &) {}, [&](const String & file_name) + { + String full_path = getMetadataPath() + file_name; + LOG_INFO(log, "Trying load partially dropped table from " << full_path); + try + { + auto ast = parseQueryFromMetadata(context, full_path, /*throw_on_error*/ true, /*remove_empty*/false); + if (!ast) //TODO why? + return; + auto & query = ast->as(); + auto [_, table] = createTableFromAST(query, database_name, getTableDataPath(query), context, has_force_restore_data_flag); + time_t drop_time = Poco::File(full_path).getLastModified().epochTime(); + tables_to_drop.push_back({table, context.getPath() + getTableDataPath(query), drop_time}); + } + catch (Exception & e) + { + e.addMessage("Cannot complete table drop " + full_path); + //TODO may be remove data dir (if UUID successfully parsed) and .tmp_drop metadata? + } + }); + DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag); drop_task->activateAndSchedule(); } diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index c9da90a3fd0..92f2864e715 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -342,7 +342,33 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & table_na return static_cast(0); } -void DatabaseOnDisk::iterateMetadataFiles(const Context & /*context*/, const IteratingFunction & iterating_function) const +void DatabaseOnDisk::iterateMetadataFiles(const Context & context, + const DatabaseOnDisk::IteratingFunction & process_metadata_file) const +{ + IteratingFunction process_tmp_drop_metadata_file = [&](const String & file_name) + { + static const char * tmp_drop_ext = ".sql.tmp_drop"; + const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); + if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists()) + { + Poco::File(getMetadataPath() + file_name).renameTo(context.getPath() + getMetadataPath() + object_name + ".sql"); + LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously and will be restored"); + process_metadata_file(object_name + ".sql"); + } + else + { + LOG_INFO(log, "Removing file " << getMetadataPath() + file_name); + Poco::File(getMetadataPath() + file_name).remove(); + } + }; + + IteratingFunction do_nothing = [](const String &){}; + //FIXME refactor this trash + iterateMetadataFiles(context, process_metadata_file, dynamic_cast(this) ? do_nothing : process_tmp_drop_metadata_file); +} + +void DatabaseOnDisk::iterateMetadataFiles(const Context & /*context*/, const IteratingFunction & process_metadata_file, + const IteratingFunction & process_tmp_drop_metadata_file) const { Poco::DirectoryIterator dir_end; for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) @@ -355,39 +381,22 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & /*context*/, const Ite if (endsWith(dir_it.name(), ".sql.bak")) continue; - // There are files that we tried to delete previously static const char * tmp_drop_ext = ".sql.tmp_drop"; if (endsWith(dir_it.name(), tmp_drop_ext)) { - //const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext)); - //if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists()) - //{ - // /// TODO maybe complete table drop and remove all table data (including data on other volumes and metadata in ZK) - // //TODO check all paths - // Poco::File(dir_it->path()).renameTo(context.getPath() + getMetadataPath() + object_name + ".sql"); - // LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously and will be restored"); - // iterating_function(object_name + ".sql"); - //} - //else - //{ - // LOG_INFO(log, "Removing file " << dir_it->path()); - // Poco::File(dir_it->path()).remove(); - //} - continue; + /// There are files that we tried to delete previously + process_tmp_drop_metadata_file(dir_it.name()); } - - /// There are files .sql.tmp - delete - if (endsWith(dir_it.name(), ".sql.tmp")) + else if (endsWith(dir_it.name(), ".sql.tmp")) { + /// There are files .sql.tmp - delete LOG_INFO(log, "Removing file " << dir_it->path()); Poco::File(dir_it->path()).remove(); - continue; } - - /// The required files have names like `table_name.sql` - if (endsWith(dir_it.name(), ".sql")) + else if (endsWith(dir_it.name(), ".sql")) { - iterating_function(dir_it.name()); + /// The required files have names like `table_name.sql` + process_metadata_file(dir_it.name()); } else throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + getMetadataPath(), @@ -437,7 +446,11 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str auto & create = ast->as(); if (create.uuid != UUIDHelpers::Nil) { + //FIXME it can be .sql or .sql.tmp_drop String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName(); + if (Poco::Path(table_name).makeFile().getExtension() == "sql") + table_name = Poco::Path(table_name).makeFile().getBaseName(); + if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER) LOG_WARNING(log, "File " << metadata_file_path << " contains both UUID and table name. " "Will use name `" << table_name << "` instead of `" << create.table << "`"); diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 602a435c633..608d529a410 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -66,7 +66,10 @@ protected: static constexpr const char * drop_suffix = ".tmp_drop"; using IteratingFunction = std::function; - void iterateMetadataFiles(const Context & context, const IteratingFunction & iterating_function) const; + + void iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const; + void iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file, + const IteratingFunction & process_tmp_drop_metadata_file) const; ASTPtr getCreateTableQueryImpl( const Context & context, @@ -76,6 +79,7 @@ protected: ASTPtr parseQueryFromMetadata(const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false) const; ASTPtr getCreateQueryFromMetadata(const Context & context, const String & metadata_path, bool throw_on_error) const; + //bool detachTableAndRemoveMetadata(const String & table_name); //void replaceMetadata(const ASTPtr & create, ); diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 5df90575146..28e8ae90a8d 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -29,6 +29,7 @@ #include #include #include +#include "DatabaseAtomic.h" namespace DB diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index 41c03b5103e..620a0ddf868 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -27,7 +27,7 @@ public: const String & name, const StorageInMemoryMetadata & metadata) override; -private: +protected: void startupTables(ThreadPool & thread_pool); }; From 9d0ac7aae1d7ff40585c57ae295567f9f5966483 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 27 Jan 2020 23:31:39 +0300 Subject: [PATCH 032/743] better DatabaseAtomic::drop() --- dbms/src/Databases/DatabaseAtomic.cpp | 114 +++++++++++++++--- dbms/src/Databases/DatabaseAtomic.h | 19 +-- .../src/Interpreters/InterpreterDropQuery.cpp | 4 +- 3 files changed, 108 insertions(+), 29 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index a047456e5d5..affa606d387 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -13,6 +14,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int TABLE_ALREADY_EXISTS; extern const int FILE_DOESNT_EXIST; + extern const int DATABASE_NOT_EMPTY; } DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_) @@ -46,8 +48,55 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const void DatabaseAtomic::drop(const Context &) { - Poco::File(getMetadataPath()).remove(false); - //TODO check all tables dropped + + //constexpr size_t max_attempts = 5; + //for (size_t i = 0; i < max_attempts; ++i) + //{ + // auto it = tables_to_drop.begin(); + // while (it != tables_to_drop.end()) + // { + // if (it->table.unique()) + // { + // /// No queries use table, it can be safely dropped + // dropTableFinally(*it); + // it = tables_to_drop.erase(it); + // } + // ++it; + // } + // + // if (tables_to_drop.empty()) + // { + // Poco::File(getMetadataPath()).remove(false); + // return; + // } + //} + //throw Exception("Cannot drop database", ErrorCodes::TABLE_WAS_NOT_DROPPED); + + /// IDatabase::drop() is called under global context lock (TODO can it be fixed?) + + auto it = std::find_if(tables_to_drop.begin(), tables_to_drop.end(), [](const TableToDrop & elem) + { + return !elem.table.unique(); + }); + if (it != tables_to_drop.end()) + throw Exception("Cannot drop database " + getDatabaseName() + + ". It contains table " + it->table->getStorageID().getNameForLogs() + + ", which is used by " + std::to_string(it->table.use_count() - 1) + " queries. " + "Client should retry later.", ErrorCodes::DATABASE_NOT_EMPTY); + + for (auto & table : tables_to_drop) + { + try + { + dropTableFinally(table); + } + catch (...) + { + tryLogCurrentException(log, "Cannot drop table. Metadata " + table.data_path + " will be removed forcefully. " + "Garbage may be left in /store directory and ZooKeeper."); + } + } + Poco::File(getMetadataPath()).remove(true); } void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) @@ -130,22 +179,43 @@ void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore { iterateMetadataFiles(context, [](const String &) {}, [&](const String & file_name) { + /// Process .sql.tmp_drop files with metadata of partially dropped tables String full_path = getMetadataPath() + file_name; LOG_INFO(log, "Trying load partially dropped table from " << full_path); + ASTPtr ast; + const ASTCreateQuery * create = nullptr; try { - auto ast = parseQueryFromMetadata(context, full_path, /*throw_on_error*/ true, /*remove_empty*/false); - if (!ast) //TODO why? + ast = parseQueryFromMetadata(context, full_path, /*throw_on_error*/ false, /*remove_empty*/false); + create = typeid_cast(ast.get()); + if (!create || create->uuid == UUIDHelpers::Nil) + { + LOG_WARNING(log, "Cannot parse metadata of partially dropped table from " << full_path + << ". Removing metadata. Garbage may be left in /store directory and ZooKeeper."); + if (Poco::File(full_path).exists()) + Poco::File(full_path).remove(); return; - auto & query = ast->as(); - auto [_, table] = createTableFromAST(query, database_name, getTableDataPath(query), context, has_force_restore_data_flag); + } + auto [_, table] = createTableFromAST(*create, database_name, getTableDataPath(*create), context, has_force_restore_data_flag); time_t drop_time = Poco::File(full_path).getLastModified().epochTime(); - tables_to_drop.push_back({table, context.getPath() + getTableDataPath(query), drop_time}); + tables_to_drop.push_back({table, context.getPath() + getTableDataPath(*create), drop_time}); } - catch (Exception & e) + catch (...) { - e.addMessage("Cannot complete table drop " + full_path); - //TODO may be remove data dir (if UUID successfully parsed) and .tmp_drop metadata? + if (!create) + throw; + auto table_data_relative_path = getTableDataPath(*create); + if (table_data_relative_path.empty()) + throw; + + Poco::File table_data{context.getPath() + table_data_relative_path}; + tryLogCurrentException(log, "Cannot load partially dropped table from: " + full_path + + ". Parsed query: " + serializeAST(*create) + + ". Removing metadata and " + table_data.path() + + ". Garbage may be left in ZooKeeper."); + if (table_data.exists()) + table_data.remove(true); + Poco::File{full_path}.remove(); } }); @@ -157,7 +227,6 @@ void DatabaseAtomic::shutdown() { drop_task->deactivate(); DatabaseWithDictionaries::shutdown(); - //TODO try drop tables } void DatabaseAtomic::dropTableDataTask() @@ -192,15 +261,7 @@ void DatabaseAtomic::dropTableDataTask() { try { - LOG_INFO(log, "Trying to drop table " + table.table->getStorageID().getNameForLogs()); - table.table->drop(); - table.table->is_dropped = true; - Poco::File table_data_dir{table.data_path}; - if (table_data_dir.exists()) - table_data_dir.remove(true); - - String metadata_tmp_drop = getObjectMetadataPath(table.table->getStorageID().getTableName()) + drop_suffix; - Poco::File(metadata_tmp_drop).remove(); + dropTableFinally(table); } catch (...) { @@ -216,6 +277,19 @@ void DatabaseAtomic::dropTableDataTask() drop_task->scheduleAfter(reschedule_time_ms); } +void DatabaseAtomic::dropTableFinally(const DatabaseAtomic::TableToDrop & table) const +{ + LOG_INFO(log, "Trying to drop table " + table.table->getStorageID().getNameForLogs()); + table.table->drop(); + table.table->is_dropped = true; + Poco::File table_data_dir{table.data_path}; + if (table_data_dir.exists()) + table_data_dir.remove(true); + + String metadata_tmp_drop = getObjectMetadataPath(table.table->getStorageID().getTableName()) + drop_suffix; + Poco::File(metadata_tmp_drop).remove(); +} + } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index 6e80df57eaa..0f97c00d20c 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -37,14 +37,6 @@ public: void shutdown() override; private: - void dropTableDataTask(); - -private: - static constexpr time_t drop_delay_s = 10; - static constexpr size_t reschedule_time_ms = 5000; - - //TODO store path in DatabaseWithOwnTables::tables - std::map table_name_to_path; struct TableToDrop { StoragePtr table; @@ -53,6 +45,17 @@ private: //time_t last_attempt_time; }; using TablesToDrop = std::list; + + void dropTableDataTask(); + void dropTableFinally(const TableToDrop & table) const; + +private: + static constexpr time_t drop_delay_s = 10; + static constexpr size_t reschedule_time_ms = 5000; + + //TODO store path in DatabaseWithOwnTables::tables + std::map table_name_to_path; + TablesToDrop tables_to_drop; std::mutex tables_to_drop_mutex; diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index ad04dd4f41c..efe18ecb5e9 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -105,7 +105,9 @@ BlockIO InterpreterDropQuery::executeToTable( database_and_table.second->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); + TableStructureWriteLockHolder table_lock; + if (database_and_table.first->getEngineName() != "Atomic") + table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); database_and_table.first->dropTable(context, table_name); } From b82693cb07df6ab2b6595188950a39651917bc2f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Jan 2020 22:39:52 +0300 Subject: [PATCH 033/743] fix some tests --- dbms/src/Databases/DatabaseAtomic.cpp | 4 ++- dbms/src/Databases/DatabaseOnDisk.cpp | 2 +- dbms/src/Interpreters/Context.h | 2 ++ .../src/Interpreters/InterpreterDropQuery.cpp | 7 ++-- .../test_atomic_drop_table/test.py | 2 +- .../test_distributed_format/test.py | 23 ++++++------ .../test.py | 29 +++++++-------- .../integration/test_system_merges/test.py | 36 +++++++++++-------- .../00040_aggregating_materialized_view.sql | 10 +++--- .../00041_aggregating_materialized_view.sql | 16 ++++----- 10 files changed, 73 insertions(+), 58 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index affa606d387..3bb85400263 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -135,9 +135,11 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam { LOG_INFO(log, "Mark table " + table->getStorageID().getNameForLogs() + " to drop."); + /// Context:getPath acquires lock + auto data_path = context.getPath() + table_data_path_relative; std::lock_guard lock(tables_to_drop_mutex); time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - tables_to_drop.push_back({table, context.getPath() + table_data_path_relative, current_time}); + tables_to_drop.push_back({table, data_path, current_time}); } } catch (...) diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 92f2864e715..01f6dc42c2f 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -351,7 +351,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists()) { - Poco::File(getMetadataPath() + file_name).renameTo(context.getPath() + getMetadataPath() + object_name + ".sql"); + Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql"); LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously and will be restored"); process_metadata_file(object_name + ".sql"); } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index c870c279fc8..9541fcc8bd6 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -402,6 +402,8 @@ public: std::chrono::steady_clock::duration closeSessions() const; /// For methods below you may need to acquire a lock by yourself. + /// NOTE: It's dangerous. While holding Context lock you should not take any other locks or call methods, which may take any locks, + /// until you are sure that all locks are always taken in right order. std::unique_lock getLock() const; const Context & getQueryContext() const; diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index efe18ecb5e9..09e87a6b7d8 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -222,6 +222,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQ executeToDictionary(database_name, current_dictionary, kind, false, false, false); } + //FIXME get rid of global context lock here auto context_lock = context.getLock(); /// Someone could have time to delete the database before us. @@ -231,14 +232,14 @@ BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQ if (!context.getDatabase(database_name)->empty(context)) throw Exception("New table appeared in database being dropped. Try dropping it again.", ErrorCodes::DATABASE_NOT_EMPTY); - /// Delete database information from the RAM - context.detachDatabase(database_name); - database->shutdown(); /// Delete the database. database->drop(context); + /// Delete database information from the RAM + context.detachDatabase(database_name); + /// Old ClickHouse versions did not store database.sql files Poco::File database_metadata_file(context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"); if (database_metadata_file.exists()) diff --git a/dbms/tests/integration/test_atomic_drop_table/test.py b/dbms/tests/integration/test_atomic_drop_table/test.py index 7d845baeec6..279d13ac4da 100644 --- a/dbms/tests/integration/test_atomic_drop_table/test.py +++ b/dbms/tests/integration/test_atomic_drop_table/test.py @@ -13,7 +13,7 @@ node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) def start_cluster(): try: cluster.start() - node1.query("CREATE DATABASE zktest;") + node1.query("CREATE DATABASE zktest ENGINE=Ordinary;") node1.query( ''' CREATE TABLE zktest.atomic_drop_table (n UInt32) diff --git a/dbms/tests/integration/test_distributed_format/test.py b/dbms/tests/integration/test_distributed_format/test.py index 2bcc8ab8063..d34eb9c9817 100644 --- a/dbms/tests/integration/test_distributed_format/test.py +++ b/dbms/tests/integration/test_distributed_format/test.py @@ -16,6 +16,7 @@ node = cluster.add_instance('node', config_dir="configs", main_configs=['configs def started_cluster(): try: cluster.start() + node.query("create database test engine=Ordinary") yield cluster finally: @@ -23,37 +24,37 @@ def started_cluster(): def test_single_file(started_cluster): - node.query("create table distr_1 (x UInt64, s String) engine = Distributed('test_cluster', database, table)") - node.query("insert into distr_1 values (1, 'a'), (2, 'bb'), (3, 'ccc')") + node.query("create table test.distr_1 (x UInt64, s String) engine = Distributed('test_cluster', database, table)") + node.query("insert into test.distr_1 values (1, 'a'), (2, 'bb'), (3, 'ccc')") - query = "select * from file('/var/lib/clickhouse/data/default/distr_1/default@not_existing:9000/1.bin', 'Distributed')" + query = "select * from file('/var/lib/clickhouse/data/test/distr_1/default@not_existing:9000/1.bin', 'Distributed')" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) assert out == '1\ta\n2\tbb\n3\tccc\n' - query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/default/distr_1/default@not_existing:9000/1.bin');" \ + query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_1/default@not_existing:9000/1.bin');" \ "select * from t" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) assert out == '1\ta\n2\tbb\n3\tccc\n' - node.query("drop table distr_1") + node.query("drop table test.distr_1") def test_two_files(started_cluster): - node.query("create table distr_2 (x UInt64, s String) engine = Distributed('test_cluster', database, table)") - node.query("insert into distr_2 values (0, '_'), (1, 'a')") - node.query("insert into distr_2 values (2, 'bb'), (3, 'ccc')") + node.query("create table test.distr_2 (x UInt64, s String) engine = Distributed('test_cluster', database, table)") + node.query("insert into test.distr_2 values (0, '_'), (1, 'a')") + node.query("insert into test.distr_2 values (2, 'bb'), (3, 'ccc')") - query = "select * from file('/var/lib/clickhouse/data/default/distr_2/default@not_existing:9000/{1,2,3,4}.bin', 'Distributed') order by x" + query = "select * from file('/var/lib/clickhouse/data/test/distr_2/default@not_existing:9000/{1,2,3,4}.bin', 'Distributed') order by x" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) assert out == '0\t_\n1\ta\n2\tbb\n3\tccc\n' - query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/default/distr_2/default@not_existing:9000/{1,2,3,4}.bin');" \ + query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_2/default@not_existing:9000/{1,2,3,4}.bin');" \ "select * from t order by x" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) assert out == '0\t_\n1\ta\n2\tbb\n3\tccc\n' - node.query("drop table distr_2") + node.query("drop table test.distr_2") diff --git a/dbms/tests/integration/test_distributed_storage_configuration/test.py b/dbms/tests/integration/test_distributed_storage_configuration/test.py index a7c33b059c3..4061b36f1f2 100644 --- a/dbms/tests/integration/test_distributed_storage_configuration/test.py +++ b/dbms/tests/integration/test_distributed_storage_configuration/test.py @@ -16,6 +16,7 @@ node = cluster.add_instance('node', def start_cluster(): try: cluster.start() + node.query('CREATE DATABASE test ENGINE=Ordinary') yield cluster finally: cluster.shutdown() @@ -25,41 +26,41 @@ def _files_in_dist_mon(node, root, table): 'bash', '-c', # `-maxdepth 1` to avoid /tmp/ subdirectory - 'find /{root}/data/default/{table}/default@127%2E0%2E0%2E2:9000 -maxdepth 1 -type f | wc -l'.format(root=root, table=table) + 'find /{root}/data/test/{table}/default@127%2E0%2E0%2E2:9000 -maxdepth 1 -type f | wc -l'.format(root=root, table=table) ]).split('\n')[0]) def test_different_versions(start_cluster): - node.query('CREATE TABLE foo (key Int) Engine=Memory()') + node.query('CREATE TABLE test.foo (key Int) Engine=Memory()') node.query(""" - CREATE TABLE dist_foo (key Int) + CREATE TABLE test.dist_foo (key Int) Engine=Distributed( test_cluster_two_shards, - currentDatabase(), + test, foo, key%2, 'default' ) """) # manual only - node.query('SYSTEM STOP DISTRIBUTED SENDS dist_foo') + node.query('SYSTEM STOP DISTRIBUTED SENDS test.dist_foo') - node.query('INSERT INTO dist_foo SELECT * FROM numbers(100)') + node.query('INSERT INTO test.dist_foo SELECT * FROM numbers(100)') assert _files_in_dist_mon(node, 'disk1', 'dist_foo') == 1 assert _files_in_dist_mon(node, 'disk2', 'dist_foo') == 0 - assert node.query('SELECT count() FROM dist_foo') == '100\n' - node.query('SYSTEM FLUSH DISTRIBUTED dist_foo') - assert node.query('SELECT count() FROM dist_foo') == '200\n' + assert node.query('SELECT count() FROM test.dist_foo') == '100\n' + node.query('SYSTEM FLUSH DISTRIBUTED test.dist_foo') + assert node.query('SELECT count() FROM test.dist_foo') == '200\n' # # RENAME # - node.query('RENAME TABLE dist_foo TO dist2_foo') + node.query('RENAME TABLE test.dist_foo TO test.dist2_foo') - node.query('INSERT INTO dist2_foo SELECT * FROM numbers(100)') + node.query('INSERT INTO test.dist2_foo SELECT * FROM numbers(100)') assert _files_in_dist_mon(node, 'disk1', 'dist2_foo') == 0 assert _files_in_dist_mon(node, 'disk2', 'dist2_foo') == 1 - assert node.query('SELECT count() FROM dist2_foo') == '300\n' - node.query('SYSTEM FLUSH DISTRIBUTED dist2_foo') - assert node.query('SELECT count() FROM dist2_foo') == '400\n' + assert node.query('SELECT count() FROM test.dist2_foo') == '300\n' + node.query('SYSTEM FLUSH DISTRIBUTED test.dist2_foo') + assert node.query('SELECT count() FROM test.dist2_foo') == '400\n' diff --git a/dbms/tests/integration/test_system_merges/test.py b/dbms/tests/integration/test_system_merges/test.py index 7b638ce05c7..8e3714bc23b 100644 --- a/dbms/tests/integration/test_system_merges/test.py +++ b/dbms/tests/integration/test_system_merges/test.py @@ -22,6 +22,8 @@ node2 = cluster.add_instance('node2', def started_cluster(): try: cluster.start() + node1.query('CREATE DATABASE test ENGINE=Ordinary') + node2.query('CREATE DATABASE test ENGINE=Ordinary') yield cluster finally: @@ -39,7 +41,10 @@ def split_tsv(data): def test_merge_simple(started_cluster, replicated): try: clickhouse_path = "/var/lib/clickhouse" - name = "test_merge_simple" + db_name = "test" + table_name = "merge_simple" + name = db_name + "." + table_name + table_path = "data/" + db_name + "/" + table_name nodes = [node1, node2] if replicated else [node1] engine = "ReplicatedMergeTree('/clickhouse/test_merge_simple', '{replica}')" if replicated else "MergeTree()" node_check = nodes[-1] @@ -75,15 +80,15 @@ def test_merge_simple(started_cluster, replicated): SELECT database, table, num_parts, source_part_names, source_part_paths, result_part_name, result_part_path, partition_id, is_mutation FROM system.merges WHERE table = '{name}' - """.format(name=name))) == [ + """.format(name=table_name))) == [ [ - "default", - name, + db_name, + table_name, "3", "['{}','{}','{}']".format(*parts), - "['{clickhouse}/data/default/{name}/{}/','{clickhouse}/data/default/{name}/{}/','{clickhouse}/data/default/{name}/{}/']".format(*parts, clickhouse=clickhouse_path, name=name), + "['{clickhouse}/{table_path}/{}/','{clickhouse}/{table_path}/{}/','{clickhouse}/{table_path}/{}/']".format(*parts, clickhouse=clickhouse_path, table_path=table_path), result_part, - "{clickhouse}/data/default/{name}/{}/".format(result_part, clickhouse=clickhouse_path, name=name), + "{clickhouse}/{table_path}/{}/".format(result_part, clickhouse=clickhouse_path, table_path=table_path), "all", "0" ] @@ -91,7 +96,7 @@ def test_merge_simple(started_cluster, replicated): t.join() wait.join() - assert node_check.query("SELECT * FROM system.merges WHERE table = '{name}'".format(name=name)) == "" + assert node_check.query("SELECT * FROM system.merges WHERE table = '{name}'".format(name=table_name)) == "" finally: for node in nodes: @@ -105,7 +110,10 @@ def test_merge_simple(started_cluster, replicated): def test_mutation_simple(started_cluster, replicated): try: clickhouse_path = "/var/lib/clickhouse" - name = "test_mutation_simple" + db_name = "test" + table_name = "mutation_simple" + name = db_name + "." + table_name + table_path = "data/" + db_name + "/" + table_name nodes = [node1, node2] if replicated else [node1] engine = "ReplicatedMergeTree('/clickhouse/test_mutation_simple', '{replica}')" if replicated else "MergeTree()" node_check = nodes[-1] @@ -136,15 +144,15 @@ def test_mutation_simple(started_cluster, replicated): SELECT database, table, num_parts, source_part_names, source_part_paths, result_part_name, result_part_path, partition_id, is_mutation FROM system.merges WHERE table = '{name}' - """.format(name=name))) == [ + """.format(name=table_name))) == [ [ - "default", - name, + db_name, + table_name, "1", "['{}']".format(part), - "['{clickhouse}/data/default/{name}/{}/']".format(part, clickhouse=clickhouse_path, name=name), + "['{clickhouse}/{table_path}/{}/']".format(part, clickhouse=clickhouse_path, table_path=table_path), result_part, - "{clickhouse}/data/default/{name}/{}/".format(result_part, clickhouse=clickhouse_path, name=name), + "{clickhouse}/{table_path}/{}/".format(result_part, clickhouse=clickhouse_path, table_path=table_path), "all", "1" ], @@ -153,7 +161,7 @@ def test_mutation_simple(started_cluster, replicated): time.sleep(1.5) - assert node_check.query("SELECT * FROM system.merges WHERE table = '{name}'".format(name=name)) == "" + assert node_check.query("SELECT * FROM system.merges WHERE table = '{name}'".format(name=table_name)) == "" finally: for node in nodes: diff --git a/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql b/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql index 909c4325a75..555a8b64d75 100644 --- a/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql +++ b/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql @@ -1,6 +1,6 @@ -DROP TABLE IF EXISTS test.basic; +DROP TABLE IF EXISTS test.basic_00040; -CREATE MATERIALIZED VIEW test.basic +CREATE MATERIALIZED VIEW test.basic_00040 ENGINE = AggregatingMergeTree(StartDate, (CounterID, StartDate), 8192) POPULATE AS SELECT @@ -16,7 +16,7 @@ SELECT StartDate, sumMerge(Visits) AS Visits, uniqMerge(Users) AS Users -FROM test.basic +FROM test.basic_00040 GROUP BY StartDate ORDER BY StartDate; @@ -25,7 +25,7 @@ SELECT StartDate, sumMerge(Visits) AS Visits, uniqMerge(Users) AS Users -FROM test.basic +FROM test.basic_00040 WHERE CounterID = 942285 GROUP BY StartDate ORDER BY StartDate; @@ -41,4 +41,4 @@ GROUP BY StartDate ORDER BY StartDate; -DROP TABLE test.basic; +DROP TABLE test.basic_00040; diff --git a/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql b/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql index 8ac1ef0f405..b91b0a3a2c2 100644 --- a/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql +++ b/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql @@ -14,8 +14,8 @@ ENGINE = AggregatingMergeTree(StartDate, (CounterID, StartDate), 8192) AS SELECT CounterID, StartDate, - sumState(Sign) AS Visits, - uniqState(UserID) AS Users + sumState(Sign) AS Visits, + uniqState(UserID) AS Users FROM test.visits_null GROUP BY CounterID, StartDate; @@ -30,8 +30,8 @@ FROM test.visits; SELECT StartDate, - sumMerge(Visits) AS Visits, - uniqMerge(Users) AS Users + sumMerge(Visits) AS Visits, + uniqMerge(Users) AS Users FROM test.basic GROUP BY StartDate ORDER BY StartDate; @@ -39,8 +39,8 @@ ORDER BY StartDate; SELECT StartDate, - sumMerge(Visits) AS Visits, - uniqMerge(Users) AS Users + sumMerge(Visits) AS Visits, + uniqMerge(Users) AS Users FROM test.basic WHERE CounterID = 942285 GROUP BY StartDate @@ -49,8 +49,8 @@ ORDER BY StartDate; SELECT StartDate, - sum(Sign) AS Visits, - uniq(UserID) AS Users + sum(Sign) AS Visits, + uniq(UserID) AS Users FROM test.visits WHERE CounterID = 942285 GROUP BY StartDate From 1147436d60d19c882109915c417a45e383d52e12 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Wed, 29 Jan 2020 18:50:18 +0300 Subject: [PATCH 034/743] Raw add of direct dictionaries (doesnt work) --- dbms/src/Dictionaries/DirectDictionary.cpp | 582 ++++++++++++++++++ dbms/src/Dictionaries/DirectDictionary.h | 294 +++++++++ dbms/src/Dictionaries/DirectDictionary.inc.h | 406 ++++++++++++ .../src/Dictionaries/registerDictionaries.cpp | 3 + 4 files changed, 1285 insertions(+) create mode 100644 dbms/src/Dictionaries/DirectDictionary.cpp create mode 100644 dbms/src/Dictionaries/DirectDictionary.h create mode 100644 dbms/src/Dictionaries/DirectDictionary.inc.h diff --git a/dbms/src/Dictionaries/DirectDictionary.cpp b/dbms/src/Dictionaries/DirectDictionary.cpp new file mode 100644 index 00000000000..9e0a77ebc91 --- /dev/null +++ b/dbms/src/Dictionaries/DirectDictionary.cpp @@ -0,0 +1,582 @@ +#include "DirectDictionary.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "DirectDictionary.inc.h" +#include "DictionaryBlockInputStream.h" +#include "DictionaryFactory.h" + + +/* + * + * TODO: CHANGE EVENTS TO DIRECT DICTIONARY EVENTS (WTF? WHERE R THEY DECLARED????) + * +*/ + +namespace ProfileEvents +{ + extern const Event DictCacheKeysRequested; + extern const Event DictCacheKeysRequestedMiss; + extern const Event DictCacheKeysRequestedFound; + extern const Event DictCacheKeysExpired; + extern const Event DictCacheKeysNotFound; + extern const Event DictCacheKeysHit; + extern const Event DictCacheRequestTimeNs; + extern const Event DictCacheRequests; + extern const Event DictCacheLockWriteNs; + extern const Event DictCacheLockReadNs; +} + +namespace CurrentMetrics +{ + extern const Metric DictCacheRequests; +} + + +namespace DB +{ + namespace ErrorCodes + { + extern const int TYPE_MISMATCH; + extern const int BAD_ARGUMENTS; + extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; + extern const int TOO_SMALL_BUFFER_SIZE; + } + + /* + * deleted inline size_t DirectDictionary::getCellIdx(const Key id) const + * + */ + + + DirectDictionary::DirectDictionary( + const std::string & name_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + const DictionaryLifetime dict_lifetime_ + ) + : name{name_} + , dict_struct(dict_struct_) + , source_ptr{std::move(source_ptr_)} + , dict_lifetime(dict_lifetime_) + , log(&Logger::get("ExternalDictionaries")) + , rnd_engine(randomSeed()) + { + if (!this->source_ptr->supportsSelectiveLoad()) + throw Exception{name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD}; + + createAttributes(); + } + + + void DirectDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const + { + const auto null_value = std::get(hierarchical_attribute->null_values); + + getItemsNumberImpl(*hierarchical_attribute, ids, out, [&](const size_t) { return null_value; }); + } + + +/// Allow to use single value in same way as array. + static inline DirectDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) + { + return arr[idx]; + } + static inline DirectDictionary::Key getAt(const DirectDictionary::Key & value, const size_t) + { + return value; + } + + + template + void DirectDictionary::isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const + { + /// Transform all children to parents until ancestor id or null_value will be reached. + + size_t out_size = out.size(); + memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated" + + const auto null_value = std::get(hierarchical_attribute->null_values); + + PaddedPODArray children(out_size, 0); + PaddedPODArray parents(child_ids.begin(), child_ids.end()); + + while (true) + { + size_t out_idx = 0; + size_t parents_idx = 0; + size_t new_children_idx = 0; + + while (out_idx < out_size) + { + /// Already calculated + if (out[out_idx] != 0xFF) + { + ++out_idx; + continue; + } + + /// No parent + if (parents[parents_idx] == null_value) + { + out[out_idx] = 0; + } + /// Found ancestor + else if (parents[parents_idx] == getAt(ancestor_ids, parents_idx)) + { + out[out_idx] = 1; + } + /// Loop detected + else if (children[new_children_idx] == parents[parents_idx]) + { + out[out_idx] = 1; + } + /// Found intermediate parent, add this value to search at next loop iteration + else + { + children[new_children_idx] = parents[parents_idx]; + ++new_children_idx; + } + + ++out_idx; + ++parents_idx; + } + + if (new_children_idx == 0) + break; + + /// Transform all children to its parents. + children.resize(new_children_idx); + parents.resize(new_children_idx); + + toParent(children, parents); + } + } + + void DirectDictionary::isInVectorVector( + const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const + { + isInImpl(child_ids, ancestor_ids, out); + } + + void DirectDictionary::isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const + { + isInImpl(child_ids, ancestor_id, out); + } + + void DirectDictionary::isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const + { + /// Special case with single child value. + + const auto null_value = std::get(hierarchical_attribute->null_values); + + PaddedPODArray child(1, child_id); + PaddedPODArray parent(1); + std::vector ancestors(1, child_id); + + /// Iteratively find all ancestors for child. + while (true) + { + toParent(child, parent); + + if (parent[0] == null_value) + break; + + child[0] = parent[0]; + ancestors.push_back(parent[0]); + } + + /// Assuming short hierarchy, so linear search is Ok. + for (size_t i = 0, out_size = out.size(); i < out_size; ++i) + out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end(); + } + + void DirectDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const + { + auto & attribute = getAttribute(attribute_name); + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + const auto null_value = StringRef{std::get(attribute.null_values)}; + + getItemsString(attribute, ids, out, [&](const size_t) { return null_value; }); + } + + void DirectDictionary::getString( + const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const + { + auto & attribute = getAttribute(attribute_name); + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + getItemsString(attribute, ids, out, [&](const size_t row) { return def->getDataAt(row); }); + } + + void DirectDictionary::getString( + const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const + { + auto & attribute = getAttribute(attribute_name); + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + getItemsString(attribute, ids, out, [&](const size_t) { return StringRef{def}; }); + } + + +/// returns cell_idx (always valid for replacing), 'cell is valid' flag, 'cell is outdated' flag +/// true false found and valid +/// false true not found (something outdated, maybe our cell) +/// false false not found (other id stored with valid data) +/// true true impossible +/// +/// todo: split this func to two: find_for_get and find_for_set + DirectDictionary::FindResult DirectDictionary::findCellIdx(const Key & id, const CellMetadata::time_point_t now) const + { + auto pos = getCellIdx(id); + auto oldest_id = pos; + auto oldest_time = CellMetadata::time_point_t::max(); + const auto stop = pos + max_collision_length; + for (; pos < stop; ++pos) + { + const auto cell_idx = pos & size_overlap_mask; + const auto & cell = cells[cell_idx]; + + if (cell.id != id) + { + /// maybe we already found nearest expired cell (try minimize collision_length on insert) + if (oldest_time > now && oldest_time > cell.expiresAt()) + { + oldest_time = cell.expiresAt(); + oldest_id = cell_idx; + } + continue; + } + + if (cell.expiresAt() < now) + { + return {cell_idx, false, true}; + } + + return {cell_idx, true, false}; + } + + return {oldest_id, false, false}; + } + + + /* + * deleted most part of has, that stood for + * looking for a key in cache + * + * TODO: check whether we need last two arguments + * in update function (seems like no) + * + */ + + void DirectDictionary::has(const PaddedPODArray & ids, PaddedPODArray & out) const + { + std::vector required_ids(ids.size()); + std::copy(std::begin(ids), std::end(ids), std::begin(required_ids)); + + /// request new values + update( + required_ids, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + out[row] = true; + }, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + out[row] = false; + }); + } + + + void DirectDictionary::createAttributes() + { + const auto attributes_size = dict_struct.attributes.size(); + attributes.reserve(attributes_size); + + bytes_allocated += size * sizeof(CellMetadata); + bytes_allocated += attributes_size * sizeof(attributes.front()); + + for (const auto & attribute : dict_struct.attributes) + { + attribute_index_by_name.emplace(attribute.name, attributes.size()); + attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); + + if (attribute.hierarchical) + { + hierarchical_attribute = &attributes.back(); + + if (hierarchical_attribute->type != AttributeUnderlyingType::utUInt64) + throw Exception{name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH}; + } + } + } + + DirectDictionary::Attribute DirectDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) + { + Attribute attr{type, {}, {}}; + + switch (type) + { +#define DISPATCH(TYPE) \ + case AttributeUnderlyingType::ut##TYPE: \ + attr.null_values = TYPE(null_value.get>()); \ + attr.arrays = std::make_unique>(size); \ + bytes_allocated += size * sizeof(TYPE); \ + break; + DISPATCH(UInt8) + DISPATCH(UInt16) + DISPATCH(UInt32) + DISPATCH(UInt64) + DISPATCH(UInt128) + DISPATCH(Int8) + DISPATCH(Int16) + DISPATCH(Int32) + DISPATCH(Int64) + DISPATCH(Decimal32) + DISPATCH(Decimal64) + DISPATCH(Decimal128) + DISPATCH(Float32) + DISPATCH(Float64) +#undef DISPATCH + case AttributeUnderlyingType::utString: + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); + bytes_allocated += size * sizeof(StringRef); + if (!string_arena) + string_arena = std::make_unique(); + break; + } + + return attr; + } + + void DirectDictionary::setDefaultAttributeValue(Attribute & attribute, const Key idx) const + { + switch (attribute.type) + { + case AttributeUnderlyingType::utUInt8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utUInt16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utUInt32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utUInt64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utUInt128: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utInt8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utInt16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utInt32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utInt64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utFloat32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utFloat64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + + case AttributeUnderlyingType::utDecimal32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utDecimal64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utDecimal128: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + + case AttributeUnderlyingType::utString: + { + const auto & null_value_ref = std::get(attribute.null_values); + auto & string_ref = std::get>(attribute.arrays)[idx]; + + if (string_ref.data != null_value_ref.data()) + { + if (string_ref.data) + string_arena->free(const_cast(string_ref.data), string_ref.size); + + string_ref = StringRef{null_value_ref}; + } + + break; + } + } + } + + void DirectDictionary::setAttributeValue(Attribute & attribute, const Key idx, const Field & value) const + { + switch (attribute.type) + { + case AttributeUnderlyingType::utUInt8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utUInt16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utUInt32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utUInt64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utUInt128: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utInt8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utInt16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utInt32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utInt64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utFloat32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utFloat64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + + case AttributeUnderlyingType::utDecimal32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utDecimal64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utDecimal128: + std::get>(attribute.arrays)[idx] = value.get(); + break; + + case AttributeUnderlyingType::utString: + { + const auto & string = value.get(); + auto & string_ref = std::get>(attribute.arrays)[idx]; + const auto & null_value_ref = std::get(attribute.null_values); + + /// free memory unless it points to a null_value + if (string_ref.data && string_ref.data != null_value_ref.data()) + string_arena->free(const_cast(string_ref.data), string_ref.size); + + const auto str_size = string.size(); + if (str_size != 0) + { + auto string_ptr = string_arena->alloc(str_size + 1); + std::copy(string.data(), string.data() + str_size + 1, string_ptr); + string_ref = StringRef{string_ptr, str_size}; + } + else + string_ref = {}; + + break; + } + } + } + + DirectDictionary::Attribute & DirectDictionary::getAttribute(const std::string & attribute_name) const + { + const auto it = attribute_index_by_name.find(attribute_name); + if (it == std::end(attribute_index_by_name)) + throw Exception{name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; + + return attributes[it->second]; + } + + /* + * I've deleted: + * bool CacheDictionary::isEmptyCell(const UInt64 idx) const + * and + * PaddedPODArray CacheDictionary::getCachedIds() const + */ + + BlockInputStreamPtr DirectDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const + { + using BlockInputStreamType = DictionaryBlockInputStream; + + /* deleted pre-last argument getCachedIds() from this return (will something break then?) */ + return std::make_shared(shared_from_this(), max_block_size, column_names); + } + + std::exception_ptr DirectDictionary::getLastException() const + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + return last_exception; + } + + void registerDictionaryDirect(DictionaryFactory & factory) + { + auto create_layout = [=](const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr) -> DictionaryPtr + { + if (dict_struct.key) + throw Exception{"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD}; + + if (dict_struct.range_min || dict_struct.range_max) + throw Exception{name + + ": elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + ErrorCodes::BAD_ARGUMENTS}; + const auto & layout_prefix = config_prefix + ".layout"; + + /* + * + * seems like this stands only for cache dictionaries + * + const auto size = config.getInt(layout_prefix + ".cache.size_in_cells"); + if (size == 0) + throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; + + */ + + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + if (require_nonempty) + throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", + ErrorCodes::BAD_ARGUMENTS}; + + const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; + + /* deleted last argument (size) in this return */ + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime); + }; + factory.registerLayout("direct", create_layout, false); + } + + +} diff --git a/dbms/src/Dictionaries/DirectDictionary.h b/dbms/src/Dictionaries/DirectDictionary.h new file mode 100644 index 00000000000..1431adccf02 --- /dev/null +++ b/dbms/src/Dictionaries/DirectDictionary.h @@ -0,0 +1,294 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "DictionaryStructure.h" +#include "IDictionary.h" +#include "IDictionarySource.h" + + +namespace DB +{ + class DirectDictionary final : public IDictionary + { + public: + /* Removed last argument (size_) */ + DirectDictionary( + const std::string & name_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + const DictionaryLifetime dict_lifetime_ + ); + + std::string getName() const override { return name; } + + std::string getTypeName() const override { return "Cache"; } + + size_t getBytesAllocated() const override { return bytes_allocated + (string_arena ? string_arena->size() : 0); } + + size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } + + double getHitRate() const override + { + return static_cast(hit_count.load(std::memory_order_acquire)) / query_count.load(std::memory_order_relaxed); + } + + size_t getElementCount() const override { return element_count.load(std::memory_order_relaxed); } + + double getLoadFactor() const override { return static_cast(element_count.load(std::memory_order_relaxed)) / size; } + + bool isCached() const override { return false; } + + std::shared_ptr clone() const override + { + return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, size); + } + + const IDictionarySource * getSource() const override { return source_ptr.get(); } + + const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } + + const DictionaryStructure & getStructure() const override { return dict_struct; } + + bool isInjective(const std::string & attribute_name) const override + { + return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective; + } + + bool hasHierarchy() const override { return hierarchical_attribute; } + + void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const override; + + void isInVectorVector( + const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; + void isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const override; + void isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; + + std::exception_ptr getLastException() const override; + + template + using ResultArrayType = std::conditional_t, DecimalPaddedPODArray, PaddedPODArray>; + +#define DECLARE(TYPE) \ + void get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const; + +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const PaddedPODArray & ids, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void + getString(const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) + const; + +#define DECLARE(TYPE) \ + void get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void getString(const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const; + + void has(const PaddedPODArray & ids, PaddedPODArray & out) const override; + + BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + + private: + template + using ContainerType = Value[]; + template + using ContainerPtrType = std::unique_ptr>; + + struct CellMetadata final + { + using time_point_t = std::chrono::system_clock::time_point; + using time_point_rep_t = time_point_t::rep; + using time_point_urep_t = std::make_unsigned_t; + + static constexpr UInt64 EXPIRES_AT_MASK = std::numeric_limits::max(); + static constexpr UInt64 IS_DEFAULT_MASK = ~EXPIRES_AT_MASK; + + UInt64 id; + /// Stores both expiration time and `is_default` flag in the most significant bit + time_point_urep_t data; + + /// Sets expiration time, resets `is_default` flag to false + time_point_t expiresAt() const { return ext::safe_bit_cast(data & EXPIRES_AT_MASK); } + void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast(t); } + + bool isDefault() const { return (data & IS_DEFAULT_MASK) == IS_DEFAULT_MASK; } + void setDefault() { data |= IS_DEFAULT_MASK; } + }; + + struct Attribute final + { + AttributeUnderlyingType type; + std::variant< + UInt8, + UInt16, + UInt32, + UInt64, + UInt128, + Int8, + Int16, + Int32, + Int64, + Decimal32, + Decimal64, + Decimal128, + Float32, + Float64, + String> + null_values; + std::variant< + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType> + arrays; + }; + + void createAttributes(); + + Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value); + + template + void getItemsNumberImpl( + Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const; + + template + void getItemsString(Attribute & attribute, const PaddedPODArray & ids, ColumnString * out, DefaultGetter && get_default) const; + + template + void update(const std::vector & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const; + + PaddedPODArray getCachedIds() const; + + bool isEmptyCell(const UInt64 idx) const; + + size_t getCellIdx(const Key id) const; + + void setDefaultAttributeValue(Attribute & attribute, const Key idx) const; + + void setAttributeValue(Attribute & attribute, const Key idx, const Field & value) const; + + Attribute & getAttribute(const std::string & attribute_name) const; + + struct FindResult + { + const size_t cell_idx; + const bool valid; + const bool outdated; + }; + + FindResult findCellIdx(const Key & id, const CellMetadata::time_point_t now) const; + + template + void isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; + + const std::string name; + const DictionaryStructure dict_struct; + mutable DictionarySourcePtr source_ptr; + const DictionaryLifetime dict_lifetime; + Logger * const log; + + mutable std::shared_mutex rw_lock; + + /// Actual size will be increased to match power of 2 + const size_t size; + + /// all bits to 1 mask (size - 1) (0b1000 - 1 = 0b111) + const size_t size_overlap_mask; + + /// Max tries to find cell, overlaped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3 + static constexpr size_t max_collision_length = 10; + + const size_t zero_cell_idx{getCellIdx(0)}; + std::map attribute_index_by_name; + mutable std::vector attributes; + mutable std::vector cells; + Attribute * hierarchical_attribute = nullptr; + std::unique_ptr string_arena; + + mutable std::exception_ptr last_exception; + mutable size_t error_count = 0; + mutable std::chrono::system_clock::time_point backoff_end_time; + + mutable pcg64 rnd_engine; + + mutable size_t bytes_allocated = 0; + mutable std::atomic element_count{0}; + mutable std::atomic hit_count{0}; + mutable std::atomic query_count{0}; + }; + +} diff --git a/dbms/src/Dictionaries/DirectDictionary.inc.h b/dbms/src/Dictionaries/DirectDictionary.inc.h new file mode 100644 index 00000000000..68010b0fe19 --- /dev/null +++ b/dbms/src/Dictionaries/DirectDictionary.inc.h @@ -0,0 +1,406 @@ +#include "CacheDictionary.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event DictCacheKeysRequested; + extern const Event DictCacheKeysRequestedMiss; + extern const Event DictCacheKeysRequestedFound; + extern const Event DictCacheKeysExpired; + extern const Event DictCacheKeysNotFound; + extern const Event DictCacheKeysHit; + extern const Event DictCacheRequestTimeNs; + extern const Event DictCacheRequests; + extern const Event DictCacheLockWriteNs; + extern const Event DictCacheLockReadNs; +} + +namespace CurrentMetrics +{ + extern const Metric DictCacheRequests; +} + +namespace DB +{ + namespace ErrorCodes + { + extern const int TYPE_MISMATCH; + } + + template + void CacheDictionary::getItemsNumberImpl( + Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const + { + /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } + std::unordered_map> outdated_ids; + auto & attribute_array = std::get>(attribute.arrays); + const auto rows = ext::size(ids); + + size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; + + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + /// fetch up-to-date values, decide which ones require update + for (const auto row : ext::range(0, rows)) + { + const auto id = ids[row]; + + /** cell should be updated if either: + * 1. ids do not match, + * 2. cell has expired, + * 3. explicit defaults were specified and cell was set default. */ + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + outdated_ids[id].push_back(row); + if (find_result.outdated) + ++cache_expired; + else + ++cache_not_found; + } + else + { + ++cache_hit; + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + out[row] = cell.isDefault() ? get_default(row) : static_cast(attribute_array[cell_idx]); + } + } + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); + ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); + ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); + + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); + + if (outdated_ids.empty()) + return; + + std::vector required_ids(outdated_ids.size()); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); + + /// request new values + update( + required_ids, + [&](const auto id, const auto cell_idx) + { + const auto attribute_value = attribute_array[cell_idx]; + + for (const size_t row : outdated_ids[id]) + out[row] = static_cast(attribute_value); + }, + [&](const auto id, const auto) + { + for (const size_t row : outdated_ids[id]) + out[row] = get_default(row); + }); + } + + template + void CacheDictionary::getItemsString( + Attribute & attribute, const PaddedPODArray & ids, ColumnString * out, DefaultGetter && get_default) const + { + const auto rows = ext::size(ids); + + /// save on some allocations + out->getOffsets().reserve(rows); + + auto & attribute_array = std::get>(attribute.arrays); + + auto found_outdated_values = false; + + /// perform optimistic version, fallback to pessimistic if failed + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + /// fetch up-to-date values, discard on fail + for (const auto row : ext::range(0, rows)) + { + const auto id = ids[row]; + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + found_outdated_values = true; + break; + } + else + { + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; + out->insertData(string_ref.data, string_ref.size); + } + } + } + + /// optimistic code completed successfully + if (!found_outdated_values) + { + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows, std::memory_order_release); + return; + } + + /// now onto the pessimistic one, discard possible partial results from the optimistic path + out->getChars().resize_assume_reserved(0); + out->getOffsets().resize_assume_reserved(0); + + /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } + std::unordered_map> outdated_ids; + /// we are going to store every string separately + std::unordered_map map; + + size_t total_length = 0; + size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + for (const auto row : ext::range(0, ids.size())) + { + const auto id = ids[row]; + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + outdated_ids[id].push_back(row); + if (find_result.outdated) + ++cache_expired; + else + ++cache_not_found; + } + else + { + ++cache_hit; + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; + + if (!cell.isDefault()) + map[id] = String{string_ref}; + + total_length += string_ref.size + 1; + } + } + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); + ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); + ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); + + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); + + /// request new values + if (!outdated_ids.empty()) + { + std::vector required_ids(outdated_ids.size()); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); + + update( + required_ids, + [&](const auto id, const auto cell_idx) + { + const auto attribute_value = attribute_array[cell_idx]; + + map[id] = String{attribute_value}; + total_length += (attribute_value.size + 1) * outdated_ids[id].size(); + }, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + total_length += get_default(row).size + 1; + }); + } + + out->getChars().reserve(total_length); + + for (const auto row : ext::range(0, ext::size(ids))) + { + const auto id = ids[row]; + const auto it = map.find(id); + + const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row); + out->insertData(string_ref.data, string_ref.size); + } + } + + template + void CacheDictionary::update( + const std::vector & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const + { + CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests}; + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size()); + + std::unordered_map remaining_ids{requested_ids.size()}; + for (const auto id : requested_ids) + remaining_ids.insert({id, 0}); + + const auto now = std::chrono::system_clock::now(); + + const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; + + if (now > backoff_end_time) + { + try + { + if (error_count) + { + /// Recover after error: we have to clone the source here because + /// it could keep connections which should be reset after error. + source_ptr = source_ptr->clone(); + } + + Stopwatch watch; + auto stream = source_ptr->loadIds(requested_ids); + stream->readPrefix(); + + while (const auto block = stream->read()) + { + const auto id_column = typeid_cast(block.safeGetByPosition(0).column.get()); + if (!id_column) + throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH}; + + const auto & ids = id_column->getData(); + + /// cache column pointers + const auto column_ptrs = ext::map( + ext::range(0, attributes.size()), [&block](size_t i) { return block.safeGetByPosition(i + 1).column.get(); }); + + for (const auto i : ext::range(0, ids.size())) + { + const auto id = ids[i]; + + const auto find_result = findCellIdx(id, now); + const auto & cell_idx = find_result.cell_idx; + + auto & cell = cells[cell_idx]; + + for (const auto attribute_idx : ext::range(0, attributes.size())) + { + const auto & attribute_column = *column_ptrs[attribute_idx]; + auto & attribute = attributes[attribute_idx]; + + setAttributeValue(attribute, cell_idx, attribute_column[i]); + } + + /// if cell id is zero and zero does not map to this cell, then the cell is unused + if (cell.id == 0 && cell_idx != zero_cell_idx) + element_count.fetch_add(1, std::memory_order_relaxed); + + cell.id = id; + if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) + { + std::uniform_int_distribution distribution{dict_lifetime.min_sec, dict_lifetime.max_sec}; + cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)}); + } + else + cell.setExpiresAt(std::chrono::time_point::max()); + + /// inform caller + on_cell_updated(id, cell_idx); + /// mark corresponding id as found + remaining_ids[id] = 1; + } + } + + stream->readSuffix(); + + error_count = 0; + last_exception = std::exception_ptr{}; + backoff_end_time = std::chrono::system_clock::time_point{}; + + ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed()); + } + catch (...) + { + ++error_count; + last_exception = std::current_exception(); + backoff_end_time = now + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count)); + + tryLogException(last_exception, log, "Could not update cache dictionary '" + getName() + + "', next update is scheduled at " + DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(backoff_end_time))); + } + } + + size_t not_found_num = 0, found_num = 0; + + /// Check which ids have not been found and require setting null_value + for (const auto & id_found_pair : remaining_ids) + { + if (id_found_pair.second) + { + ++found_num; + continue; + } + ++not_found_num; + + const auto id = id_found_pair.first; + + const auto find_result = findCellIdx(id, now); + const auto & cell_idx = find_result.cell_idx; + auto & cell = cells[cell_idx]; + + if (error_count) + { + if (find_result.outdated) + { + /// We have expired data for that `id` so we can continue using it. + bool was_default = cell.isDefault(); + cell.setExpiresAt(backoff_end_time); + if (was_default) + cell.setDefault(); + if (was_default) + on_id_not_found(id, cell_idx); + else + on_cell_updated(id, cell_idx); + continue; + } + /// We don't have expired data for that `id` so all we can do is to rethrow `last_exception`. + std::rethrow_exception(last_exception); + } + + /// Check if cell had not been occupied before and increment element counter if it hadn't + if (cell.id == 0 && cell_idx != zero_cell_idx) + element_count.fetch_add(1, std::memory_order_relaxed); + + cell.id = id; + + if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) + { + std::uniform_int_distribution distribution{dict_lifetime.min_sec, dict_lifetime.max_sec}; + cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)}); + } + else + cell.setExpiresAt(std::chrono::time_point::max()); + + /// Set null_value for each attribute + cell.setDefault(); + for (auto & attribute : attributes) + setDefaultAttributeValue(attribute, cell_idx); + + /// inform caller that the cell has not been found + on_id_not_found(id, cell_idx); + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num); + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num); + ProfileEvents::increment(ProfileEvents::DictCacheRequests); + } + +} diff --git a/dbms/src/Dictionaries/registerDictionaries.cpp b/dbms/src/Dictionaries/registerDictionaries.cpp index ee320d7177b..44cb1f2858f 100644 --- a/dbms/src/Dictionaries/registerDictionaries.cpp +++ b/dbms/src/Dictionaries/registerDictionaries.cpp @@ -21,6 +21,7 @@ void registerDictionaryTrie(DictionaryFactory & factory); void registerDictionaryFlat(DictionaryFactory & factory); void registerDictionaryHashed(DictionaryFactory & factory); void registerDictionaryCache(DictionaryFactory & factory); +void registerDictionaryDirect(DictionaryFactory & factory); void registerDictionaries() @@ -48,6 +49,8 @@ void registerDictionaries() registerDictionaryFlat(factory); registerDictionaryHashed(factory); registerDictionaryCache(factory); + /// added new dictionary type (i bet i wont forget to remove this comment) + registerDictionaryDirect(factory); } } From 7f3b5efebe12d53f70e48ba4ecd990c2b8c8801e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 30 Jan 2020 15:51:47 +0300 Subject: [PATCH 035/743] SHOW CREATE for DatabaseMemory --- dbms/src/Databases/DatabaseMemory.cpp | 27 +++++++++++++--- dbms/src/Databases/DatabaseMemory.h | 3 ++ dbms/src/Databases/DatabasesCommon.cpp | 32 ++++++++++++------- dbms/src/Databases/DatabasesCommon.h | 3 ++ .../01069_database_memory.reference | 1 + .../0_stateless/01069_database_memory.sql | 3 ++ 6 files changed, 53 insertions(+), 16 deletions(-) diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index 765d80c9fee..aa3446a076e 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_TABLE; +} + DatabaseMemory::DatabaseMemory(const String & name_) : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")") , data_path("data/" + escapeForFileName(database_name) + "/") @@ -18,16 +23,19 @@ void DatabaseMemory::createTable( const Context & /*context*/, const String & table_name, const StoragePtr & table, - const ASTPtr & /*query*/) + const ASTPtr & query) { - attachTable(table_name, table); + std::lock_guard lock{mutex}; + attachTableUnlocked(table_name, table); + create_queries.emplace(table_name, query); } void DatabaseMemory::dropTable( const Context & /*context*/, const String & table_name) { - auto table = detachTable(table_name); + std::lock_guard lock{mutex}; + auto table = detachTableUnlocked(table_name); try { table->drop(); @@ -37,9 +45,11 @@ void DatabaseMemory::dropTable( } catch (...) { - attachTable(table_name, table); + attachTableUnlocked(table_name, table); + throw; } table->is_dropped = true; + create_queries.erase(table_name); } ASTPtr DatabaseMemory::getCreateDatabaseQuery(const Context & /*context*/) const @@ -51,4 +61,13 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery(const Context & /*context*/) const return create_query; } +ASTPtr DatabaseMemory::getCreateTableQueryImpl(const Context &, const String & table_name, bool throw_on_error) const +{ + std::lock_guard lock{mutex}; + auto it = create_queries.find(table_name); + if (it == create_queries.end() && throw_on_error) + throw Exception("No table " + table_name + " in database " + database_name, ErrorCodes::UNKNOWN_TABLE); + return it->second; +} + } diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 27abf6e87be..2d8ec6c21f9 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -33,6 +33,7 @@ public: const Context & context, const String & table_name) override; + ASTPtr getCreateTableQueryImpl(const Context & /*context*/, const String & name, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery(const Context & /*context*/) const override; /// DatabaseMemory allows to create tables, which store data on disk. @@ -44,6 +45,8 @@ public: private: String data_path; + using NameToASTCreate = std::unordered_map; + NameToASTCreate create_queries; }; } diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 8179d5c312e..30fd1f4c34f 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -67,25 +67,33 @@ bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name) { - StoragePtr res; - { - std::lock_guard lock(mutex); - if (dictionaries.count(table_name)) - throw Exception("Cannot detach dictionary " + database_name + "." + table_name + " as table, use DETACH DICTIONARY query.", ErrorCodes::UNKNOWN_TABLE); + std::lock_guard lock(mutex); + return detachTableUnlocked(table_name); +} - auto it = tables.find(table_name); - if (it == tables.end()) - throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - res = it->second; - tables.erase(it); - } +StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_name) +{ + StoragePtr res; + if (dictionaries.count(table_name)) + throw Exception("Cannot detach dictionary " + database_name + "." + table_name + " as table, use DETACH DICTIONARY query.", ErrorCodes::UNKNOWN_TABLE); + + auto it = tables.find(table_name); + if (it == tables.end()) + throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + res = it->second; + tables.erase(it); return res; } -void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table, const String &) +void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) { std::lock_guard lock(mutex); + attachTableUnlocked(table_name, table, relative_table_path); +} + +void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table, const String &) +{ if (!tables.emplace(table_name, table).second) throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index bca09eb8dcb..5c6bb77abee 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -47,6 +47,9 @@ protected: Poco::Logger * log; DatabaseWithOwnTablesBase(const String & name_, const String & logger); + + void attachTableUnlocked(const String & table_name, const StoragePtr & table, const String & relative_table_path = {}); + StoragePtr detachTableUnlocked(const String & table_name); }; } diff --git a/dbms/tests/queries/0_stateless/01069_database_memory.reference b/dbms/tests/queries/0_stateless/01069_database_memory.reference index b2fa2bd204c..35175dd4b62 100644 --- a/dbms/tests/queries/0_stateless/01069_database_memory.reference +++ b/dbms/tests/queries/0_stateless/01069_database_memory.reference @@ -5,3 +5,4 @@ CREATE DATABASE memory_01069 ENGINE = Memory() 4 3 4 +CREATE TABLE memory_01069.file (`n` UInt8) ENGINE = File(CSV) diff --git a/dbms/tests/queries/0_stateless/01069_database_memory.sql b/dbms/tests/queries/0_stateless/01069_database_memory.sql index 645790e3f5e..ed006a5ce04 100644 --- a/dbms/tests/queries/0_stateless/01069_database_memory.sql +++ b/dbms/tests/queries/0_stateless/01069_database_memory.sql @@ -15,4 +15,7 @@ DROP TABLE memory_01069.mt; SELECT * FROM memory_01069.mt ORDER BY n; -- { serverError 60 } SELECT * FROM memory_01069.file ORDER BY n; +SHOW CREATE TABLE memory_01069.mt; -- { serverError 60 } +SHOW CREATE TABLE memory_01069.file; + DROP DATABASE memory_01069; From 68f63c4b20ce8b8586c009247d4f260a45a07da2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 30 Jan 2020 22:00:51 +0300 Subject: [PATCH 036/743] store temporary tables in DatabaseMemory --- dbms/src/Databases/DatabaseMemory.cpp | 2 +- dbms/src/Databases/DatabaseMemory.h | 3 +- dbms/src/Interpreters/Context.cpp | 124 +++++++++++++----- dbms/src/Interpreters/Context.h | 9 +- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../src/Interpreters/InterpreterDropQuery.cpp | 4 +- .../Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- 8 files changed, 108 insertions(+), 40 deletions(-) diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index aa3446a076e..e51dfbb7ea9 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -66,7 +66,7 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const Context &, const String & t std::lock_guard lock{mutex}; auto it = create_queries.find(table_name); if (it == create_queries.end() && throw_on_error) - throw Exception("No table " + table_name + " in database " + database_name, ErrorCodes::UNKNOWN_TABLE); + throw Exception("There is no metadata of table " + table_name + " in database " + database_name, ErrorCodes::UNKNOWN_TABLE); return it->second; } diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 2d8ec6c21f9..246ea551107 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -19,7 +19,8 @@ namespace DB class DatabaseMemory : public DatabaseWithOwnTablesBase { public: - DatabaseMemory(const String & name_); + //FIXME default name + DatabaseMemory(const String & name_ = "_temporary_and_external_tables"); String getEngineName() const override { return "Memory"; } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index c4d64c75053..61bb7cddfc7 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -57,6 +57,8 @@ #include #include +#include + namespace ProfileEvents { extern const Event ContextLock; @@ -99,6 +101,57 @@ namespace ErrorCodes extern const int UNKNOWN_POLICY; } +struct TemporaryTableHolder : boost::noncopyable +{ + static constexpr const char * database_name = "_temporary_and_external_tables"; + + TemporaryTableHolder(const Context & context_, + DatabaseMemory & external_tables_, + const StoragePtr & table, + const ASTPtr & query = {}) + : context(context_), external_tables(external_tables_) + { + if (query) + { + ASTCreateQuery & create = dynamic_cast(*query); + if (create.uuid == UUIDHelpers::Nil) + create.uuid = UUIDHelpers::generateV4(); + id = create.uuid; + } + else + id = UUIDHelpers::generateV4(); + external_tables.createTable(context, "_data_" + toString(id), table, query); + } + + TemporaryTableHolder(TemporaryTableHolder && other) + : context(other.context), external_tables(other.external_tables), id(other.id) + { + other.id = UUIDHelpers::Nil; + } + + ~TemporaryTableHolder() + { + external_tables.dropTable(context, "_data_" + toString(id)); + } + + StorageID getGlobalTableID() const + { + return StorageID{database_name, "_data_" + toString(id), id}; + } + + StoragePtr getTable() const + { + auto table = external_tables.tryGetTable(context, "_data_" + toString(id)); + if (!table) + throw Exception("Temporary table " + getGlobalTableID().getNameForLogs() + " not found", ErrorCodes::LOGICAL_ERROR); + return table; + } + + const Context & context; + DatabaseMemory & external_tables; + UUID id; +}; + /** Set of known objects (environment), that could be used in query. * Shared (global) part. Order of members (especially, order of destruction) is very important. @@ -134,6 +187,8 @@ struct ContextShared mutable VolumePtr tmp_volume; /// Volume for the the temporary files that occur when processing the request. Databases databases; /// List of databases and tables in them. + DatabaseMemory temporary_and_external_tables; + mutable std::optional embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization. mutable std::optional external_dictionaries_loader; mutable std::optional external_models_loader; @@ -808,18 +863,10 @@ Dependencies Context::getDependencies(const StorageID & from) const { auto lock = getLock(); - String db = resolveDatabase(from.database_name, current_database); + StorageID resolved = resolveStorageIDUnlocked(from); + resolved.uuid = UUIDHelpers::Nil; // FIXME see addDependencyUnsafe() - if (from.database_name.empty() && tryGetExternalTable(from.table_name)) - { - /// Table is temporary. Access granted. - } - else - { - checkDatabaseAccessRightsImpl(db); - } - - ViewDependencies::const_iterator iter = shared->view_dependencies.find(StorageID(db, from.table_name)); + ViewDependencies::const_iterator iter = shared->view_dependencies.find(resolved); if (iter == shared->view_dependencies.end()) return {}; @@ -859,7 +906,7 @@ bool Context::isDatabaseExist(const String & database_name) const bool Context::isExternalTableExist(const String & table_name) const { - return external_tables.end() != external_tables.find(table_name); + return external_tables_mapping.count(table_name); } @@ -922,8 +969,8 @@ Tables Context::getExternalTables() const auto lock = getLock(); Tables res; - for (auto & table : external_tables) - res[table.first] = table.second.first; + for (auto & table : external_tables_mapping) + res[table.first] = table.second->getTable(); if (session_context && session_context != this) { @@ -941,11 +988,11 @@ Tables Context::getExternalTables() const StoragePtr Context::tryGetExternalTable(const String & table_name) const { - TableAndCreateASTs::const_iterator jt = external_tables.find(table_name); - if (external_tables.end() == jt) + auto it = external_tables_mapping.find(table_name); + if (external_tables_mapping.end() == it) return StoragePtr(); - return jt->second.first; + return it->second->getTable(); } StoragePtr Context::getTable(const String & database_name, const String & table_name) const @@ -1016,10 +1063,11 @@ StoragePtr Context::getTableImpl(const StorageID & table_id, std::optional(*this, shared->temporary_and_external_tables, storage, ast)); } @@ -1035,16 +1083,9 @@ bool Context::hasScalar(const String & name) const } -StoragePtr Context::tryRemoveExternalTable(const String & table_name) +bool Context::removeExternalTable(const String & table_name) { - TableAndCreateASTs::const_iterator it = external_tables.find(table_name); - - if (external_tables.end() == it) - return StoragePtr(); - - auto storage = it->second.first; - external_tables.erase(it); - return storage; + return external_tables_mapping.erase(table_name); } @@ -1131,11 +1172,11 @@ DatabasePtr Context::detachDatabase(const String & database_name) ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const { - TableAndCreateASTs::const_iterator jt = external_tables.find(table_name); - if (external_tables.end() == jt) + auto it = external_tables_mapping.find(table_name); + if (external_tables_mapping.end() == it) throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist", ErrorCodes::UNKNOWN_TABLE); - return jt->second.second; + return shared->temporary_and_external_tables.getCreateTableQuery(*this, it->second->getGlobalTableID().table_name); } Settings Context::getSettings() const @@ -2199,6 +2240,27 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } +StorageID Context::resolveStorageIDUnlocked(StorageID storage_id) const +{ + if (storage_id.uuid != UUIDHelpers::Nil) + { + //TODO maybe update table and db name? + //TODO add flag `resolved` to StorageID and check access rights if it was not previously resolved + return storage_id; + } + if (storage_id.database_name.empty()) + { + auto it = external_tables_mapping.find(storage_id.getTableName()); + if (it != external_tables_mapping.end()) + return it->second->getGlobalTableID(); /// Do not check access rights for session-local table + if (current_database.empty()) + throw Exception("Default database is not selected", ErrorCodes::UNKNOWN_DATABASE); + storage_id.database_name = current_database; + } + checkDatabaseAccessRightsImpl(storage_id.database_name); + return storage_id; +} + SessionCleaner::~SessionCleaner() { diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 9541fcc8bd6..119853cae23 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -128,6 +128,8 @@ struct IHostContext using IHostContextPtr = std::shared_ptr; +struct TemporaryTableHolder; + /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -161,7 +163,9 @@ private: String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification. /// Thus, used in HTTP interface. If not specified - then some globally default format is used. // TODO maybe replace with DatabaseMemory? - TableAndCreateASTs external_tables; /// Temporary tables. + //TableAndCreateASTs external_tables; /// Temporary tables. + using TemporaryTablesMapping = std::map>; + TemporaryTablesMapping external_tables_mapping; Scalars scalars; StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id. @@ -298,7 +302,8 @@ public: void addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast = {}); void addScalar(const String & name, const Block & block); bool hasScalar(const String & name) const; - StoragePtr tryRemoveExternalTable(const String & table_name); + bool removeExternalTable(const String & table_name); + StorageID resolveStorageIDUnlocked(StorageID storage_id) const; StoragePtr executeTableFunction(const ASTPtr & table_expression); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index e7c54f5a6a4..3a1cf146e67 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -665,7 +665,7 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } } - else if (context.tryGetExternalTable(table_name) && create.if_not_exists) + else if (context.isExternalTableExist(table_name) && create.if_not_exists) return false; StoragePtr res; diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 09e87a6b7d8..c3e48e79994 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -179,7 +179,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr } else if (kind == ASTDropQuery::Kind::Drop) { - context_handle.tryRemoveExternalTable(table_name); + context_handle.removeExternalTable(table_name); table->shutdown(); /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); @@ -278,7 +278,7 @@ void InterpreterDropQuery::checkAccess(const ASTDropQuery & drop) bool allow_ddl = settings.allow_ddl; /// It's allowed to drop temporary tables. - if ((!readonly && allow_ddl) || (drop.database.empty() && context.tryGetExternalTable(drop.table) && readonly >= 2)) + if ((!readonly && allow_ddl) || (drop.database.empty() && context.isExternalTableExist(drop.table) && readonly >= 2)) return; if (readonly) diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 787b5d3717b..9d83b6d1e2a 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -209,7 +209,7 @@ void InterpreterInsertQuery::checkAccess(const ASTInsertQuery & query) const Settings & settings = context.getSettingsRef(); auto readonly = settings.readonly; - if (!readonly || (query.database.empty() && context.tryGetExternalTable(query.table) && readonly >= 2)) + if (!readonly || (query.database.empty() && context.isExternalTableExist(query.table) && readonly >= 2)) { return; } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 70c3c8fc8f5..1334deaac38 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -338,7 +338,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Save the new temporary tables in the query context for (const auto & it : query_analyzer->getExternalTables()) - if (!context->tryGetExternalTable(it.first)) + if (!context->isExternalTableExist(it.first)) context->addExternalTable(it.first, it.second); } From 5438cef23a140f91dde8c9b837270d2c94a7d79d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 18 Mar 2020 02:51:35 +0300 Subject: [PATCH 037/743] fixes --- dbms/programs/server/Server.cpp | 2 +- dbms/src/Databases/DatabaseAtomic.cpp | 30 +++++++++++-------- dbms/src/Databases/DatabaseMemory.h | 3 +- .../Databases/DatabaseWithDictionaries.cpp | 2 -- dbms/src/Databases/DatabasesCommon.cpp | 12 +++++--- dbms/src/Databases/DatabasesCommon.h | 1 - .../Formats/ParsedTemplateFormatString.cpp | 20 ++----------- dbms/src/Interpreters/Context.cpp | 3 -- dbms/src/Interpreters/Context.h | 3 -- dbms/src/Interpreters/DatabaseCatalog.cpp | 6 +--- .../Interpreters/InterpreterCreateQuery.cpp | 14 +++------ dbms/src/Interpreters/loadMetadata.cpp | 2 +- dbms/src/Parsers/ASTCreateQuery.cpp | 2 +- dbms/src/Storages/StorageDistributed.h | 1 - dbms/src/TableFunctions/TableFunctionFile.cpp | 3 +- .../00751_default_databasename_for_view.sql | 2 +- .../00041_aggregating_materialized_view.sql | 2 +- 17 files changed, 39 insertions(+), 69 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 7fa0a080597..731a71a60af 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -558,7 +558,7 @@ int Server::main(const std::vector & /*args*/) /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) attachSystemTablesServer(*DatabaseCatalog::instance().getSystemDatabase(), has_zookeeper); /// Then, load remaining databases - loadMetadata(*global_context); + loadMetadata(*global_context, default_database); DatabaseCatalog::instance().loadDatabases(); } catch (...) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index 3bb85400263..906e6903966 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -84,18 +84,21 @@ void DatabaseAtomic::drop(const Context &) ", which is used by " + std::to_string(it->table.use_count() - 1) + " queries. " "Client should retry later.", ErrorCodes::DATABASE_NOT_EMPTY); - for (auto & table : tables_to_drop) - { - try - { - dropTableFinally(table); - } - catch (...) - { - tryLogCurrentException(log, "Cannot drop table. Metadata " + table.data_path + " will be removed forcefully. " - "Garbage may be left in /store directory and ZooKeeper."); - } - } + //FIXME maybe make `tables_to_drop` global for all Atomic databases? + + //for (auto & table : tables_to_drop) + //{ + // try + // { + // /// IStorage::drop() may use DatabaseCatalog, so databases mutex will be acquired (possible deadlock here) + // dropTableFinally(table); + // } + // catch (...) + // { + // tryLogCurrentException(log, "Cannot drop table. Metadata " + table.data_path + " will be removed forcefully. " + // "Garbage may be left in /store directory and ZooKeeper."); + // } + //} Poco::File(getMetadataPath()).remove(true); } @@ -172,9 +175,10 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n table->renameInMemory(to_database.getDatabaseName(), to_table_name); /// NOTE Non-atomic. - to_database.attachTable(to_table_name, table, getTableDataPath(table_name)); + auto path = getTableDataPath(table_name); detachTable(table_name); Poco::File(getObjectMetadataPath(table_name)).renameTo(to_database.getObjectMetadataPath(to_table_name)); + to_database.attachTable(to_table_name, table, path); } void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 246ea551107..2d8ec6c21f9 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -19,8 +19,7 @@ namespace DB class DatabaseMemory : public DatabaseWithOwnTablesBase { public: - //FIXME default name - DatabaseMemory(const String & name_ = "_temporary_and_external_tables"); + DatabaseMemory(const String & name_); String getEngineName() const override { return "Memory"; } diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index 28f3869ac1e..d4d95184490 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes } - void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const Context & context) { String full_name = getDatabaseName() + "." + dictionary_name; @@ -60,7 +59,6 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S { const auto & settings = context.getSettingsRef(); - /** The code is based on the assumption that all threads share the same order of operations: * - create the .sql.tmp file; * - add the dictionary to ExternalDictionariesLoader; diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 64f7dc2ac24..8947f0520cf 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -83,8 +83,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n auto table_id = res->getStorageID(); if (table_id.hasUUID()) { - /// For now it's the only database, which contains storages with UUID - assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE); + assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic"); DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid); } @@ -104,8 +103,7 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c auto table_id = table->getStorageID(); if (table_id.hasUUID()) { - /// For now it's the only database, which contains storages with UUID - assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE); + assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic"); DatabaseCatalog::instance().addUUIDMapping(table_id.uuid, shared_from_this(), table); } } @@ -123,7 +121,13 @@ void DatabaseWithOwnTablesBase::shutdown() for (const auto & kv : tables_snapshot) { + auto table_id = kv.second->getStorageID(); kv.second->shutdown(); + if (table_id.hasUUID()) + { + assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic"); + DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid); + } } std::lock_guard lock(mutex); diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 5c6bb77abee..3a69a043660 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -15,7 +15,6 @@ namespace DB class Context; - /// A base class for databases that manage their own list of tables. class DatabaseWithOwnTablesBase : public IDatabase { diff --git a/dbms/src/Formats/ParsedTemplateFormatString.cpp b/dbms/src/Formats/ParsedTemplateFormatString.cpp index 89c0a589aef..a01a8adcf26 100644 --- a/dbms/src/Formats/ParsedTemplateFormatString.cpp +++ b/dbms/src/Formats/ParsedTemplateFormatString.cpp @@ -77,14 +77,7 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum case Column: column_names.emplace_back(); - try - { - pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back()); - } - catch (const DB::Exception & e) - { - throwInvalidFormat(e.message(), columnsCount()); - } + pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back()); if (*pos == ':') state = Format; @@ -107,16 +100,7 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum errno = 0; column_idx = strtoull(column_names.back().c_str(), &col_idx_end, 10); if (col_idx_end != column_names.back().c_str() + column_names.back().size() || errno) - { - try - { - column_idx = idx_by_name(column_names.back()); - } - catch (const DB::Exception & e) - { - throwInvalidFormat(e.message(), columnsCount()); - } - } + column_idx = idx_by_name(column_names.back()); } format_idx_to_column_idx.emplace_back(column_idx); break; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 3591ac5607f..f081086af4e 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -58,8 +58,6 @@ #include #include -#include - namespace ProfileEvents { extern const Event ContextLock; @@ -930,7 +928,6 @@ StoragePtr Context::getViewSource() return view_source; } - Settings Context::getSettings() const { return settings; diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 690518d919e..5d8351ed598 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -88,7 +88,6 @@ class ShellCommand; class ICompressionCodec; class AccessControlManager; class SettingsConstraints; -struct StorageID; class RemoteHostFilter; struct StorageID; class IDisk; @@ -131,8 +130,6 @@ struct IHostContext using IHostContextPtr = std::shared_ptr; -struct TemporaryTableHolder; - /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). diff --git a/dbms/src/Interpreters/DatabaseCatalog.cpp b/dbms/src/Interpreters/DatabaseCatalog.cpp index ce303add03e..bf160cfc056 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.cpp +++ b/dbms/src/Interpreters/DatabaseCatalog.cpp @@ -126,11 +126,7 @@ void DatabaseCatalog::shutdown() std::lock_guard lock(databases_mutex); - for (auto & elem : uuid_map) - { - std::lock_guard map_lock(elem.mutex); - elem.map.clear(); - } + assert(std::find_if_not(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { return elem.map.empty(); }) == uuid_map.end()); databases.clear(); view_dependencies.clear(); } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 2b98f7907ac..56bcd6b35c2 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -105,12 +105,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) engine->name = old_style_database ? "Ordinary" : "Atomic"; storage->set(storage->engine, engine); create.set(create.storage, storage); - - if (database_name == "datasets") - { - //FIXME it's just to run stateful and stress tests without updating docker images - engine->name = "Ordinary"; - } } else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) { @@ -559,7 +553,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.temporary && create.database.empty()) create.database = current_database; - if (!create.to_table_id.empty() && create.to_table_id.database_name.empty()) + if (create.to_table_id && create.to_table_id.database_name.empty()) create.to_table_id.database_name = current_database; if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view)) @@ -587,7 +581,7 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, String data_path; DatabasePtr database; - const String & table_name = create.table; + const String table_name = create.table; bool need_add_to_database = !create.temporary; if (need_add_to_database) { @@ -642,7 +636,7 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, } else { - if (context.tryResolveStorageID({"", table_name}, Context::ResolveExternal) && create.if_not_exists) + if (create.if_not_exists && context.tryResolveStorageID({"", table_name}, Context::ResolveExternal)) return false; auto temporary_table = TemporaryTableHolder(context, properties.columns, query_ptr); @@ -803,7 +797,7 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const } if (create.to_table_id) - required_access.emplace_back(AccessType::INSERT, create.to_table_id.getDatabaseName(), create.to_table_id.getTableName()); + required_access.emplace_back(AccessType::INSERT, create.to_table_id.database_name, create.to_table_id.table_name); return required_access; } diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 563a2e53709..67ca6783eef 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -54,7 +54,7 @@ static void loadDatabase( if (Poco::File(database_metadata_file).exists()) { - /// There are .sql file with database creation statement. + /// There is .sql file with database creation statement. ReadBufferFromFile in(database_metadata_file, 1024); readStringUntilEOF(database_attach_query, in); } diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index 9ff196d2133..d5942753f78 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -258,7 +258,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); as_table_function->formatImpl(settings, state, frame); } - if (!to_table_id.empty()) + if (to_table_id) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "") diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 8abdfc5f509..34e6e66ef35 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -82,7 +82,6 @@ public: /// Removes temporary data in local filesystem. void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; void renameOnDisk(const String & new_path_to_table_data); diff --git a/dbms/src/TableFunctions/TableFunctionFile.cpp b/dbms/src/TableFunctions/TableFunctionFile.cpp index b8119b34d70..c27999e6199 100644 --- a/dbms/src/TableFunctions/TableFunctionFile.cpp +++ b/dbms/src/TableFunctions/TableFunctionFile.cpp @@ -10,8 +10,7 @@ namespace DB StoragePtr TableFunctionFile::getStorage( const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const { - StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, compression_method, - columns, ConstraintsDescription{}, global_context}; + StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, compression_method, columns, ConstraintsDescription{}, global_context}; return StorageFile::create(source, global_context.getUserFilesPath(), args); } diff --git a/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.sql b/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.sql index 0f962b31196..5fa43555b2b 100644 --- a/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.sql +++ b/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.sql @@ -1,4 +1,4 @@ -CREATE DATABASE test_00751; +CREATE DATABASE IF NOT EXISTS test_00751; USE test_00751; DROP TABLE IF EXISTS t_00751; diff --git a/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql b/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql index b91b0a3a2c2..0e59d2a88a9 100644 --- a/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql +++ b/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql @@ -49,7 +49,7 @@ ORDER BY StartDate; SELECT StartDate, - sum(Sign) AS Visits, + sum(Sign) AS Visits, uniq(UserID) AS Users FROM test.visits WHERE CounterID = 942285 From 3d058c206059a5959f99ded1baee0d9b746d38b5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 18 Mar 2020 20:38:52 +0300 Subject: [PATCH 038/743] fixes --- dbms/src/Databases/DatabaseAtomic.cpp | 4 ++-- dbms/src/Databases/DatabaseOnDisk.cpp | 5 ++++- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 9 ++++----- dbms/src/Storages/StorageMaterializedView.cpp | 16 +++++++++------- .../integration/test_distributed_format/test.py | 10 +++++----- .../test_merge_tree_s3/configs/users.xml | 1 + .../configs/compact_parts.xml | 2 +- .../configs/users.d/not_optimize_count.xml | 3 ++- .../configs_old/users.d/not_optimize_count.xml | 7 +++++++ .../integration/test_polymorphic_parts/test.py | 2 +- dbms/tests/integration/test_row_policy/test.py | 4 ++-- .../0_stateless/01053_drop_database_mat_view.sql | 2 +- 12 files changed, 39 insertions(+), 26 deletions(-) create mode 100644 dbms/tests/integration/test_polymorphic_parts/configs_old/users.d/not_optimize_count.xml diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index 906e6903966..ab46ce52c91 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -139,10 +139,10 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam { LOG_INFO(log, "Mark table " + table->getStorageID().getNameForLogs() + " to drop."); /// Context:getPath acquires lock - auto data_path = context.getPath() + table_data_path_relative; + auto table_data_path = context.getPath() + table_data_path_relative; std::lock_guard lock(tables_to_drop_mutex); time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - tables_to_drop.push_back({table, data_path, current_time}); + tables_to_drop.push_back({table, table_data_path, current_time}); } } catch (...) diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 184f283b5e9..dabbac81f3c 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -246,7 +246,9 @@ void DatabaseOnDisk::renameTable( TableStructureWriteLockHolder table_lock; String table_metadata_path; ASTPtr attach_query; - StoragePtr table = detachTable(table_name); + /// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case. + StoragePtr table = tryGetTable(context, table_name); + detachTable(table_name); try { table_lock = table->lockExclusively(context.getCurrentQueryId()); @@ -451,6 +453,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName(); if (Poco::Path(table_name).makeFile().getExtension() == "sql") table_name = Poco::Path(table_name).makeFile().getBaseName(); + table_name = unescapeForFileName(table_name); if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER) LOG_WARNING(log, "File " << metadata_file_path << " contains both UUID and table name. " diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 56bcd6b35c2..2a4f1e06998 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -603,16 +603,11 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, if (!create.attach && create.uuid == UUIDHelpers::Nil && database->getEngineName() == "Atomic") create.uuid = UUIDHelpers::generateV4(); - data_path = database->getTableDataPath(create); - /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. */ guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name); - if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists()) - throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS); - /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. if (database->isTableExist(context, table_name)) { @@ -633,6 +628,10 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, else throw Exception("Table " + create.database + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } + + data_path = database->getTableDataPath(create); + if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists()) + throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS); } else { diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index d135e9f8829..d635fc953ad 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -34,9 +34,11 @@ namespace ErrorCodes extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW; } -static inline String generateInnerTableName(const String & table_name) +static inline String generateInnerTableName(const StorageID & view_id) { - return ".inner." + table_name; + if (view_id.hasUUID()) + return ".inner." + toString(view_id.uuid); + return ".inner." + view_id.getTableName(); } static StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, const Context & context, bool add_default_db = true) @@ -128,14 +130,14 @@ StorageMaterializedView::StorageMaterializedView( else if (attach_) { /// If there is an ATTACH request, then the internal table must already be created. - target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID().table_name)); + target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID())); } else { /// We will create a query to create an internal table. auto manual_create_query = std::make_shared(); manual_create_query->database = getStorageID().database_name; - manual_create_query->table = generateInnerTableName(getStorageID().table_name); + manual_create_query->table = generateInnerTableName(getStorageID()); auto new_columns_list = std::make_shared(); new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr()); @@ -327,9 +329,10 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name) { - if (has_inner_table && tryGetTargetTable()) + auto old_table_id = getStorageID(); + if (has_inner_table && tryGetTargetTable() && !old_table_id.hasUUID()) { - auto new_target_table_name = generateInnerTableName(new_table_name); + auto new_target_table_name = generateInnerTableName({new_database_name, new_table_name}); auto rename = std::make_shared(); ASTRenameQuery::Table from; @@ -349,7 +352,6 @@ void StorageMaterializedView::renameInMemory(const String & new_database_name, c target_table_id.table_name = new_target_table_name; } - auto old_table_id = getStorageID(); IStorage::renameInMemory(new_database_name, new_table_name); // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated DatabaseCatalog::instance().updateDependency(select_table_id, old_table_id, select_table_id, getStorageID()); diff --git a/dbms/tests/integration/test_distributed_format/test.py b/dbms/tests/integration/test_distributed_format/test.py index a1cc17f32ae..211127f624a 100644 --- a/dbms/tests/integration/test_distributed_format/test.py +++ b/dbms/tests/integration/test_distributed_format/test.py @@ -61,18 +61,18 @@ def test_two_files(started_cluster): def test_single_file_old(started_cluster): - node.query("create table distr_3 (x UInt64, s String) engine = Distributed('test_cluster', database, table)") - node.query("insert into distr_3 values (1, 'a'), (2, 'bb'), (3, 'ccc')") + node.query("create table test.distr_3 (x UInt64, s String) engine = Distributed('test_cluster', database, table)") + node.query("insert into test.distr_3 values (1, 'a'), (2, 'bb'), (3, 'ccc')") - query = "select * from file('/var/lib/clickhouse/data/default/distr_3/default@not_existing:9000/1.bin', 'Distributed')" + query = "select * from file('/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin', 'Distributed')" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) assert out == '1\ta\n2\tbb\n3\tccc\n' - query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/default/distr_3/default@not_existing:9000/1.bin');" \ + query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin');" \ "select * from t" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) assert out == '1\ta\n2\tbb\n3\tccc\n' - node.query("drop table distr_3") + node.query("drop table test.distr_3") diff --git a/dbms/tests/integration/test_merge_tree_s3/configs/users.xml b/dbms/tests/integration/test_merge_tree_s3/configs/users.xml index 6061af8e33d..7ee1e57fc70 100644 --- a/dbms/tests/integration/test_merge_tree_s3/configs/users.xml +++ b/dbms/tests/integration/test_merge_tree_s3/configs/users.xml @@ -2,6 +2,7 @@ + Ordinary diff --git a/dbms/tests/integration/test_polymorphic_parts/configs/compact_parts.xml b/dbms/tests/integration/test_polymorphic_parts/configs/compact_parts.xml index d4e550f8893..e14c3f0ceae 100644 --- a/dbms/tests/integration/test_polymorphic_parts/configs/compact_parts.xml +++ b/dbms/tests/integration/test_polymorphic_parts/configs/compact_parts.xml @@ -2,4 +2,4 @@ 512 - \ No newline at end of file + diff --git a/dbms/tests/integration/test_polymorphic_parts/configs/users.d/not_optimize_count.xml b/dbms/tests/integration/test_polymorphic_parts/configs/users.d/not_optimize_count.xml index caec593aa40..82689093adf 100644 --- a/dbms/tests/integration/test_polymorphic_parts/configs/users.d/not_optimize_count.xml +++ b/dbms/tests/integration/test_polymorphic_parts/configs/users.d/not_optimize_count.xml @@ -2,6 +2,7 @@ 0 + Ordinary - \ No newline at end of file + diff --git a/dbms/tests/integration/test_polymorphic_parts/configs_old/users.d/not_optimize_count.xml b/dbms/tests/integration/test_polymorphic_parts/configs_old/users.d/not_optimize_count.xml new file mode 100644 index 00000000000..5a06453b214 --- /dev/null +++ b/dbms/tests/integration/test_polymorphic_parts/configs_old/users.d/not_optimize_count.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/dbms/tests/integration/test_polymorphic_parts/test.py b/dbms/tests/integration/test_polymorphic_parts/test.py index f7256de9d9a..55f1237cd1d 100644 --- a/dbms/tests/integration/test_polymorphic_parts/test.py +++ b/dbms/tests/integration/test_polymorphic_parts/test.py @@ -168,7 +168,7 @@ def test_different_part_types_on_replicas(start_cluster, table, part_type): "WHERE table = '{}' AND active GROUP BY part_type ORDER BY part_type".format(table))) == TSV(expected) -node7 = cluster.add_instance('node7', config_dir="configs", with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True) +node7 = cluster.add_instance('node7', config_dir="configs_old", with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True) node8 = cluster.add_instance('node8', config_dir="configs", with_zookeeper=True) settings7 = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760} diff --git a/dbms/tests/integration/test_row_policy/test.py b/dbms/tests/integration/test_row_policy/test.py index 6db24f5799e..9beb2d755af 100644 --- a/dbms/tests/integration/test_row_policy/test.py +++ b/dbms/tests/integration/test_row_policy/test.py @@ -24,7 +24,7 @@ def started_cluster(): cluster.start() instance.query(''' - CREATE DATABASE mydb; + CREATE DATABASE mydb ENGINE=Ordinary; CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1); @@ -42,7 +42,7 @@ def started_cluster(): INSERT INTO mydb.`.filtered_table4` values (0, 0), (0, 1), (1, 0), (1, 1); ''') instance2.query(''' - CREATE DATABASE mydb; + CREATE DATABASE mydb ENGINE=Ordinary; CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1); diff --git a/dbms/tests/queries/0_stateless/01053_drop_database_mat_view.sql b/dbms/tests/queries/0_stateless/01053_drop_database_mat_view.sql index d5763461b42..60803bced7e 100644 --- a/dbms/tests/queries/0_stateless/01053_drop_database_mat_view.sql +++ b/dbms/tests/queries/0_stateless/01053_drop_database_mat_view.sql @@ -1,5 +1,5 @@ DROP DATABASE IF EXISTS some_tests; -CREATE DATABASE some_tests; +CREATE DATABASE some_tests ENGINE=Ordinary; create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; show tables from some_tests; From d75e2d78ff8f4c82f9a03e111b214e453cc057f7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 20 Mar 2020 00:14:52 +0300 Subject: [PATCH 039/743] cleanup task in DatabaseCatalog --- dbms/programs/server/Server.cpp | 5 + dbms/src/Databases/DatabaseAtomic.cpp | 199 +----------------- dbms/src/Databases/DatabaseAtomic.h | 24 +-- dbms/src/Databases/DatabaseLazy.cpp | 2 +- dbms/src/Databases/DatabaseMySQL.h | 1 - dbms/src/Databases/DatabaseOnDisk.cpp | 17 +- dbms/src/Databases/DatabaseOnDisk.h | 3 +- dbms/src/Databases/DatabaseOrdinary.cpp | 2 +- dbms/src/Interpreters/DatabaseCatalog.cpp | 186 +++++++++++++++- dbms/src/Interpreters/DatabaseCatalog.h | 33 ++- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- dbms/src/Storages/IStorage.cpp | 6 + dbms/src/Storages/IStorage.h | 4 + 13 files changed, 255 insertions(+), 229 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 731a71a60af..02cc5e0bf03 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -391,6 +391,11 @@ int Server::main(const std::vector & /*args*/) Poco::File(dictionaries_lib_path).createDirectories(); } + { + /// Directory with metadata of tables, which was marked as dropped by Atomic database + Poco::File(path + "metadata_dropped/").createDirectories(); + } + if (config().has("interserver_http_port") && config().has("interserver_https_port")) throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index ab46ce52c91..20280f81f27 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -21,9 +21,7 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & co : DatabaseOrdinary(name_, metadata_path_, context_) { data_path = "store/"; - auto log_name = "DatabaseAtomic (" + name_ + ")"; - log = &Logger::get(log_name); - drop_task = context_.getSchedulePool().createTask(log_name, [this](){ this->dropTableDataTask(); }); + log = &Logger::get("DatabaseAtomic (" + name_ + ")"); } String DatabaseAtomic::getTableDataPath(const String & table_name) const @@ -38,9 +36,7 @@ String DatabaseAtomic::getTableDataPath(const String & table_name) const String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const { - //stringToUUID(query.uuid); /// Check UUID is valid - const size_t uuid_prefix_len = 3; - auto tmp = data_path + toString(query.uuid).substr(0, uuid_prefix_len) + '/' + toString(query.uuid) + '/'; + auto tmp = data_path + getPathForUUID(query.uuid); assert(tmp != data_path && !tmp.empty()); return tmp; @@ -48,57 +44,6 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const void DatabaseAtomic::drop(const Context &) { - - //constexpr size_t max_attempts = 5; - //for (size_t i = 0; i < max_attempts; ++i) - //{ - // auto it = tables_to_drop.begin(); - // while (it != tables_to_drop.end()) - // { - // if (it->table.unique()) - // { - // /// No queries use table, it can be safely dropped - // dropTableFinally(*it); - // it = tables_to_drop.erase(it); - // } - // ++it; - // } - // - // if (tables_to_drop.empty()) - // { - // Poco::File(getMetadataPath()).remove(false); - // return; - // } - //} - //throw Exception("Cannot drop database", ErrorCodes::TABLE_WAS_NOT_DROPPED); - - /// IDatabase::drop() is called under global context lock (TODO can it be fixed?) - - auto it = std::find_if(tables_to_drop.begin(), tables_to_drop.end(), [](const TableToDrop & elem) - { - return !elem.table.unique(); - }); - if (it != tables_to_drop.end()) - throw Exception("Cannot drop database " + getDatabaseName() + - ". It contains table " + it->table->getStorageID().getNameForLogs() + - ", which is used by " + std::to_string(it->table.use_count() - 1) + " queries. " - "Client should retry later.", ErrorCodes::DATABASE_NOT_EMPTY); - - //FIXME maybe make `tables_to_drop` global for all Atomic databases? - - //for (auto & table : tables_to_drop) - //{ - // try - // { - // /// IStorage::drop() may use DatabaseCatalog, so databases mutex will be acquired (possible deadlock here) - // dropTableFinally(table); - // } - // catch (...) - // { - // tryLogCurrentException(log, "Cannot drop table. Metadata " + table.data_path + " will be removed forcefully. " - // "Garbage may be left in /store directory and ZooKeeper."); - // } - //} Poco::File(getMetadataPath()).remove(true); } @@ -119,39 +64,17 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) return DatabaseWithDictionaries::detachTable(name); } -void DatabaseAtomic::dropTable(const Context & context, const String & table_name) +void DatabaseAtomic::dropTable(const Context &, const String & table_name) { String table_metadata_path = getObjectMetadataPath(table_name); - String table_metadata_path_drop = table_metadata_path + drop_suffix; - String table_data_path_relative = getTableDataPath(table_name); - assert(!table_data_path_relative.empty()); + //FIXME StoragePtr table = detachTable(table_name); - try - { - // FIXME - // 1. CREATE table_name: + table_name.sql - // 2. DROP table_name: table_name.sql -> table_name.sql.tmp_drop - // 3. CREATE table_name: + table_name.sql - // 4. DROP table_name: table_name.sql -> table_name.sql.tmp_drop overwrites table_name.sql.tmp_drop - Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); - { - LOG_INFO(log, "Mark table " + table->getStorageID().getNameForLogs() + " to drop."); - /// Context:getPath acquires lock - auto table_data_path = context.getPath() + table_data_path_relative; - std::lock_guard lock(tables_to_drop_mutex); - time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - tables_to_drop.push_back({table, table_data_path, current_time}); - } - } - catch (...) - { - LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); - attachTable(table_name, table, table_data_path_relative); - Poco::File(table_metadata_path_drop).renameTo(table_metadata_path); - throw; - } + String table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID()); + LOG_INFO(log, "Mark table " + table->getStorageID().getNameForLogs() + " to drop."); + Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); + DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop); } void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, @@ -183,119 +106,13 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) { - iterateMetadataFiles(context, [](const String &) {}, [&](const String & file_name) - { - /// Process .sql.tmp_drop files with metadata of partially dropped tables - String full_path = getMetadataPath() + file_name; - LOG_INFO(log, "Trying load partially dropped table from " << full_path); - ASTPtr ast; - const ASTCreateQuery * create = nullptr; - try - { - ast = parseQueryFromMetadata(context, full_path, /*throw_on_error*/ false, /*remove_empty*/false); - create = typeid_cast(ast.get()); - if (!create || create->uuid == UUIDHelpers::Nil) - { - LOG_WARNING(log, "Cannot parse metadata of partially dropped table from " << full_path - << ". Removing metadata. Garbage may be left in /store directory and ZooKeeper."); - if (Poco::File(full_path).exists()) - Poco::File(full_path).remove(); - return; - } - auto [_, table] = createTableFromAST(*create, database_name, getTableDataPath(*create), context, has_force_restore_data_flag); - time_t drop_time = Poco::File(full_path).getLastModified().epochTime(); - tables_to_drop.push_back({table, context.getPath() + getTableDataPath(*create), drop_time}); - } - catch (...) - { - if (!create) - throw; - auto table_data_relative_path = getTableDataPath(*create); - if (table_data_relative_path.empty()) - throw; - - Poco::File table_data{context.getPath() + table_data_relative_path}; - tryLogCurrentException(log, "Cannot load partially dropped table from: " + full_path + - ". Parsed query: " + serializeAST(*create) + - ". Removing metadata and " + table_data.path() + - ". Garbage may be left in ZooKeeper."); - if (table_data.exists()) - table_data.remove(true); - Poco::File{full_path}.remove(); - } - }); - DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag); - drop_task->activateAndSchedule(); } void DatabaseAtomic::shutdown() { - drop_task->deactivate(); DatabaseWithDictionaries::shutdown(); } -void DatabaseAtomic::dropTableDataTask() -{ - LOG_INFO(log, String("Wake up ") + __PRETTY_FUNCTION__); - TableToDrop table; - try - { - std::lock_guard lock(tables_to_drop_mutex); - LOG_INFO(log, "There are " + std::to_string(tables_to_drop.size()) + " tables to drop"); - time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - auto it = std::find_if(tables_to_drop.begin(), tables_to_drop.end(), [current_time, this](const TableToDrop & elem) - { - LOG_INFO(log, "Check table " + elem.table->getStorageID().getNameForLogs() + ": " + - "refcount = " + std::to_string(elem.table.unique()) + ", " + - "time elapsed = " + std::to_string(current_time - elem.drop_time)); - return elem.table.unique() && elem.drop_time + drop_delay_s < current_time; - }); - if (it != tables_to_drop.end()) - { - table = std::move(*it); - LOG_INFO(log, "Will try drop " + table.table->getStorageID().getNameForLogs()); - tables_to_drop.erase(it); - } - } - catch (...) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - } - - if (table.table) - { - try - { - dropTableFinally(table); - } - catch (...) - { - tryLogCurrentException(log, "Cannot drop table " + table.table->getStorageID().getNameForLogs() + - ". Will retry later."); - { - std::lock_guard lock(tables_to_drop_mutex); - tables_to_drop.emplace_back(std::move(table)); - } - } - } - - drop_task->scheduleAfter(reschedule_time_ms); -} - -void DatabaseAtomic::dropTableFinally(const DatabaseAtomic::TableToDrop & table) const -{ - LOG_INFO(log, "Trying to drop table " + table.table->getStorageID().getNameForLogs()); - table.table->drop(); - table.table->is_dropped = true; - Poco::File table_data_dir{table.data_path}; - if (table_data_dir.exists()) - table_data_dir.remove(true); - - String metadata_tmp_drop = getObjectMetadataPath(table.table->getStorageID().getTableName()) + drop_suffix; - Poco::File(metadata_tmp_drop).remove(); -} - - } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index 0f97c00d20c..3e43cc07bd7 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -31,35 +31,23 @@ public: String getTableDataPath(const String & table_name) const override; String getTableDataPath(const ASTCreateQuery & query) const override; + inline static String getPathForUUID(const UUID & uuid) + { + const size_t uuid_prefix_len = 3; + return toString(uuid).substr(0, uuid_prefix_len) + '/' + toString(uuid) + '/'; + } + void drop(const Context & /*context*/) override; void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; void shutdown() override; private: - struct TableToDrop - { - StoragePtr table; - String data_path; - time_t drop_time; - //time_t last_attempt_time; - }; - using TablesToDrop = std::list; - - void dropTableDataTask(); - void dropTableFinally(const TableToDrop & table) const; - -private: - static constexpr time_t drop_delay_s = 10; - static constexpr size_t reschedule_time_ms = 5000; //TODO store path in DatabaseWithOwnTables::tables std::map table_name_to_path; - TablesToDrop tables_to_drop; - std::mutex tables_to_drop_mutex; - BackgroundSchedulePoolTaskHolder drop_task; }; diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 25a308faf00..4d7de532d38 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -227,7 +227,7 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table StoragePtr table; Context context_copy(context); /// some tables can change context, but not LogTables - auto ast = parseQueryFromMetadata(context, table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false); + auto ast = parseQueryFromMetadata(log, context, table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false); if (ast) { auto & ast_create = ast->as(); diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index 01885be8d70..d7c53ed7ceb 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -1,7 +1,6 @@ #pragma once #include "config_core.h" -#define USE_MYSQL 1 #if USE_MYSQL #include diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index dabbac81f3c..d475666a246 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -56,8 +56,8 @@ std::pair createTableFromAST( { const auto & table_function = ast_create_query.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); - //FIXME storage will have wrong database name StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table); + storage->resetStorageIDForTableFunction({ast_create_query.database, ast_create_query.table, ast_create_query.uuid}); return {ast_create_query.table, storage}; } /// We do not directly use `InterpreterCreateQuery::execute`, because @@ -254,7 +254,7 @@ void DatabaseOnDisk::renameTable( table_lock = table->lockExclusively(context.getCurrentQueryId()); table_metadata_path = getObjectMetadataPath(table_name); - attach_query = parseQueryFromMetadata(context, table_metadata_path); + attach_query = parseQueryFromMetadata(log, context, table_metadata_path); auto & create = attach_query->as(); create.table = to_table_name; if (from_ordinary_to_atomic) @@ -407,7 +407,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & /*context*/, const Ite } } -ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const String & metadata_file_path, bool throw_on_error /*= true*/, bool remove_empty /*= false*/) const +ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * loger, const Context & context, const String & metadata_file_path, bool throw_on_error /*= true*/, bool remove_empty /*= false*/) { String query; @@ -429,7 +429,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str */ if (remove_empty && query.empty()) { - LOG_ERROR(log, "File " << metadata_file_path << " is empty. Removing."); + LOG_ERROR(loger, "File " << metadata_file_path << " is empty. Removing."); Poco::File(metadata_file_path).remove(); return nullptr; } @@ -439,7 +439,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str const char * pos = query.data(); std::string error_message; auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false, - "in file " + getMetadataPath(), /* allow_multi_statements = */ false, 0, settings.max_parser_depth); + "in file " + metadata_file_path, /* allow_multi_statements = */ false, 0, settings.max_parser_depth); if (!ast && throw_on_error) throw Exception(error_message, ErrorCodes::SYNTAX_ERROR); @@ -449,14 +449,11 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str auto & create = ast->as(); if (create.uuid != UUIDHelpers::Nil) { - //FIXME it can be .sql or .sql.tmp_drop String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName(); - if (Poco::Path(table_name).makeFile().getExtension() == "sql") - table_name = Poco::Path(table_name).makeFile().getBaseName(); table_name = unescapeForFileName(table_name); if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER) - LOG_WARNING(log, "File " << metadata_file_path << " contains both UUID and table name. " + LOG_WARNING(loger, "File " << metadata_file_path << " contains both UUID and table name. " "Will use name `" << table_name << "` instead of `" << create.table << "`"); create.table = table_name; } @@ -466,7 +463,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const Context & context, const String & database_metadata_path, bool throw_on_error) const { - ASTPtr ast = parseQueryFromMetadata(context, database_metadata_path, throw_on_error); + ASTPtr ast = parseQueryFromMetadata(log, context, database_metadata_path, throw_on_error); if (ast) { diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 608d529a410..1a80eff25d7 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -61,6 +61,8 @@ public: String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.table); } String getMetadataPath() const override { return metadata_path; } + static ASTPtr parseQueryFromMetadata(Poco::Logger * log, const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false); + protected: static constexpr const char * create_suffix = ".tmp"; static constexpr const char * drop_suffix = ".tmp_drop"; @@ -76,7 +78,6 @@ protected: const String & table_name, bool throw_on_error) const override; - ASTPtr parseQueryFromMetadata(const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false) const; ASTPtr getCreateQueryFromMetadata(const Context & context, const String & metadata_path, bool throw_on_error) const; diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 82071e9c858..9afe3b06ce8 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -125,7 +125,7 @@ void DatabaseOrdinary::loadStoredObjects( String full_path = getMetadataPath() + file_name; try { - auto ast = parseQueryFromMetadata(context, full_path, /*throw_on_error*/ true, /*remove_empty*/ false); + auto ast = parseQueryFromMetadata(log, context, full_path, /*throw_on_error*/ true, /*remove_empty*/ false); if (ast) { auto * create_query = ast->as(); diff --git a/dbms/src/Interpreters/DatabaseCatalog.cpp b/dbms/src/Interpreters/DatabaseCatalog.cpp index bf160cfc056..bda1a75349a 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.cpp +++ b/dbms/src/Interpreters/DatabaseCatalog.cpp @@ -4,9 +4,16 @@ #include #include #include +#include #include #include #include +#include +#include +#include +#include +#include +#include namespace DB { @@ -103,10 +110,18 @@ void DatabaseCatalog::loadDatabases() auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); + + loadMarkedAsDroppedTables(); + auto task_holder = global_context->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->dropTableDataTask(); }); + drop_task = std::make_unique(std::move(task_holder)); + (*drop_task)->activateAndSchedule(); } void DatabaseCatalog::shutdown() { + if (drop_task) + (*drop_task)->deactivate(); + /** At this point, some tables may have threads that block our mutex. * To shutdown them correctly, we will copy the current list of tables, * and ask them all to finish their work. @@ -343,14 +358,14 @@ void DatabaseCatalog::removeUUIDMapping(const UUID & uuid) throw Exception("Mapping for table with UUID=" + toString(uuid) + " doesn't exist", ErrorCodes::LOGICAL_ERROR); } -DatabaseCatalog::DatabaseCatalog(const Context * global_context_) +DatabaseCatalog::DatabaseCatalog(Context * global_context_) : global_context(global_context_), log(&Poco::Logger::get("DatabaseCatalog")) { if (!global_context) throw Exception("DatabaseCatalog is not initialized. It's a bug.", ErrorCodes::LOGICAL_ERROR); } -DatabaseCatalog & DatabaseCatalog::init(const Context * global_context_) +DatabaseCatalog & DatabaseCatalog::init(Context * global_context_) { static DatabaseCatalog database_catalog(global_context_); return database_catalog; @@ -433,6 +448,173 @@ DatabaseAndTable DatabaseCatalog::tryGetDatabaseAndTable(const StorageID & table return res; } +void DatabaseCatalog::loadMarkedAsDroppedTables() +{ + std::map dropped_metadata; + String path = global_context->getPath() + "metadata_dropped/"; + Poco::DirectoryIterator dir_end; + for (Poco::DirectoryIterator it(path); it != dir_end; ++it) + { + if (!it.name().ends_with(".sql")) + continue; + + /// Process .sql files with metadata of tables which were marked as dropped + String full_path = path + it.name(); + + Strings name_parts; + boost::split(name_parts, it.name(), boost::is_any_of(".")); + if (name_parts.size() != 4) /// Unexpected file + continue; + + StorageID dropped_id = StorageID::createEmpty(); + dropped_id.database_name = unescapeForFileName(name_parts[0]); + dropped_id.table_name = unescapeForFileName(name_parts[1]); + dropped_id.uuid = parse(name_parts[2]); + + dropped_metadata.emplace(std::move(full_path), std::move(dropped_id)); + } + + ThreadPool pool(SettingMaxThreads().getAutoValue()); + for (const auto & elem : dropped_metadata) + { + pool.scheduleOrThrowOnError([&]() + { + this->enqueueDroppedTableCleanup(elem.second, nullptr, elem.first); + }); + } + pool.wait(); +} + +String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const +{ + return global_context->getPath() + "metadata_dropped/" + + escapeForFileName(table_id.getDatabaseName()) + "." + + escapeForFileName(table_id.getTableName()) + "." + + toString(table_id.uuid) + ".sql"; +} + +void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay) +{ + assert(table_id.hasUUID()); + assert(!table || table->getStorageID().uuid == table_id.uuid); + assert(dropped_metadata_path == getPathForDroppedMetadata(table_id)); + + time_t drop_time; + if (table) + drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + else + { + /// Try load table from metadata to drop it correctly (e.g. remove metadata from zk) + LOG_INFO(log, "Trying load partially dropped table " << table_id.getNameForLogs() << " from " << dropped_metadata_path); + ASTPtr ast = DatabaseOnDisk::parseQueryFromMetadata(log, *global_context, dropped_metadata_path, /*throw_on_error*/ false, /*remove_empty*/false); + auto create = typeid_cast(ast.get()); + assert(!create || create->uuid == table_id.uuid); + + if (create) + { + String data_path = "store/" + DatabaseAtomic::getPathForUUID(table_id.uuid); + create->database = table_id.database_name; + create->table = table_id.table_name; + try + { + table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, *global_context, false).second; + } + catch (...) + { + tryLogCurrentException(log, "Cannot load partially dropped table " + table_id.getNameForLogs() + + " from: " + dropped_metadata_path + + ". Parsed query: " + serializeAST(*create) + + ". Will remove metadata and " + data_path + + ". Garbage may be left in ZooKeeper."); + } + } + else + { + LOG_WARNING(log, "Cannot parse metadata of partially dropped table " + << table_id.getNameForLogs() << " from " << dropped_metadata_path + << ". Will remove metadata file and data directory. Garbage may be left in /store directory and ZooKeeper."); + } + + drop_time = Poco::File(dropped_metadata_path).getLastModified().epochTime(); + } + + std::lock_guard lock(tables_marked_droped_mutex); + if (ignore_delay) + tables_marked_droped.push_front({table_id, table, dropped_metadata_path, 0}); + else + tables_marked_droped.push_back({table_id, table, dropped_metadata_path, drop_time}); +} + +void DatabaseCatalog::dropTableDataTask() +{ + //LOG_INFO(log, String("Wake up ") + __PRETTY_FUNCTION__); + TableMarkedAsDropped table; + try + { + std::lock_guard lock(tables_marked_droped_mutex); + LOG_INFO(log, "There are " + std::to_string(tables_marked_droped.size()) + " tables to drop"); + time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + auto it = std::find_if(tables_marked_droped.begin(), tables_marked_droped.end(), [current_time/*, this*/](const auto & elem) + { + //LOG_INFO(log, "Check table " + elem.table_id.getNameForLogs() + ": " + + // "refcount = " + std::to_string(elem.table.use_count()) + ", " + + // "time elapsed = " + std::to_string(current_time - elem.drop_time)); + return (!elem.table || elem.table.unique()) && elem.drop_time + drop_delay_s < current_time; + }); + if (it != tables_marked_droped.end()) + { + table = std::move(*it); + LOG_INFO(log, "Will try drop " + table.table_id.getNameForLogs()); + tables_marked_droped.erase(it); + } + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + if (table.table_id) + { + try + { + dropTableFinally(table); + } + catch (...) + { + tryLogCurrentException(log, "Cannot drop table " + table.table_id.getNameForLogs() + + ". Will retry later."); + { + std::lock_guard lock(tables_marked_droped_mutex); + tables_marked_droped.emplace_back(std::move(table)); + } + } + } + + (*drop_task)->scheduleAfter(reschedule_time_ms); +} + +void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) const +{ + if (table.table) + { + LOG_INFO(log, "Trying to drop table " + table.table_id.getNameForLogs()); + table.table->drop(); + table.table->is_dropped = true; + } + + /// Even if table is not loaded, try remove its data from disk. + /// TODO remove data from all volumes + String data_path = global_context->getPath() + "store/" + DatabaseAtomic::getPathForUUID(table.table_id.uuid); + Poco::File table_data_dir{data_path}; + if (table_data_dir.exists()) + { + LOG_INFO(log, "Removing data directory " << data_path << " of table " << table.table_id.getNameForLogs()); + table_data_dir.remove(true); + } + + Poco::File(table.metadata_path).remove(); +} + DDLGuard::DDLGuard(Map & map_, std::unique_lock guards_lock_, const String & elem) : map(map_), guards_lock(std::move(guards_lock_)) diff --git a/dbms/src/Interpreters/DatabaseCatalog.h b/dbms/src/Interpreters/DatabaseCatalog.h index bede7061d1a..bf8889da393 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.h +++ b/dbms/src/Interpreters/DatabaseCatalog.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -91,6 +92,7 @@ struct TemporaryTableHolder : boost::noncopyable ///TODO maybe remove shared_ptr from here? using TemporaryTablesMapping = std::map>; +class BackgroundSchedulePoolTaskHolder; class DatabaseCatalog : boost::noncopyable { @@ -98,7 +100,7 @@ public: static constexpr const char * TEMPORARY_DATABASE = "_temporary_and_external_tables"; static constexpr const char * SYSTEM_DATABASE = "system"; - static DatabaseCatalog & init(const Context * global_context_); + static DatabaseCatalog & init(Context * global_context_); static DatabaseCatalog & instance(); void loadDatabases(); @@ -151,11 +153,15 @@ public: void removeUUIDMapping(const UUID & uuid); DatabaseAndTable tryGetByUUID(const UUID & uuid) const; + String getPathForDroppedMetadata(const StorageID & table_id) const; + void enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay = false); + private: - DatabaseCatalog(const Context * global_context_); + DatabaseCatalog(Context * global_context_); void assertDatabaseExistsUnlocked(const String & database_name) const; void assertDatabaseDoesntExistUnlocked(const String & database_name) const; + struct UUIDToStorageMapPart { std::unordered_map map; @@ -170,9 +176,25 @@ private: return uuid.toUnderType().low >> (64 - bits_for_first_level); } + struct TableMarkedAsDropped + { + StorageID table_id = StorageID::createEmpty(); + StoragePtr table; + String metadata_path; + time_t drop_time; + }; + using TablesMarkedAsDropped = std::list; + + void loadMarkedAsDroppedTables(); + void dropTableDataTask(); + void dropTableFinally(const TableMarkedAsDropped & table) const; + + static constexpr time_t drop_delay_s = 10; + static constexpr size_t reschedule_time_ms = 100; + private: /// For some reason Context is required to get Storage from Database object - const Context * global_context; + Context * global_context; mutable std::mutex databases_mutex; ViewDependencies view_dependencies; @@ -191,6 +213,11 @@ private: DDLGuards ddl_guards; /// If you capture mutex and ddl_guards_mutex, then you need to grab them strictly in this order. mutable std::mutex ddl_guards_mutex; + + TablesMarkedAsDropped tables_marked_droped; + mutable std::mutex tables_marked_droped_mutex; + + std::unique_ptr drop_task; }; } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 2a4f1e06998..e513430e69d 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -649,8 +649,8 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, { const auto & table_function = create.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); - //FIXME storage will have wrong database name res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table); + res->resetStorageIDForTableFunction({create.database, create.table, create.uuid}); } else { diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 45c116b6ab6..45c4251e4cb 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -431,4 +431,10 @@ void IStorage::renameInMemory(const String & new_database_name, const String & n storage_id.table_name = new_table_name; } +void IStorage::resetStorageIDForTableFunction(const StorageID & actual_table_id) +{ + std::lock_guard lock(id_mutex); + storage_id = actual_table_id; +} + } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index c899d035892..342fd7c0391 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -94,6 +94,10 @@ public: /// The name of the table. StorageID getStorageID() const; + /// This method is required to set actual storage id for storage created by table function. Do not use it + /// TODO refactor table names in table functions + void resetStorageIDForTableFunction(const StorageID & actual_table_id); + /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } From 9061b169f72e46654844b93b714c20c687e31c1c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 20 Mar 2020 03:07:52 +0300 Subject: [PATCH 040/743] no delay drop option --- dbms/src/Databases/DatabaseAtomic.cpp | 4 +- dbms/src/Databases/DatabaseAtomic.h | 2 +- dbms/src/Databases/DatabaseLazy.cpp | 5 +- dbms/src/Databases/DatabaseLazy.h | 3 +- dbms/src/Databases/DatabaseMemory.cpp | 3 +- dbms/src/Databases/DatabaseMemory.h | 3 +- dbms/src/Databases/DatabaseMySQL.cpp | 2 +- dbms/src/Databases/DatabaseMySQL.h | 2 +- dbms/src/Databases/DatabaseOnDisk.cpp | 2 +- dbms/src/Databases/DatabaseOnDisk.h | 3 +- dbms/src/Databases/IDatabase.h | 3 +- .../src/Interpreters/InterpreterDropQuery.cpp | 47 +++++++++---------- dbms/src/Interpreters/InterpreterDropQuery.h | 2 +- dbms/src/Parsers/ASTDropQuery.cpp | 3 ++ dbms/src/Parsers/ASTDropQuery.h | 2 + dbms/src/Parsers/ParserDropQuery.cpp | 6 +++ dbms/src/Storages/StorageMaterializedView.cpp | 2 +- ...00029_test_zookeeper_optimize_exception.sh | 3 ++ .../00121_drop_column_zookeeper.sql | 7 +-- ...olumn_in_partition_concurrent_zookeeper.sh | 5 +- ...46_clear_column_in_partition_zookeeper.sql | 5 +- ...lized_view_and_too_many_parts_zookeeper.sh | 14 +++--- ...sert_have_data_before_quorum_zookeeper.sql | 5 +- ...ert_lost_part_and_alive_part_zookeeper.sql | 5 +- ...0732_quorum_insert_lost_part_zookeeper.sql | 5 +- ..._old_data_and_without_quorum_zookeeper.sql | 5 +- ...m_insert_simple_test_1_parts_zookeeper.sql | 5 +- ...m_insert_simple_test_2_parts_zookeeper.sql | 5 +- ...r_modify_order_by_replicated_zookeeper.sql | 5 +- ...y_replicated_merge_tree_optimize_final.sql | 5 +- ...ent_move_partition_from_table_zookeeper.sh | 5 +- 31 files changed, 103 insertions(+), 70 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index 20280f81f27..b36191f19a9 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -64,7 +64,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) return DatabaseWithDictionaries::detachTable(name); } -void DatabaseAtomic::dropTable(const Context &, const String & table_name) +void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool no_delay) { String table_metadata_path = getObjectMetadataPath(table_name); @@ -74,7 +74,7 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name) String table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID()); LOG_INFO(log, "Mark table " + table->getStorageID().getNameForLogs() + " to drop."); Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); - DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop); + DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); } void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index 3e43cc07bd7..e52ffec4777 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -23,7 +23,7 @@ public: const String & to_table_name) override; //void removeTable(const Context & context, const String & table_name) override; - void dropTable(const Context & context, const String & table_name) override; + void dropTable(const Context & context, const String & table_name, bool no_delay) override; void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path = {}) override; StoragePtr detachTable(const String & name) override; diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 4d7de532d38..430e5ca47cd 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -66,10 +66,11 @@ void DatabaseLazy::createTable( void DatabaseLazy::dropTable( const Context & context, - const String & table_name) + const String & table_name, + bool no_delay) { SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::dropTable(context, table_name); + DatabaseOnDisk::dropTable(context, table_name, no_delay); } void DatabaseLazy::renameTable( diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index effda57a4ba..e95f7700422 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -34,7 +34,8 @@ public: void dropTable( const Context & context, - const String & table_name) override; + const String & table_name, + bool no_delay = false) override; void renameTable( const Context & context, diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index b34c18d295b..9ee5b45f064 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -32,7 +32,8 @@ void DatabaseMemory::createTable( void DatabaseMemory::dropTable( const Context & /*context*/, - const String & table_name) + const String & table_name, + bool /*no_delay*/) { std::lock_guard lock{mutex}; auto table = detachTableUnlocked(table_name); diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 2d8ec6c21f9..b96716dc8e3 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -31,7 +31,8 @@ public: void dropTable( const Context & context, - const String & table_name) override; + const String & table_name, + bool no_delay = false) override; ASTPtr getCreateTableQueryImpl(const Context & /*context*/, const String & name, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery(const Context & /*context*/) const override; diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index c0dc59cab01..a2d05dad7b8 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -431,7 +431,7 @@ void DatabaseMySQL::loadStoredObjects(Context &, bool) } } -void DatabaseMySQL::dropTable(const Context &, const String & table_name) +void DatabaseMySQL::dropTable(const Context &, const String & table_name, bool /*no_delay*/) { std::lock_guard lock{mutex}; diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index d7c53ed7ceb..a69d433ad57 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -52,7 +52,7 @@ public: StoragePtr detachTable(const String & table_name) override; - void dropTable(const Context &, const String & table_name) override; + void dropTable(const Context &, const String & table_name, bool no_delay = false) override; void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path = {}) override; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index d475666a246..e4469ec35eb 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -192,7 +192,7 @@ void DatabaseOnDisk::createTable( } } -void DatabaseOnDisk::dropTable(const Context & context, const String & table_name) +void DatabaseOnDisk::dropTable(const Context & context, const String & table_name, bool /*no_delay*/) { String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_path_drop = table_metadata_path + drop_suffix; diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 1a80eff25d7..293c3e2a254 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -40,7 +40,8 @@ public: void dropTable( const Context & context, - const String & table_name) override; + const String & table_name, + bool no_delay = false) override; void renameTable( const Context & context, diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 7c4585d1de0..d395161c730 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -172,7 +172,8 @@ public: /// Delete the table from the database, drop table and delete the metadata. virtual void dropTable( const Context & /*context*/, - const String & /*name*/) + const String & /*name*/, + [[maybe_unused]] bool no_delay = false) { throw Exception("There is no DROP TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index b815c80999d..25f28af92ec 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -38,7 +38,7 @@ BlockIO InterpreterDropQuery::execute() if (!drop.table.empty()) { if (!drop.is_dictionary) - return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock); + return executeToTable({drop.database, drop.table, drop.uuid}, drop); else return executeToDictionary(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock); } @@ -50,46 +50,42 @@ BlockIO InterpreterDropQuery::execute() BlockIO InterpreterDropQuery::executeToTable( - const String & database_name_, - const String & table_name, - ASTDropQuery::Kind kind, - bool if_exists, - bool is_temporary, - bool no_ddl_lock) + const StorageID & table_id_, + const ASTDropQuery & query) { - if (is_temporary || database_name_.empty()) + if (query.temporary || table_id_.database_name.empty()) { - if (context.tryResolveStorageID({"", table_name}, Context::ResolveExternal)) - return executeToTemporaryTable(table_name, kind); + if (context.tryResolveStorageID(table_id_, Context::ResolveExternal)) + return executeToTemporaryTable(table_id_.getTableName(), query.kind); } - if (is_temporary) + if (query.temporary) { - if (if_exists) + if (query.if_exists) return {}; - throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist.", + throw Exception("Temporary table " + backQuoteIfNeed(table_id_.getTableName()) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); } - String database_name = context.resolveDatabase(database_name_); + auto table_id = context.resolveStorageID(table_id_, Context::ResolveOrdinary); - auto ddl_guard = (!no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(database_name, table_name) : nullptr); + auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); - auto [database, table] = tryGetDatabaseAndTable(database_name, table_name, if_exists); + auto [database, table] = tryGetDatabaseAndTable(table_id.database_name, table_id.table_name, query.if_exists); if (database && table) { - auto table_id = table->getStorageID(); - if (kind == ASTDropQuery::Kind::Detach) + table_id = table->getStorageID(); + if (query.kind == ASTDropQuery::Kind::Detach) { context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id); table->shutdown(); /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); /// Drop table from memory, don't touch data and metadata - database->detachTable(table_name); + database->detachTable(table_id.table_name); } - else if (kind == ASTDropQuery::Kind::Truncate) + else if (query.kind == ASTDropQuery::Kind::Truncate) { context.checkAccess(table->isView() ? AccessType::TRUNCATE_VIEW : AccessType::TRUNCATE_TABLE, table_id); table->checkTableCanBeDropped(); @@ -99,7 +95,7 @@ BlockIO InterpreterDropQuery::executeToTable( /// Drop table data, don't touch metadata table->truncate(query_ptr, context, table_lock); } - else if (kind == ASTDropQuery::Kind::Drop) + else if (query.kind == ASTDropQuery::Kind::Drop) { context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id); table->checkTableCanBeDropped(); @@ -111,7 +107,7 @@ BlockIO InterpreterDropQuery::executeToTable( if (database->getEngineName() != "Atomic") table_lock = table->lockExclusively(context.getCurrentQueryId()); - database->dropTable(context, table_name); + database->dropTable(context, table_id.table_name, query.no_delay); } } @@ -214,10 +210,13 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS context.checkAccess(AccessType::DROP_DATABASE, database_name); /// DETACH or DROP all tables and dictionaries inside database + ASTDropQuery query; + query.kind = kind; + query.database = database_name; for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { - String current_table_name = iterator->name(); - executeToTable(database_name, current_table_name, kind, false, false, false); + query.table = iterator->name(); + executeToTable({query.database, query.table}, query); } for (auto iterator = database->getDictionariesIterator(context); iterator->isValid(); iterator->next()) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.h b/dbms/src/Interpreters/InterpreterDropQuery.h index 6a9c249973e..1f97b982269 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.h +++ b/dbms/src/Interpreters/InterpreterDropQuery.h @@ -30,7 +30,7 @@ private: BlockIO executeToDatabase(const String & database_name, ASTDropQuery::Kind kind, bool if_exists); - BlockIO executeToTable(const String & database_name, const String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool is_temporary, bool no_ddl_lock); + BlockIO executeToTable(const StorageID & table_id, const ASTDropQuery & query); BlockIO executeToDictionary(const String & database_name, const String & dictionary_name, ASTDropQuery::Kind kind, bool if_exists, bool is_temporary, bool no_ddl_lock); diff --git a/dbms/src/Parsers/ASTDropQuery.cpp b/dbms/src/Parsers/ASTDropQuery.cpp index 56d0878ceed..841f54c4f3d 100644 --- a/dbms/src/Parsers/ASTDropQuery.cpp +++ b/dbms/src/Parsers/ASTDropQuery.cpp @@ -63,6 +63,9 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState settings.ostr << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); formatOnCluster(settings); + + if (no_delay) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " NO DELAY" << (settings.hilite ? hilite_none : ""); } } diff --git a/dbms/src/Parsers/ASTDropQuery.h b/dbms/src/Parsers/ASTDropQuery.h index 52112aa4154..9bc2a929a03 100644 --- a/dbms/src/Parsers/ASTDropQuery.h +++ b/dbms/src/Parsers/ASTDropQuery.h @@ -28,6 +28,8 @@ public: /// We dropping dictionary, so print correct word bool is_dictionary{false}; + bool no_delay{false}; + /** Get the text that identifies this element. */ String getID(char) const override; ASTPtr clone() const override; diff --git a/dbms/src/Parsers/ParserDropQuery.cpp b/dbms/src/Parsers/ParserDropQuery.cpp index c82b4d4997c..0485dbca288 100644 --- a/dbms/src/Parsers/ParserDropQuery.cpp +++ b/dbms/src/Parsers/ParserDropQuery.cpp @@ -59,6 +59,7 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect ParserToken s_dot(TokenType::Dot); ParserKeyword s_if_exists("IF EXISTS"); ParserIdentifier name_p; + ParserKeyword s_no_delay("NO DELAY"); ASTPtr database; ASTPtr table; @@ -66,6 +67,7 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect bool if_exists = false; bool temporary = false; bool is_dictionary = false; + bool no_delay = false; if (s_database.ignore(pos, expected)) { @@ -111,6 +113,9 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) return false; } + + if (s_no_delay.ignore(pos, expected)) + no_delay = true; } auto query = std::make_shared(); @@ -120,6 +125,7 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect query->if_exists = if_exists; query->temporary = temporary; query->is_dictionary = is_dictionary; + query->no_delay = no_delay; tryGetIdentifierNameInto(database, query->database); tryGetIdentifierNameInto(table, query->table); diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index d635fc953ad..4e013b76671 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -37,7 +37,7 @@ namespace ErrorCodes static inline String generateInnerTableName(const StorageID & view_id) { if (view_id.hasUUID()) - return ".inner." + toString(view_id.uuid); + return ".inner_id." + toString(view_id.uuid); return ".inner." + view_id.getTableName(); } diff --git a/dbms/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh b/dbms/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh index ef33f037690..7c55f59a3f5 100755 --- a/dbms/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh +++ b/dbms/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh @@ -21,3 +21,6 @@ echo `${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 --server_logs_file=/dev/nu | grep -c 'Code: 388. DB::Exception: .* DB::Exception: .* Cannot select parts for optimization' echo `${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 --server_logs_file=/dev/null --query="OPTIMIZE TABLE test_optimize_exception_replicated PARTITION 201710" 2>&1` \ | grep -c 'Code: 388. DB::Exception: .* DB::Exception:.* Cannot select parts for optimization' + +${CLICKHOUSE_CLIENT} --query="DROP TABLE test_optimize_exception NO DELAY" +sleep 1 diff --git a/dbms/tests/queries/0_stateless/00121_drop_column_zookeeper.sql b/dbms/tests/queries/0_stateless/00121_drop_column_zookeeper.sql index ff05cea9c84..61c23db8d33 100644 --- a/dbms/tests/queries/0_stateless/00121_drop_column_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00121_drop_column_zookeeper.sql @@ -4,8 +4,8 @@ CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickh INSERT INTO alter_00121 VALUES ('2014-01-01', 1); ALTER TABLE alter_00121 DROP COLUMN x; -DROP TABLE alter_00121; - +DROP TABLE alter_00121 NO DELAY; +SELECT sleep(1) FORMAT Null; CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_00121', 'r1', d, (d), 8192); @@ -20,4 +20,5 @@ SELECT * FROM alter_00121 ORDER BY d; ALTER TABLE alter_00121 DROP COLUMN x; SELECT * FROM alter_00121 ORDER BY d; -DROP TABLE alter_00121; +DROP TABLE alter_00121 NO DELAY; +SELECT sleep(1) FORMAT Null; diff --git a/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh b/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh index 54f5afef086..18ecc6f5dfe 100755 --- a/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh @@ -34,5 +34,6 @@ wait $ch "SELECT DISTINCT * FROM clear_column1 WHERE d != toDate('2000-03-01') ORDER BY d, i, s" $ch "SELECT DISTINCT * FROM clear_column2 WHERE d != toDate('2000-03-01') ORDER BY d, i, s" -$ch "DROP TABLE IF EXISTS clear_column1" -$ch "DROP TABLE IF EXISTS clear_column2" +$ch "DROP TABLE clear_column1 NO DELAY" +$ch "DROP TABLE clear_column2 NO DELAY" +sleep 1 diff --git a/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql index 47f644f85a2..b264a53333f 100644 --- a/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql @@ -20,8 +20,9 @@ DROP TABLE clear_column; SELECT '===Replicated case==='; SYSTEM STOP MERGES; -DROP TABLE IF EXISTS clear_column1; -DROP TABLE IF EXISTS clear_column2; +DROP TABLE IF EXISTS clear_column1 NO DELAY; +DROP TABLE IF EXISTS clear_column2 NO DELAY; +SELECT sleep(1) FORMAT Null; CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/clear_column', '1', d, d, 8192); CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/clear_column', '2', d, d, 8192); diff --git a/dbms/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh b/dbms/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh index 2d853eebff7..dca2912b032 100755 --- a/dbms/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh @@ -23,10 +23,11 @@ fi echo ${CLICKHOUSE_CLIENT} --query "SELECT _table, d FROM merge('${CLICKHOUSE_DATABASE}', '^[abc]\$') ORDER BY _table" -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS root" -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS a" -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS b" -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS c" +${CLICKHOUSE_CLIENT} --query "DROP TABLE root NO DELAY" +${CLICKHOUSE_CLIENT} --query "DROP TABLE a NO DELAY" +${CLICKHOUSE_CLIENT} --query "DROP TABLE b NO DELAY" +${CLICKHOUSE_CLIENT} --query "DROP TABLE c NO DELAY" +sleep 2 # Deduplication check for non-replicated root table echo @@ -35,5 +36,6 @@ ${CLICKHOUSE_CLIENT} --query "CREATE MATERIALIZED VIEW a (d UInt64) ENGINE = Rep ${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; ${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; ${CLICKHOUSE_CLIENT} --query "SELECT * FROM a"; -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS root" -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS a" +${CLICKHOUSE_CLIENT} --query "DROP TABLE root NO DELAY" +${CLICKHOUSE_CLIENT} --query "DROP TABLE a NO DELAY" +sleep 1 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql index c71446d0074..9348c94c981 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql @@ -26,5 +26,6 @@ INSERT INTO quorum1 VALUES (6, '2020-12-16'); SELECT x FROM quorum1 ORDER BY x; SELECT x FROM quorum2 ORDER BY x; -DROP TABLE IF EXISTS quorum1; -DROP TABLE IF EXISTS quorum2; +DROP TABLE quorum1 NO DELAY; +DROP TABLE quorum2 NO DELAY; +SELECT sleep(1) FORMAT Null; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql index d99459d38ef..5bfd54c3768 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql @@ -34,5 +34,6 @@ SYSTEM SYNC REPLICA quorum1; SELECT x FROM quorum1 ORDER BY x; SELECT x FROM quorum2 ORDER BY x; -DROP TABLE IF EXISTS quorum1; -DROP TABLE IF EXISTS quorum2; +DROP TABLE quorum1 NO DELAY; +DROP TABLE quorum2 NO DELAY; +SELECT sleep(1) FORMAT Null; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql index 39d2b5feaf1..001932c852e 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql @@ -31,5 +31,6 @@ SYSTEM SYNC REPLICA quorum1; SELECT x FROM quorum1 ORDER BY x; SELECT x FROM quorum2 ORDER BY x; -DROP TABLE IF EXISTS quorum1; -DROP TABLE IF EXISTS quorum2; +DROP TABLE quorum1 NO DELAY; +DROP TABLE quorum2 NO DELAY; +SELECT sleep(1) FORMAT Null; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql index 6e05beeaf25..1644856dc8d 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql @@ -24,5 +24,6 @@ INSERT INTO quorum2 VALUES (4, toDate('2020-12-16')); -- { serverError 319 } SELECT x FROM quorum1 ORDER BY x; SELECT x FROM quorum2 ORDER BY x; -DROP TABLE IF EXISTS quorum1; -DROP TABLE IF EXISTS quorum2; +DROP TABLE quorum1 NO DELAY; +DROP TABLE quorum2 NO DELAY; +SELECT sleep(1) FORMAT Null; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql index 6c28a8e0553..69fd709add8 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -30,5 +30,6 @@ INSERT INTO quorum1 VALUES (4, '2018-11-15'); SELECT sum(x) FROM quorum1; SELECT sum(x) FROM quorum2; -DROP TABLE IF EXISTS quorum1; -DROP TABLE IF EXISTS quorum2; +DROP TABLE quorum1 NO DELAY; +DROP TABLE quorum2 NO DELAY; +SELECT sleep(1) FORMAT Null; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql index 6b8ba6c0af4..74e625cafa4 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql @@ -16,5 +16,6 @@ INSERT INTO quorum1 VALUES (3, '2018-12-16'); SELECT x FROM quorum1 ORDER BY x; SELECT x FROM quorum2 ORDER BY x; -DROP TABLE IF EXISTS quorum1; -DROP TABLE IF EXISTS quorum2; +DROP TABLE quorum1 NO DELAY; +DROP TABLE quorum2 NO DELAY; +SELECT sleep(1) FORMAT Null; diff --git a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index f7ea0c1c0d3..96740d63778 100644 --- a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -48,5 +48,6 @@ SELECT sleep(1) Format Null; SELECT '*** Check SHOW CREATE TABLE after offline ALTER ***'; SHOW CREATE TABLE test.summing_r2; -DROP TABLE test.summing_r1; -DROP TABLE test.summing_r2; +DROP TABLE test.summing_r1 NO DELAY; +DROP TABLE test.summing_r2 NO DELAY; +SELECT sleep(1) Format Null; diff --git a/dbms/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql b/dbms/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql index a79462bf77f..54726fa837a 100644 --- a/dbms/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql +++ b/dbms/tests/queries/0_stateless/00925_zookeeper_empty_replicated_merge_tree_optimize_final.sql @@ -5,5 +5,6 @@ CREATE TABLE replicated_optimize2 (d Date, k UInt64, i32 Int32) ENGINE=Replicate OPTIMIZE TABLE replicated_optimize1 FINAL; -DROP TABLE replicated_optimize1; -DROP TABLE replicated_optimize2; +DROP TABLE replicated_optimize1 NO DELAY; +DROP TABLE replicated_optimize2 NO DELAY; +SELECT sleep(1) FORMAT Null; diff --git a/dbms/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh b/dbms/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh index 32f8cfb2370..bafa3a95b3b 100755 --- a/dbms/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh @@ -58,10 +58,11 @@ timeout $TIMEOUT bash -c thread4 2> /dev/null & wait -echo "DROP TABLE $CLICKHOUSE_DATABASE.src" | ${CLICKHOUSE_CLIENT} -echo "DROP TABLE $CLICKHOUSE_DATABASE.dst" | ${CLICKHOUSE_CLIENT} +echo "DROP TABLE $CLICKHOUSE_DATABASE.src NO DELAY" | ${CLICKHOUSE_CLIENT} +echo "DROP TABLE $CLICKHOUSE_DATABASE.dst NO DELAY" | ${CLICKHOUSE_CLIENT} # Check for deadlocks echo "SELECT * FROM system.processes WHERE query_id LIKE 'query%'" | ${CLICKHOUSE_CLIENT} echo 'did not crash' +sleep 1 From f7793cab325b7de21fba749f525d83957c4d67d9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 20 Mar 2020 15:45:06 +0300 Subject: [PATCH 041/743] check table id in ALTER --- dbms/src/Databases/DatabaseAtomic.h | 2 +- dbms/src/Databases/DatabaseLazy.cpp | 4 ++-- dbms/src/Databases/DatabaseLazy.h | 6 +++--- dbms/src/Databases/DatabaseMemory.h | 2 +- dbms/src/Databases/DatabaseMySQL.cpp | 2 +- dbms/src/Databases/DatabaseMySQL.h | 4 ++-- dbms/src/Databases/DatabaseOnDisk.h | 2 +- dbms/src/Databases/DatabaseOrdinary.cpp | 14 +++++++++++++- dbms/src/Databases/DatabaseOrdinary.h | 2 +- dbms/src/Databases/DatabasesCommon.cpp | 8 ++++++++ dbms/src/Databases/DatabasesCommon.h | 3 ++- dbms/src/Databases/IDatabase.h | 2 +- .../src/Interpreters/tests/expression_analyzer.cpp | 4 ++-- dbms/src/Storages/IStorage.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 4 ++-- dbms/src/Storages/StorageNull.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 6 +++--- 21 files changed, 49 insertions(+), 28 deletions(-) diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index e52ffec4777..fc80c2c03e4 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -25,7 +25,7 @@ public: //void removeTable(const Context & context, const String & table_name) override; void dropTable(const Context & context, const String & table_name, bool no_delay) override; - void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path = {}) override; + void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; StoragePtr detachTable(const String & name) override; String getTableDataPath(const String & table_name) const override; diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 430e5ca47cd..b911687586e 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -41,7 +41,7 @@ void DatabaseLazy::loadStoredObjects( iterateMetadataFiles(context, [this](const String & file_name) { const std::string table_name = file_name.substr(0, file_name.size() - 4); - attachTable(table_name, nullptr); + attachTable(table_name, nullptr, {}); }); } @@ -95,7 +95,7 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name void DatabaseLazy::alterTable( const Context & /* context */, - const String & /* table_name */, + const StorageID & /*table_id*/, const StorageInMemoryMetadata & /* metadata */) { clearExpiredTables(); diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index ead7b85e020..ed2001bee1f 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -35,7 +35,7 @@ public: void dropTable( const Context & context, const String & table_name, - bool no_delay = false) override; + bool no_delay) override; void renameTable( const Context & context, @@ -45,7 +45,7 @@ public: void alterTable( const Context & context, - const String & name, + const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; time_t getObjectMetadataModificationTime(const String & table_name) const override; @@ -62,7 +62,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; - void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path = {}) override; + void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; StoragePtr detachTable(const String & table_name) override; diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index b96716dc8e3..aed67e42291 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -32,7 +32,7 @@ public: void dropTable( const Context & context, const String & table_name, - bool no_delay = false) override; + bool no_delay) override; ASTPtr getCreateTableQueryImpl(const Context & /*context*/, const String & name, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery(const Context & /*context*/) const override; diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index a2d05dad7b8..b3c62e97d7e 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -505,7 +505,7 @@ void DatabaseMySQL::createTable(const Context & context, const String & table_na throw Exception("The MySQL database engine can only execute attach statements of type attach table database_name.table_name", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - attachTable(table_name, storage); + attachTable(table_name, storage, {}); } } diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index cbe3a23eced..c74aadacbf8 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -52,9 +52,9 @@ public: StoragePtr detachTable(const String & table_name) override; - void dropTable(const Context &, const String & table_name, bool no_delay = false) override; + void dropTable(const Context &, const String & table_name, bool no_delay) override; - void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path = {}) override; + void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; protected: ASTPtr getCreateTableQueryImpl(const Context & context, const String & name, bool throw_on_error) const override; diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 293c3e2a254..55b24f9be9a 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -41,7 +41,7 @@ public: void dropTable( const Context & context, const String & table_name, - bool no_delay = false) override; + bool no_delay) override; void renameTable( const Context & context, diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 9afe3b06ce8..97688504d7f 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -37,6 +37,7 @@ namespace DB namespace ErrorCodes { + extern const int CANNOT_ASSIGN_ALTER; } @@ -218,9 +219,10 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool) void DatabaseOrdinary::alterTable( const Context & context, - const String & table_name, + const StorageID & table_id, const StorageInMemoryMetadata & metadata) { + String table_name = table_id.table_name; /// Read the definition of the table and replace the necessary parts with new ones. String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_tmp_path = table_metadata_path + ".tmp"; @@ -278,6 +280,16 @@ void DatabaseOrdinary::alterTable( out.close(); } + std::lock_guard lock{mutex}; + auto actual_table_id = getTableUnlocked(table_id.table_name)->getStorageID(); + if (table_id.database_name != actual_table_id.database_name || + table_id.table_name != actual_table_id.table_name || + table_id.uuid != actual_table_id.uuid) + { + Poco::File(table_metadata_tmp_path).remove(); + throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER); + } + try { /// rename atomically replaces the old file with the new one. diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index 620a0ddf868..8e2ce2d0774 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -24,7 +24,7 @@ public: void alterTable( const Context & context, - const String & name, + const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; protected: diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 8947f0520cf..37c5b95dfb2 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -147,4 +147,12 @@ DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase() } } +StoragePtr DatabaseWithOwnTablesBase::getTableUnlocked(const String & table_name) const +{ + auto it = tables.find(table_name); + if (it != tables.end()) + return it->second; + throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); +} + } diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 9f389fc4bb7..7f699c55422 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -29,7 +29,7 @@ public: bool empty(const Context & context) const override; - void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path = {}) override; + void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; StoragePtr detachTable(const String & table_name) override; @@ -49,6 +49,7 @@ protected: void attachTableUnlocked(const String & table_name, const StoragePtr & table, const String & relative_table_path = {}); StoragePtr detachTableUnlocked(const String & table_name); + StoragePtr getTableUnlocked(const String & table_name) const; }; } diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index d395161c730..2669edbc78d 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -227,7 +227,7 @@ public: /// You must call under the TableStructureLock of the corresponding table . If engine_modifier is empty, then engine does not change. virtual void alterTable( const Context & /*context*/, - const String & /*name*/, + const StorageID & /*table_id*/, const StorageInMemoryMetadata & /*metadata*/) { throw Exception(getEngineName() + ": alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index 2f8b6b2aef2..349c0ac7bf4 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -101,8 +101,8 @@ int main() auto system_database = std::make_shared("system"); DatabaseCatalog::instance().attachDatabase("system", system_database); //context.setCurrentDatabase("system"); - system_database->attachTable("one", StorageSystemOne::create("one")); - system_database->attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false)); + system_database->attachTable("one", StorageSystemOne::create("one"), {}); + system_database->attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false), {}); size_t success = 0; for (auto & entry : queries) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 45c4251e4cb..322c937aaeb 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -382,7 +382,7 @@ void IStorage::alter( auto table_id = getStorageID(); StorageInMemoryMetadata metadata = getInMemoryMetadata(); params.apply(metadata); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 217d474defe..ef860a9179e 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -738,7 +738,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, StorageInMemoryMetadata metadata = getInMemoryMetadata(); params.apply(metadata); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 4ef8f39d1f7..5665329d6a9 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -469,7 +469,7 @@ void StorageDistributed::alter(const AlterCommands & params, const Context & con checkAlterIsPossible(params, context.getSettingsRef()); StorageInMemoryMetadata metadata = getInMemoryMetadata(); params.apply(metadata); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 4e013b76671..0f32f4c073e 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -286,7 +286,7 @@ void StorageMaterializedView::alter( } /// end modify query - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 46e76a3fcde..862a7a2b87d 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -439,7 +439,7 @@ void StorageMerge::alter( StorageInMemoryMetadata storage_metadata = getInMemoryMetadata(); params.apply(storage_metadata); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, storage_metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, storage_metadata); setColumns(storage_metadata.columns); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index eafde9ec15c..74ca2fb4155 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -279,7 +279,7 @@ void StorageMergeTree::alter( { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); update_metadata(); } @@ -296,7 +296,7 @@ void StorageMergeTree::alter( lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); update_metadata(); diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 878be5bbf2d..10b9ecdac13 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -53,7 +53,7 @@ void StorageNull::alter( StorageInMemoryMetadata metadata = getInMemoryMetadata(); params.apply(metadata); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 43f51927692..d9b730ae668 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -494,7 +494,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column } auto table_id = getStorageID(); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(global_context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(global_context, table_id, metadata); /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. @@ -3270,7 +3270,7 @@ void StorageReplicatedMergeTree::alter( changeSettings(metadata.settings_ast, table_lock_holder); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(query_context, table_id, metadata); return; } @@ -3331,7 +3331,7 @@ void StorageReplicatedMergeTree::alter( /// Just change settings current_metadata.settings_ast = future_metadata.settings_ast; changeSettings(current_metadata.settings_ast, table_lock_holder); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, current_metadata); + DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(query_context, table_id, current_metadata); } /// We can be sure, that in case of successfull commit in zookeeper our From 221bb893435e36e55d6ca8d139f5c46b63161b1c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 23 Mar 2020 03:12:13 +0300 Subject: [PATCH 042/743] use renameat2 for metadata --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Common/rename.cpp | 67 ++++++++++++ dbms/src/Common/rename.h | 13 +++ dbms/src/Databases/DatabaseAtomic.cpp | 101 ++++++++++++++---- dbms/src/Databases/DatabaseAtomic.h | 3 + dbms/src/Databases/DatabaseOnDisk.cpp | 32 +++++- dbms/src/Databases/DatabaseOnDisk.h | 3 + dbms/src/Databases/DatabaseOrdinary.cpp | 21 +--- dbms/src/Interpreters/DatabaseCatalog.cpp | 11 ++ dbms/src/Interpreters/DatabaseCatalog.h | 3 + .../Interpreters/InterpreterCreateQuery.cpp | 23 +++- .../src/Interpreters/InterpreterDropQuery.cpp | 4 +- dbms/src/Parsers/ASTCreateQuery.h | 1 + 13 files changed, 237 insertions(+), 46 deletions(-) create mode 100644 dbms/src/Common/rename.cpp create mode 100644 dbms/src/Common/rename.h diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index dfb39f1624e..a897fba4de8 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -489,6 +489,7 @@ namespace ErrorCodes extern const int INCORRECT_ACCESS_ENTITY_DEFINITION = 515; extern const int AUTHENTICATION_FAILED = 516; extern const int CANNOT_ASSIGN_ALTER = 517; + extern const int ATOMIC_RENAME_FAIL = 518; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/rename.cpp b/dbms/src/Common/rename.cpp new file mode 100644 index 00000000000..c97666a8be1 --- /dev/null +++ b/dbms/src/Common/rename.cpp @@ -0,0 +1,67 @@ +#include +#include + +#if defined(_GNU_SOURCE) +#include +#include +#include +#endif + +namespace DB +{ + +#if defined(__NR_renameat2) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ATOMIC_RENAME_FAIL; + extern const int SYSTEM_ERROR; +} + +static void renameat2(const std::string & old_path, const std::string & new_path, int flags) +{ + if (old_path.empty() || new_path.empty()) + throw Exception("Path is empty", ErrorCodes::LOGICAL_ERROR); + if (old_path[0] != '/' || new_path[0] != '/') + throw Exception("Path is relative", ErrorCodes::LOGICAL_ERROR); + + /// int olddirfd (ignored for absolute oldpath), const char *oldpath, + /// int newdirfd (ignored for absolute newpath), const char *newpath, + /// unsigned int flags + if (0 == syscall(__NR_renameat2, 0, old_path.c_str(), 0, new_path.c_str(), flags)) + return; + + if (errno == EEXIST) + throwFromErrnoWithPath("Cannot RENAME_NOREPLACE " + old_path + " to " + new_path, new_path, ErrorCodes::ATOMIC_RENAME_FAIL); + if (errno == ENOENT) + throwFromErrno("Paths cannot be exchanged because " + old_path + " or " + new_path + " does not exist", ErrorCodes::ATOMIC_RENAME_FAIL); + throwFromErrnoWithPath("Cannot rename " + old_path + " to " + new_path, old_path, ErrorCodes::SYSTEM_ERROR); +} + +#else +#define RENAME_NOREPLACE 0 +#define RENAME_EXCHANGE 0 + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +static [[noreturn]] void renameat2(const std::string &, const std::string &, int) +{ + throw Exception("Compiled without renameat2() support", ErrorCodes::UNSUPPORTED_METHOD); +} + +#endif + +void renameNoReplace(const std::string & old_path, const std::string & new_path) +{ + renameat2(old_path, new_path, RENAME_NOREPLACE); +} + +void renameExchange(const std::string & old_path, const std::string & new_path) +{ + renameat2(old_path, new_path, RENAME_EXCHANGE); +} + +} diff --git a/dbms/src/Common/rename.h b/dbms/src/Common/rename.h new file mode 100644 index 00000000000..1b42754507e --- /dev/null +++ b/dbms/src/Common/rename.h @@ -0,0 +1,13 @@ +#pragma once +#include + +namespace DB +{ + +/// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception +void renameNoReplace(const std::string & old_path, const std::string & new_path); + +/// Atomically exchange oldpath and newpath. Throw exception if some of them does not exist +void renameExchange(const std::string & old_path, const std::string & new_path); + +} diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index b36191f19a9..da83a355b7a 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -14,7 +15,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int TABLE_ALREADY_EXISTS; extern const int FILE_DOESNT_EXIST; - extern const int DATABASE_NOT_EMPTY; + extern const int CANNOT_ASSIGN_ALTER; } DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_) @@ -50,30 +51,32 @@ void DatabaseAtomic::drop(const Context &) void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) { assert(relative_table_path != data_path && !relative_table_path.empty()); - DatabaseWithDictionaries::attachTable(name, table, relative_table_path); std::lock_guard lock(mutex); + DatabaseWithDictionaries::attachTableUnlocked(name, table, relative_table_path); table_name_to_path.emplace(std::make_pair(name, relative_table_path)); } StoragePtr DatabaseAtomic::detachTable(const String & name) { - { - std::lock_guard lock(mutex); - table_name_to_path.erase(name); - } - return DatabaseWithDictionaries::detachTable(name); + std::lock_guard lock(mutex); + auto table = DatabaseWithDictionaries::detachTableUnlocked(name); + table_name_to_path.erase(name); + return table; } void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool no_delay) { String table_metadata_path = getObjectMetadataPath(table_name); - - //FIXME - StoragePtr table = detachTable(table_name); - - String table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID()); - LOG_INFO(log, "Mark table " + table->getStorageID().getNameForLogs() + " to drop."); - Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); + String table_metadata_path_drop; + StoragePtr table; + { + std::lock_guard lock(mutex); + table = getTableUnlocked(table_name); + table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID()); + Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); + DatabaseWithDictionaries::detachTableUnlocked(table_name); /// Should never throw + table_name_to_path.erase(table_name); + } DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); } @@ -94,14 +97,38 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n if (!table) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - /// Update database and table name in memory without moving any data on disk - table->renameInMemory(to_database.getDatabaseName(), to_table_name); + String old_metadata_path = getObjectMetadataPath(table_name); + String new_metadata_path = to_database.getObjectMetadataPath(table_name); - /// NOTE Non-atomic. - auto path = getTableDataPath(table_name); - detachTable(table_name); - Poco::File(getObjectMetadataPath(table_name)).renameTo(to_database.getObjectMetadataPath(to_table_name)); - to_database.attachTable(to_table_name, table, path); + if (this == &to_database) + { + std::lock_guard lock(mutex); + renameNoReplace(old_metadata_path, new_metadata_path); + auto table_data_path = table_name_to_path.find(table_name)->second; + tables.erase(table_name); + table_name_to_path.erase(table_name); + table->renameInMemory(to_database.getDatabaseName(), to_table_name); + tables.emplace(to_table_name, table); + table_name_to_path.emplace(to_table_name, table_data_path); + } + else + { + String table_data_path; + { + std::lock_guard lock(mutex); + renameNoReplace(old_metadata_path, new_metadata_path); + table_data_path = table_name_to_path.find(table_name)->second; + tables.erase(table_name); + table_name_to_path.erase(table_name); + DatabaseCatalog::instance().updateUUIDMapping(table->getStorageID().uuid, to_database.shared_from_this(), table); + } + table->renameInMemory(to_database.getDatabaseName(), to_table_name); + auto & to_atomic_db = dynamic_cast(to_database); + + std::lock_guard lock(to_atomic_db.mutex); + to_atomic_db.tables.emplace(to_table_name, table); + to_atomic_db.table_name_to_path.emplace(to_table_name, table_data_path); + } } void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) @@ -114,5 +141,37 @@ void DatabaseAtomic::shutdown() DatabaseWithDictionaries::shutdown(); } +void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, + const String & table_metadata_tmp_path, const String & table_metadata_path) +{ + auto table_data_path = getTableDataPath(query); + try + { + std::lock_guard lock{mutex}; + renameNoReplace(table_metadata_tmp_path, table_metadata_path); + attachTableUnlocked(query.table, table, table_data_path); /// Should never throw + table_name_to_path.emplace(query.table, table_data_path); + } + catch (...) + { + Poco::File(table_metadata_tmp_path).remove(); + throw; + } + +} + +void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) +{ + SCOPE_EXIT({ Poco::File(table_metadata_tmp_path).remove(); }); + + std::lock_guard lock{mutex}; + auto actual_table_id = getTableUnlocked(table_id.table_name)->getStorageID(); + + if (table_id.uuid != actual_table_id.uuid) + throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER); + + renameExchange(table_metadata_tmp_path, table_metadata_path); +} + } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index fc80c2c03e4..c1c0a8fc75a 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -43,6 +43,9 @@ public: void shutdown() override; private: + void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override; + void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, + const String & table_metadata_tmp_path, const String & table_metadata_path) override; //TODO store path in DatabaseWithOwnTables::tables std::map table_name_to_path; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index e4469ec35eb..fa8b1266bd9 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -140,6 +140,8 @@ void DatabaseOnDisk::createTable( const ASTPtr & query) { const auto & settings = context.getSettingsRef(); + const auto & create = query->as(); + assert(getDatabaseName() == create.database && table_name == create.table); /// Create a file with metadata if necessary - if the query is not ATTACH. /// Write the query of `ATTACH table` to it. @@ -153,6 +155,7 @@ void DatabaseOnDisk::createTable( /// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH. /// But there is protection from it - see using DDLGuard in InterpreterCreateQuery. + if (isDictionaryExist(context, table_name)) throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); @@ -160,6 +163,13 @@ void DatabaseOnDisk::createTable( if (isTableExist(context, table_name)) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + if (create.attach_short_syntax) + { + /// Metadata already exists, table was detached + attachTable(table_name, table, getTableDataPath(create)); + return; + } + String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_tmp_path = table_metadata_path + create_suffix; String statement; @@ -176,10 +186,16 @@ void DatabaseOnDisk::createTable( out.close(); } + commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path); +} + +void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, + const String & table_metadata_tmp_path, const String & table_metadata_path) +{ try { /// Add a table to the map of known tables. - attachTable(table_name, table, getTableDataPath(query->as())); + attachTable(query.table, table, getTableDataPath(query)); /// If it was ATTACH query and file with table metadata already exist /// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one. @@ -475,4 +491,18 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const Context & context, const return ast; } +void DatabaseOnDisk::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path) +{ + try + { + /// rename atomically replaces the old file with the new one. + Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path); + } + catch (...) + { + Poco::File(table_metadata_tmp_path).remove(); + throw; + } +} + } diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 55b24f9be9a..c2d3f5392d2 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -81,6 +81,9 @@ protected: ASTPtr getCreateQueryFromMetadata(const Context & context, const String & metadata_path, bool throw_on_error) const; + virtual void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path); + virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, + const String & table_metadata_tmp_path, const String & table_metadata_path); //bool detachTableAndRemoveMetadata(const String & table_name); //void replaceMetadata(const ASTPtr & create, ); diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 97688504d7f..72736d8f25f 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -280,26 +280,7 @@ void DatabaseOrdinary::alterTable( out.close(); } - std::lock_guard lock{mutex}; - auto actual_table_id = getTableUnlocked(table_id.table_name)->getStorageID(); - if (table_id.database_name != actual_table_id.database_name || - table_id.table_name != actual_table_id.table_name || - table_id.uuid != actual_table_id.uuid) - { - Poco::File(table_metadata_tmp_path).remove(); - throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER); - } - - try - { - /// rename atomically replaces the old file with the new one. - Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path); - } - catch (...) - { - Poco::File(table_metadata_tmp_path).remove(); - throw; - } + commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path); } } diff --git a/dbms/src/Interpreters/DatabaseCatalog.cpp b/dbms/src/Interpreters/DatabaseCatalog.cpp index bda1a75349a..e1191edbd89 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.cpp +++ b/dbms/src/Interpreters/DatabaseCatalog.cpp @@ -358,6 +358,17 @@ void DatabaseCatalog::removeUUIDMapping(const UUID & uuid) throw Exception("Mapping for table with UUID=" + toString(uuid) + " doesn't exist", ErrorCodes::LOGICAL_ERROR); } +void DatabaseCatalog::updateUUIDMapping(const UUID & uuid, DatabasePtr database, StoragePtr table) +{ + assert(uuid != UUIDHelpers::Nil && getFirstLevelIdx(uuid) < uuid_map.size()); + UUIDToStorageMapPart & map_part = uuid_map[getFirstLevelIdx(uuid)]; + std::lock_guard lock{map_part.mutex}; + auto it = map_part.map.find(uuid); + if (it == map_part.map.end()) + throw Exception("Mapping for table with UUID=" + toString(uuid) + " doesn't exist", ErrorCodes::LOGICAL_ERROR); + it->second = std::make_pair(std::move(database), std::move(table)); +} + DatabaseCatalog::DatabaseCatalog(Context * global_context_) : global_context(global_context_), log(&Poco::Logger::get("DatabaseCatalog")) { diff --git a/dbms/src/Interpreters/DatabaseCatalog.h b/dbms/src/Interpreters/DatabaseCatalog.h index bf8889da393..8d800faaebc 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.h +++ b/dbms/src/Interpreters/DatabaseCatalog.h @@ -151,6 +151,9 @@ public: /// Such tables can be accessed by persistent UUID instead of database and table name. void addUUIDMapping(const UUID & uuid, DatabasePtr database, StoragePtr table); void removeUUIDMapping(const UUID & uuid); + /// For moving table between databases + void updateUUIDMapping(const UUID & uuid, DatabasePtr database, StoragePtr table); + DatabaseAndTable tryGetByUUID(const UUID & uuid) const; String getPathForDroppedMetadata(const StorageID & table_id) const; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index e513430e69d..beca15e6ca0 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -539,15 +539,31 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String current_database = context.getCurrentDatabase(); + String metadata_path_tmp; + bool created_tmp = false; + SCOPE_EXIT({ if (created_tmp) Poco::File(metadata_path_tmp).remove(); }); + // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) { + auto database_name = create.database.empty() ? current_database : create.database; + auto database = DatabaseCatalog::instance().getDatabase(database_name); + + //TODO do we really need it? refactor it if we do + if (database->getEngineName() == "Atomic") + { + metadata_path_tmp = database->getObjectMetadataPath(create.table) + ".tmp"; + if (!Poco::File(metadata_path_tmp).createFile()) + throw Exception("Cannot attach table because of concurrent query.", ErrorCodes::TABLE_ALREADY_EXISTS); + created_tmp = true; + } + bool if_not_exists = create.if_not_exists; // Table SQL definition is available even if the table is detached - auto database_name = create.database.empty() ? current_database : create.database; - auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateTableQuery(context, create.table); + auto query = database->getCreateTableQuery(context, create.table); create = query->as(); // Copy the saved create query, but use ATTACH instead of CREATE create.attach = true; + create.attach_short_syntax = true; create.if_not_exists = if_not_exists; } @@ -606,7 +622,8 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. */ - guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name); + if (database->getEngineName() != "Atomic") + guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name); /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. if (database->isTableExist(context, table_name)) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 0a65d2806a5..c0780499965 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -75,7 +75,9 @@ BlockIO InterpreterDropQuery::executeToTable( auto table_id = context.resolveStorageID(table_id_, Context::ResolveOrdinary); - auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); + std::unique_ptr ddl_guard; + if (DatabaseCatalog::instance().getDatabase(table_id.database_name)->getEngineName() != "Atomic") + ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); auto [database, table] = tryGetDatabaseAndTable(table_id.database_name, table_id.table_name, query.if_exists); diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 8b018ed3fe9..7cf956ef790 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -74,6 +74,7 @@ public: ASTSelectWithUnionQuery * select = nullptr; ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.) std::optional live_view_timeout; /// For CREATE LIVE VIEW ... WITH TIMEOUT ... + bool attach_short_syntax{false}; /** Get the text that identifies this element. */ String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + database) + delim + table; } From b47db5e0d625ac47f887dcf7d286039ab9893085 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 24 Mar 2020 01:40:40 +0300 Subject: [PATCH 043/743] fix --- dbms/src/Common/rename.cpp | 2 +- dbms/src/Databases/DatabaseAtomic.cpp | 2 +- dbms/src/Databases/DatabaseOnDisk.cpp | 1 + dbms/src/Storages/IStorage.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 4 ++-- dbms/src/Storages/StorageNull.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 6 +++--- .../0_stateless/01071_live_view_detach_dependency.sql | 2 ++ 12 files changed, 16 insertions(+), 13 deletions(-) diff --git a/dbms/src/Common/rename.cpp b/dbms/src/Common/rename.cpp index c97666a8be1..039c4304f69 100644 --- a/dbms/src/Common/rename.cpp +++ b/dbms/src/Common/rename.cpp @@ -47,7 +47,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } -static [[noreturn]] void renameat2(const std::string &, const std::string &, int) +[[noreturn]] static void renameat2(const std::string &, const std::string &, int) { throw Exception("Compiled without renameat2() support", ErrorCodes::UNSUPPORTED_METHOD); } diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index da83a355b7a..72b7ae05dd8 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -98,7 +98,7 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); String old_metadata_path = getObjectMetadataPath(table_name); - String new_metadata_path = to_database.getObjectMetadataPath(table_name); + String new_metadata_path = to_database.getObjectMetadataPath(to_table_name); if (this == &to_database) { diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index fa8b1266bd9..6c52f5160bc 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -272,6 +272,7 @@ void DatabaseOnDisk::renameTable( table_metadata_path = getObjectMetadataPath(table_name); attach_query = parseQueryFromMetadata(log, context, table_metadata_path); auto & create = attach_query->as(); + create.database = to_database.getDatabaseName(); create.table = to_table_name; if (from_ordinary_to_atomic) create.uuid = UUIDHelpers::generateV4(); diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 322c937aaeb..5db5b8de2d0 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -382,7 +382,7 @@ void IStorage::alter( auto table_id = getStorageID(); StorageInMemoryMetadata metadata = getInMemoryMetadata(); params.apply(metadata); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index ef860a9179e..6fba8d46646 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -738,7 +738,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, StorageInMemoryMetadata metadata = getInMemoryMetadata(); params.apply(metadata); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 5665329d6a9..af5e613bb37 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -469,7 +469,7 @@ void StorageDistributed::alter(const AlterCommands & params, const Context & con checkAlterIsPossible(params, context.getSettingsRef()); StorageInMemoryMetadata metadata = getInMemoryMetadata(); params.apply(metadata); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 0f32f4c073e..9a82dfa2047 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -286,7 +286,7 @@ void StorageMaterializedView::alter( } /// end modify query - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 862a7a2b87d..ccb08971e5e 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -439,7 +439,7 @@ void StorageMerge::alter( StorageInMemoryMetadata storage_metadata = getInMemoryMetadata(); params.apply(storage_metadata); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, storage_metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, storage_metadata); setColumns(storage_metadata.columns); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 74ca2fb4155..8e1523ed14a 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -279,7 +279,7 @@ void StorageMergeTree::alter( { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata); update_metadata(); } @@ -296,7 +296,7 @@ void StorageMergeTree::alter( lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata); update_metadata(); diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 10b9ecdac13..206506da02f 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -53,7 +53,7 @@ void StorageNull::alter( StorageInMemoryMetadata metadata = getInMemoryMetadata(); params.apply(metadata); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata); setColumns(std::move(metadata.columns)); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index d9b730ae668..029e903fc8e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -494,7 +494,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column } auto table_id = getStorageID(); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(global_context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(global_context, table_id, metadata); /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. @@ -3270,7 +3270,7 @@ void StorageReplicatedMergeTree::alter( changeSettings(metadata.settings_ast, table_lock_holder); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(query_context, table_id, metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, metadata); return; } @@ -3331,7 +3331,7 @@ void StorageReplicatedMergeTree::alter( /// Just change settings current_metadata.settings_ast = future_metadata.settings_ast; changeSettings(current_metadata.settings_ast, table_lock_holder); - DatabaseCatalog::instance().getDatabaseAndTable(table_id).first->alterTable(query_context, table_id, current_metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, current_metadata); } /// We can be sure, that in case of successfull commit in zookeeper our diff --git a/dbms/tests/queries/0_stateless/01071_live_view_detach_dependency.sql b/dbms/tests/queries/0_stateless/01071_live_view_detach_dependency.sql index 22d8adc503c..e16b8d67bc7 100644 --- a/dbms/tests/queries/0_stateless/01071_live_view_detach_dependency.sql +++ b/dbms/tests/queries/0_stateless/01071_live_view_detach_dependency.sql @@ -6,3 +6,5 @@ CREATE LIVE VIEW lv AS SELECT * FROM test; DETACH TABLE lv; INSERT INTO test VALUES (42); DROP TABLE test; +ATTACH TABLE lv; +DROP TABLE lv; From d0b37a5a7049c8e6e404bd8fe5c3cc6147704e9f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 24 Mar 2020 22:03:37 +0300 Subject: [PATCH 044/743] fix --- dbms/src/Databases/DatabaseOnDisk.cpp | 1 + dbms/src/Interpreters/InterpreterDropQuery.cpp | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 6c52f5160bc..fceb9d06fd5 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -281,6 +281,7 @@ void DatabaseOnDisk::renameTable( /// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID table->rename(to_database.getTableDataPath(create), to_database.getDatabaseName(), to_table_name, table_lock); + table->resetStorageIDForTableFunction({create.database, create.table, create.uuid}); /// reset UUID } catch (const Exception &) { diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 59308e3f5d5..7705a859938 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -76,8 +76,9 @@ BlockIO InterpreterDropQuery::executeToTable( auto table_id = context.resolveStorageID(table_id_, Context::ResolveOrdinary); std::unique_ptr ddl_guard; - if (DatabaseCatalog::instance().getDatabase(table_id.database_name)->getEngineName() != "Atomic") - ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); + if (auto db = DatabaseCatalog::instance().tryGetDatabase(table_id.database_name)) //FIXME + if (db->getEngineName() != "Atomic") + ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); auto [database, table] = tryGetDatabaseAndTable(table_id.database_name, table_id.table_name, query.if_exists); From e6b10b81f018f68af28edc7366ed0d238cb36339 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Mar 2020 17:10:47 +0300 Subject: [PATCH 045/743] Refactor MergingSortedTransform. Add IMergingTransform. --- dbms/CMakeLists.txt | 1 + .../Interpreters/InterpreterSelectQuery.cpp | 2 +- .../Processors/Merges/IMergingTransform.cpp | 211 +++++++++++++++++ .../IMergingTransform.h} | 91 +++----- .../MergingSortedTransform.cpp | 213 ++++-------------- .../Merges/MergingSortedTransform.h | 62 +++++ .../Transforms/MergeSortingTransform.cpp | 2 +- ...ocessors_test_merging_sorted_transform.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 9 files changed, 352 insertions(+), 234 deletions(-) create mode 100644 dbms/src/Processors/Merges/IMergingTransform.cpp rename dbms/src/Processors/{Transforms/MergingSortedTransform.h => Merges/IMergingTransform.h} (53%) rename dbms/src/Processors/{Transforms => Merges}/MergingSortedTransform.cpp (50%) create mode 100644 dbms/src/Processors/Merges/MergingSortedTransform.h diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index b254c986511..c121ed00dd1 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -299,6 +299,7 @@ add_object_library(clickhouse_processors_formats src/Processors/Formats) add_object_library(clickhouse_processors_formats_impl src/Processors/Formats/Impl) add_object_library(clickhouse_processors_transforms src/Processors/Transforms) add_object_library(clickhouse_processors_sources src/Processors/Sources) +add_object_library(clickhouse_processors_sources src/Processors/Merges) if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 4fe83afa48d..f3100dce19c 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -84,7 +84,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Processors/Merges/IMergingTransform.cpp b/dbms/src/Processors/Merges/IMergingTransform.cpp new file mode 100644 index 00000000000..d22acb71521 --- /dev/null +++ b/dbms/src/Processors/Merges/IMergingTransform.cpp @@ -0,0 +1,211 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; +} + +IMergingTransform::IMergingTransform( + size_t num_inputs, const Block & input_header, const Block & output_header, bool have_all_inputs_) + : IProcessor(InputPorts(num_inputs, input_header), {output_header}) + , merged_data(output_header), have_all_inputs(have_all_inputs_) +{ +} + +void IMergingTransform::onNewInput() +{ + throw Exception("onNewInput is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); +} + +void IMergingTransform::addInput() +{ + if (have_all_inputs) + throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); + + inputs.emplace_back(outputs.front().getHeader(), this); + onNewInput(); +} + +void IMergingTransform::setHaveAllInputs() +{ + if (have_all_inputs) + throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); + + have_all_inputs = true; +} + +void IMergingTransform::requestDataForInput(size_t input_number) +{ + if (!need_data) + throw Exception("Data was requested for several inputs in IMergingTransform:" + " " + std::to_string(next_input_to_read) + " and " + std::to_string(input_number), + ErrorCodes::LOGICAL_ERROR); + + need_data = true; + next_input_to_read = input_number; +} + +IProcessor::Status IMergingTransform::prepareSingleInput() +{ + auto & input = inputs.front(); + auto & output = outputs.front(); + + if (input.isFinished()) + { + output.finish(); + return Status::Finished; + } + + input.setNeeded(); + + if (input.hasData()) + { + if (output.canPush()) + output.push(input.pull()); + + return Status::PortFull; + } + + return Status::NeedData; +} + +IProcessor::Status IMergingTransform::prepareInitializeInputs() +{ + /// Add information about inputs. + if (input_states.empty()) + { + input_states.reserve(inputs.size()); + for (auto & input : inputs) + input_states.emplace_back(input); + } + + /// Check for inputs we need. + bool all_inputs_has_data = true; + auto it = inputs.begin(); + for (size_t i = 0; it != inputs.end(); ++i, ++it) + { + auto & input = *it; + if (input.isFinished()) + continue; + + if (!input_states[i].is_initialized) + { + // input.setNotNeeded(); + continue; + } + + input.setNeeded(); + + if (!input.hasData()) + { + all_inputs_has_data = false; + continue; + } + + auto chunk = input.pull(); + if (!chunk.hasRows()) + { + + if (!input.isFinished()) + all_inputs_has_data = false; + + continue; + } + + consume(std::move(chunk), i); + input_states[i].is_initialized = true; + } + + if (!all_inputs_has_data) + return Status::NeedData; + + initializeInputs(); + + is_initialized = true; + return Status::Ready; +} + + +IProcessor::Status IMergingTransform::prepare() +{ + if (!have_all_inputs) + return Status::NeedData; + + auto & output = outputs.front(); + + /// Special case for no inputs. + if (inputs.empty()) + { + output.finish(); + return Status::Finished; + } + + /// Check can output. + + if (output.isFinished()) + { + for (auto & in : inputs) + in.close(); + + return Status::Finished; + } + + /// Special case for single input. + if (inputs.size() == 1) + return prepareSingleInput(); + + /// Do not disable inputs, so it will work in the same way as with AsynchronousBlockInputStream, like before. + bool is_port_full = !output.canPush(); + + /// Push if has data. + if (merged_data.mergedRows() && !is_port_full) + output.push(merged_data.pull()); + + if (!is_initialized) + return prepareInitializeInputs(); + + if (is_finished) + { + + if (is_port_full) + return Status::PortFull; + + for (auto & input : inputs) + input.close(); + + outputs.front().finish(); + + return Status::Finished; + } + + if (need_data) + { + auto & input = input_states[next_input_to_read].port; + if (!input.isFinished()) + { + input.setNeeded(); + + if (!input.hasData()) + return Status::NeedData; + + auto chunk = input.pull(); + if (!chunk.hasRows() && !input.isFinished()) + return Status::NeedData; + + consume(std::move(chunk), next_input_to_read); + } + + need_data = false; + } + + if (is_port_full) + return Status::PortFull; + + return Status::Ready; +} + +} diff --git a/dbms/src/Processors/Transforms/MergingSortedTransform.h b/dbms/src/Processors/Merges/IMergingTransform.h similarity index 53% rename from dbms/src/Processors/Transforms/MergingSortedTransform.h rename to dbms/src/Processors/Merges/IMergingTransform.h index 914ad543c75..5c5a7bad1f7 100644 --- a/dbms/src/Processors/Transforms/MergingSortedTransform.h +++ b/dbms/src/Processors/Merges/IMergingTransform.h @@ -1,38 +1,34 @@ #pragma once #include -#include -#include - namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} -class MergingSortedTransform : public IProcessor +/// Base class for merging transforms. +class IMergingTransform : public IProcessor { public: - MergingSortedTransform( - const Block & header, - size_t num_inputs, - const SortDescription & description_, - size_t max_block_size, - UInt64 limit = 0, - bool quiet = false, - bool have_all_inputs = true); - - String getName() const override { return "MergingSortedTransform"; } - Status prepare() override; - void work() override; + IMergingTransform(size_t num_inputs, const Block & input_header, const Block & output_header, bool have_all_inputs); + /// Methods to add additional input port. It is possible to do only before the first call of `prepare`. void addInput(); + /// Need to be called after all inputs are added. (only if have_all_inputs was not specified). void setHaveAllInputs(); + Status prepare() override; + protected: + virtual void onNewInput(); /// Is called when new input is added. To initialize input's data. + virtual void initializeInputs() = 0; /// Is called after first chunk was read for every input. + virtual void consume(Chunk chunk, size_t input_number) = 0; /// Is called after chunk was consumed from input. + + void requestDataForInput(size_t input_number); /// Call it to say that next chunk of data is required for input. + void finish() { is_finished = true; } /// Call it when all data was inserted to merged_data. + + /// Struct which represents current merging chunk of data. + /// Also it calculates the number of merged rows. class MergedData { public: @@ -96,65 +92,30 @@ protected: MutableColumns columns; }; - /// Settings - SortDescription description; - const size_t max_block_size; - UInt64 limit; - bool has_collation = false; - bool quiet = false; - - std::atomic have_all_inputs; - MergedData merged_data; - /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) - /// If it is not nullptr then it should be populated during execution - WriteBuffer * out_row_sources_buf = nullptr; - - /// Chunks currently being merged. - std::vector source_chunks; - - SortCursorImpls cursors; - - SortingHeap queue_without_collation; - SortingHeap queue_with_collation; - private: - /// Processor state. bool is_initialized = false; bool is_finished = false; + bool need_data = false; size_t next_input_to_read = 0; - template - void merge(TSortingHeap & queue); + std::atomic have_all_inputs; - void insertFromChunk(size_t source_num); - - void updateCursor(Chunk chunk, size_t source_num) + struct InputState { - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); + explicit InputState(InputPort & port_) : port(port_) {} - chunk.setColumns(std::move(columns), num_rows); + InputPort & port; + bool is_initialized = false; + }; - auto & source_chunk = source_chunks[source_num]; + std::vector input_states; - if (source_chunk.empty()) - { - source_chunk = std::move(chunk); - cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num); - has_collation |= cursors[source_num].has_collation; - } - else - { - source_chunk = std::move(chunk); - cursors[source_num].reset(source_chunk.getColumns(), {}); - } - } + Status prepareSingleInput(); + Status prepareInitializeInputs(); }; } diff --git a/dbms/src/Processors/Transforms/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp similarity index 50% rename from dbms/src/Processors/Transforms/MergingSortedTransform.cpp rename to dbms/src/Processors/Merges/MergingSortedTransform.cpp index b9e74277023..c259ce05d76 100644 --- a/dbms/src/Processors/Transforms/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -1,10 +1,11 @@ -#include +#include #include #include #include namespace DB { + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -13,15 +14,14 @@ namespace ErrorCodes MergingSortedTransform::MergingSortedTransform( const Block & header, size_t num_inputs, - const SortDescription & description_, + SortDescription description_, size_t max_block_size_, UInt64 limit_, bool quiet_, bool have_all_inputs_) - : IProcessor(InputPorts(num_inputs, header), {header}) - , description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) - , have_all_inputs(have_all_inputs_) - , merged_data(header), source_chunks(num_inputs), cursors(num_inputs) + : IMergingTransform(num_inputs, header, header, have_all_inputs_) + , description(std::move(description_)), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) + , source_chunks(num_inputs), cursors(num_inputs) { auto & sample = outputs.front().getHeader(); /// Replace column names in description to positions. @@ -36,172 +36,56 @@ MergingSortedTransform::MergingSortedTransform( } } -void MergingSortedTransform::addInput() +void MergingSortedTransform::onNewInput() { - if (have_all_inputs) - throw Exception("MergingSortedTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); - - inputs.emplace_back(outputs.front().getHeader(), this); source_chunks.emplace_back(); cursors.emplace_back(); } -void MergingSortedTransform::setHaveAllInputs() +void MergingSortedTransform::initializeInputs() { - if (have_all_inputs) - throw Exception("MergingSortedTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); + if (has_collation) + queue_with_collation = SortingHeap(cursors); + else + queue_without_collation = SortingHeap(cursors); - have_all_inputs = true; + is_queue_initialized = true; } -IProcessor::Status MergingSortedTransform::prepare() +void MergingSortedTransform::consume(Chunk chunk, size_t input_number) { - if (!have_all_inputs) - return Status::NeedData; + updateCursor(std::move(chunk), input_number); - auto & output = outputs.front(); - - /// Special case for no inputs. - if (inputs.empty()) + if (is_queue_initialized) { - output.finish(); - return Status::Finished; - } - - /// Check can output. - - if (output.isFinished()) - { - for (auto & in : inputs) - in.close(); - - return Status::Finished; - } - - /// Do not disable inputs, so it will work in the same way as with AsynchronousBlockInputStream, like before. - bool is_port_full = !output.canPush(); - - /// Special case for single input. - if (inputs.size() == 1) - { - auto & input = inputs.front(); - if (input.isFinished()) - { - output.finish(); - return Status::Finished; - } - - input.setNeeded(); - - if (input.hasData()) - { - if (!is_port_full) - output.push(input.pull()); - - return Status::PortFull; - } - - return Status::NeedData; - } - - /// Push if has data. - if (merged_data.mergedRows() && !is_port_full) - output.push(merged_data.pull()); - - if (!is_initialized) - { - /// Check for inputs we need. - bool all_inputs_has_data = true; - auto it = inputs.begin(); - for (size_t i = 0; it != inputs.end(); ++i, ++it) - { - auto & input = *it; - if (input.isFinished()) - continue; - - if (!cursors[i].empty()) - { - // input.setNotNeeded(); - continue; - } - - input.setNeeded(); - - if (!input.hasData()) - { - all_inputs_has_data = false; - continue; - } - - auto chunk = input.pull(); - if (!chunk.hasRows()) - { - - if (!input.isFinished()) - all_inputs_has_data = false; - - continue; - } - - updateCursor(std::move(chunk), i); - } - - if (!all_inputs_has_data) - return Status::NeedData; - if (has_collation) - queue_with_collation = SortingHeap(cursors); + queue_with_collation.push(cursors[input_number]); else - queue_without_collation = SortingHeap(cursors); + queue_without_collation.push(cursors[input_number]); + } +} - is_initialized = true; - return Status::Ready; +void MergingSortedTransform::updateCursor(Chunk chunk, size_t source_num) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); + + auto & source_chunk = source_chunks[source_num]; + + if (source_chunk.empty()) + { + source_chunk = std::move(chunk); + cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num); + has_collation |= cursors[source_num].has_collation; } else { - if (is_finished) - { - - if (is_port_full) - return Status::PortFull; - - for (auto & input : inputs) - input.close(); - - outputs.front().finish(); - - return Status::Finished; - } - - if (need_data) - { - auto & input = *std::next(inputs.begin(), next_input_to_read); - if (!input.isFinished()) - { - input.setNeeded(); - - if (!input.hasData()) - return Status::NeedData; - - auto chunk = input.pull(); - if (!chunk.hasRows() && !input.isFinished()) - return Status::NeedData; - - updateCursor(std::move(chunk), next_input_to_read); - - if (has_collation) - queue_with_collation.push(cursors[next_input_to_read]); - else - queue_without_collation.push(cursors[next_input_to_read]); - } - - need_data = false; - } - - if (is_port_full) - return Status::PortFull; - - return Status::Ready; + source_chunk = std::move(chunk); + cursors[source_num].reset(source_chunk.getColumns(), {}); } } @@ -222,7 +106,7 @@ void MergingSortedTransform::merge(TSortingHeap & queue) if (limit && merged_data.totalMergedRows() >= limit) { //std::cerr << "Limit reached\n"; - is_finished = true; + finish(); return false; } @@ -274,7 +158,7 @@ void MergingSortedTransform::merge(TSortingHeap & queue) if (!current->isLast()) { -// std::cerr << "moving to next row\n"; + //std::cerr << "moving to next row\n"; queue.next(); } else @@ -282,17 +166,17 @@ void MergingSortedTransform::merge(TSortingHeap & queue) /// We will get the next block from the corresponding source, if there is one. queue.removeTop(); -// std::cerr << "It was last row, fetching next block\n"; - need_data = true; - next_input_to_read = current.impl->order; + //std::cerr << "It was last row, fetching next block\n"; + requestDataForInput(current.impl->order); if (limit && merged_data.totalMergedRows() >= limit) - is_finished = true; + finish(); return; } } - is_finished = true; + + finish(); } void MergingSortedTransform::insertFromChunk(size_t source_num) @@ -309,14 +193,14 @@ void MergingSortedTransform::insertFromChunk(size_t source_num) { num_rows = total_merged_rows_after_insertion - limit; merged_data.insertFromChunk(std::move(source_chunks[source_num]), num_rows); - is_finished = true; + finish(); } else { merged_data.insertFromChunk(std::move(source_chunks[source_num]), 0); - need_data = true; - next_input_to_read = source_num; + requestDataForInput(source_num); } + source_chunks[source_num] = Chunk(); if (out_row_sources_buf) @@ -327,5 +211,4 @@ void MergingSortedTransform::insertFromChunk(size_t source_num) } } - } diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.h b/dbms/src/Processors/Merges/MergingSortedTransform.h new file mode 100644 index 00000000000..1fac794b7d0 --- /dev/null +++ b/dbms/src/Processors/Merges/MergingSortedTransform.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/// Merges several sorted inputs into one sorted output. +class MergingSortedTransform : public IMergingTransform +{ +public: + MergingSortedTransform( + const Block & header, + size_t num_inputs, + SortDescription description, + size_t max_block_size, + UInt64 limit = 0, + bool quiet = false, + bool have_all_inputs = true); + + String getName() const override { return "MergingSortedTransform"; } + void work() override; + +protected: + + void onNewInput() override; + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + + /// Settings + SortDescription description; + const size_t max_block_size; + UInt64 limit; + bool has_collation = false; + bool quiet = false; + + /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) + /// If it is not nullptr then it should be populated during execution + WriteBuffer * out_row_sources_buf = nullptr; + + /// Chunks currently being merged. + std::vector source_chunks; + + SortCursorImpls cursors; + + SortingHeap queue_without_collation; + SortingHeap queue_with_collation; + bool is_queue_initialized = false; + +private: + + template + void merge(TSortingHeap & queue); + + void insertFromChunk(size_t source_num); + void updateCursor(Chunk chunk, size_t source_num); +}; + +} diff --git a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp index 060d860b0b5..7c447a24b8e 100644 --- a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/src/Processors/tests/processors_test_merging_sorted_transform.cpp b/dbms/src/Processors/tests/processors_test_merging_sorted_transform.cpp index 71d282585fd..477626d165d 100644 --- a/dbms/src/Processors/tests/processors_test_merging_sorted_transform.cpp +++ b/dbms/src/Processors/tests/processors_test_merging_sorted_transform.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 629a2b2cc18..10db375a6cf 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -58,7 +58,7 @@ namespace std #include #include #include -#include +#include #include #include From 0c12117f189bda531e7d5946c58b6c8f54c5ff7a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Mar 2020 20:27:48 +0300 Subject: [PATCH 046/743] Refactor MergingSortedTransform. Add IMergingTransform. --- .../Processors/Merges/IMergingTransform.cpp | 27 +++++++++-- .../src/Processors/Merges/IMergingTransform.h | 48 +++++++++++++++++-- .../Merges/MergingSortedTransform.cpp | 48 ++++++++++++++----- .../Merges/MergingSortedTransform.h | 11 ++--- .../Transforms/MergeSortingTransform.cpp | 2 + 5 files changed, 111 insertions(+), 25 deletions(-) diff --git a/dbms/src/Processors/Merges/IMergingTransform.cpp b/dbms/src/Processors/Merges/IMergingTransform.cpp index d22acb71521..92ad592e11f 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.cpp +++ b/dbms/src/Processors/Merges/IMergingTransform.cpp @@ -10,9 +10,15 @@ namespace ErrorCodes } IMergingTransform::IMergingTransform( - size_t num_inputs, const Block & input_header, const Block & output_header, bool have_all_inputs_) + size_t num_inputs, + const Block & input_header, + const Block & output_header, + size_t max_block_size, + bool use_average_block_size, + bool have_all_inputs_) : IProcessor(InputPorts(num_inputs, input_header), {output_header}) - , merged_data(output_header), have_all_inputs(have_all_inputs_) + , merged_data(output_header, use_average_block_size, max_block_size) + , have_all_inputs(have_all_inputs_) { } @@ -57,6 +63,7 @@ IProcessor::Status IMergingTransform::prepareSingleInput() if (input.isFinished()) { output.finish(); + onFinish(); return Status::Finished; } @@ -141,6 +148,7 @@ IProcessor::Status IMergingTransform::prepare() if (inputs.empty()) { output.finish(); + onFinish(); return Status::Finished; } @@ -151,6 +159,7 @@ IProcessor::Status IMergingTransform::prepare() for (auto & in : inputs) in.close(); + onFinish(); return Status::Finished; } @@ -162,8 +171,17 @@ IProcessor::Status IMergingTransform::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - if (merged_data.mergedRows() && !is_port_full) - output.push(merged_data.pull()); + bool has_data_to_push = (is_finished && merged_data.mergedRows()) || merged_data.hasEnoughRows(); + if (has_data_to_push && !is_port_full) + { + auto chunk = merged_data.pull(); + + ++total_chunks; + total_rows += chunk.getNumRows(); + total_bytes += chunk.allocatedBytes(); + + output.push(std::move(chunk)); + } if (!is_initialized) return prepareInitializeInputs(); @@ -179,6 +197,7 @@ IProcessor::Status IMergingTransform::prepare() outputs.front().finish(); + onFinish(); return Status::Finished; } diff --git a/dbms/src/Processors/Merges/IMergingTransform.h b/dbms/src/Processors/Merges/IMergingTransform.h index 5c5a7bad1f7..5680b92b6d3 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.h +++ b/dbms/src/Processors/Merges/IMergingTransform.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -9,7 +10,13 @@ namespace DB class IMergingTransform : public IProcessor { public: - IMergingTransform(size_t num_inputs, const Block & input_header, const Block & output_header, bool have_all_inputs); + IMergingTransform( + size_t num_inputs, + const Block & input_header, + const Block & output_header, + size_t max_block_size, + bool use_average_block_size, /// For adaptive granularity. Return chunks with the same avg size as inputs. + bool have_all_inputs_); /// Methods to add additional input port. It is possible to do only before the first call of `prepare`. void addInput(); @@ -23,6 +30,7 @@ protected: virtual void onNewInput(); /// Is called when new input is added. To initialize input's data. virtual void initializeInputs() = 0; /// Is called after first chunk was read for every input. virtual void consume(Chunk chunk, size_t input_number) = 0; /// Is called after chunk was consumed from input. + virtual void onFinish() {} /// Is called when all data is processed. void requestDataForInput(size_t input_number); /// Call it to say that next chunk of data is required for input. void finish() { is_finished = true; } /// Call it when all data was inserted to merged_data. @@ -32,14 +40,15 @@ protected: class MergedData { public: - explicit MergedData(const Block & header) + explicit MergedData(const Block & header, bool use_average_block_size_, UInt64 max_block_size_) + : max_block_size(max_block_size_), use_average_block_size(use_average_block_size_) { columns.reserve(header.columns()); for (const auto & column : header) columns.emplace_back(column.type->createColumn()); } - void insertRow(const ColumnRawPtrs & raw_columns, size_t row) + void insertRow(const ColumnRawPtrs & raw_columns, size_t row, size_t block_size) { size_t num_columns = raw_columns.size(); for (size_t i = 0; i < num_columns; ++i) @@ -47,6 +56,7 @@ protected: ++total_merged_rows; ++merged_rows; + sum_blocks_granularity += block_size; } void insertFromChunk(Chunk && chunk, size_t limit_rows) @@ -56,6 +66,7 @@ protected: ErrorCodes::LOGICAL_ERROR); auto num_rows = chunk.getNumRows(); + auto block_size = num_rows; columns = chunk.mutateColumns(); if (limit_rows && num_rows > limit_rows) { @@ -66,6 +77,7 @@ protected: total_merged_rows += num_rows; merged_rows = num_rows; + sum_blocks_granularity += block_size * num_rows; } Chunk pull() @@ -78,22 +90,52 @@ protected: empty_columns.swap(columns); Chunk chunk(std::move(empty_columns), merged_rows); + merged_rows = 0; + sum_blocks_granularity = 0; return chunk; } + bool hasEnoughRows() const + { + /// Never return more then max_block_size. + if (merged_rows >= max_block_size) + return true; + + if (!use_average_block_size) + return false; + + /// Zero rows always not enough. + if (merged_rows == 0) + return false; + + return merged_rows * merged_rows >= sum_blocks_granularity; + } + UInt64 totalMergedRows() const { return total_merged_rows; } UInt64 mergedRows() const { return merged_rows; } private: + UInt64 sum_blocks_granularity = 0; UInt64 total_merged_rows = 0; UInt64 merged_rows = 0; + + const UInt64 max_block_size; + const bool use_average_block_size; + MutableColumns columns; }; MergedData merged_data; +protected: + /// Profile info. + Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + UInt64 total_rows = 0; + UInt64 total_bytes = 0; + UInt64 total_chunks = 0; + private: /// Processor state. bool is_initialized = false; diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index c259ce05d76..2dc48e4265b 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -3,6 +3,8 @@ #include #include +#include + namespace DB { @@ -15,13 +17,17 @@ MergingSortedTransform::MergingSortedTransform( const Block & header, size_t num_inputs, SortDescription description_, - size_t max_block_size_, + size_t max_block_size, UInt64 limit_, bool quiet_, - bool have_all_inputs_) - : IMergingTransform(num_inputs, header, header, have_all_inputs_) - , description(std::move(description_)), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) - , source_chunks(num_inputs), cursors(num_inputs) + bool use_average_block_sizes, + bool have_all_inputs) + : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, have_all_inputs) + , description(std::move(description_)) + , limit(limit_) + , quiet(quiet_) + , source_chunks(num_inputs) + , cursors(num_inputs) { auto & sample = outputs.front().getHeader(); /// Replace column names in description to positions. @@ -110,16 +116,12 @@ void MergingSortedTransform::merge(TSortingHeap & queue) return false; } - return merged_data.mergedRows() < max_block_size; + return merged_data.hasEnoughRows(); }; /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` - while (queue.isValid()) + while (queue.isValid() && can_read_another_row()) { - /// Shouldn't happen at first iteration, but check just in case. - if (!can_read_another_row()) - return; - auto current = queue.current(); /** And what if the block is totally less or equal than the rest for the current cursor? @@ -147,7 +149,7 @@ void MergingSortedTransform::merge(TSortingHeap & queue) //std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n"; //std::cerr << "Inserting row\n"; - merged_data.insertRow(current->all_columns, current->pos); + merged_data.insertRow(current->all_columns, current->pos, current->rows); if (out_row_sources_buf) { @@ -211,4 +213,26 @@ void MergingSortedTransform::insertFromChunk(size_t source_num) } } +void MergingSortedTransform::onFinish() +{ + if (quiet) + return; + + auto * log = &Logger::get("MergingSortedBlockInputStream"); + + double seconds = total_stopwatch.elapsedSeconds(); + + std::stringstream message; + message << std::fixed << std::setprecision(2) + << "Merge sorted " << total_chunks << " blocks, " << total_rows << " rows" + << " in " << seconds << " sec."; + + if (seconds != 0) + message << ", " + << total_rows / seconds << " rows/sec., " + << total_bytes / 1000000.0 / seconds << " MB/sec."; + + LOG_DEBUG(log, message.str()); +} + } diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.h b/dbms/src/Processors/Merges/MergingSortedTransform.h index 1fac794b7d0..c176c43d0c5 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.h +++ b/dbms/src/Processors/Merges/MergingSortedTransform.h @@ -17,22 +17,23 @@ public: size_t num_inputs, SortDescription description, size_t max_block_size, - UInt64 limit = 0, - bool quiet = false, + UInt64 limit_ = 0, + bool quiet_ = false, + bool use_average_block_sizes = false, bool have_all_inputs = true); String getName() const override { return "MergingSortedTransform"; } void work() override; -protected: +private: void onNewInput() override; void initializeInputs() override; void consume(Chunk chunk, size_t input_number) override; + void onFinish() override; /// Settings SortDescription description; - const size_t max_block_size; UInt64 limit; bool has_collation = false; bool quiet = false; @@ -50,8 +51,6 @@ protected: SortingHeap queue_with_collation; bool is_queue_initialized = false; -private: - template void merge(TSortingHeap & queue); diff --git a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp index 7c447a24b8e..de5b7092024 100644 --- a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp @@ -191,6 +191,7 @@ void MergeSortingTransform::consume(Chunk chunk) { bool quiet = false; bool have_all_inputs = false; + bool use_average_block_sizes = false; external_merging_sorted = std::make_shared( header_without_constants, @@ -199,6 +200,7 @@ void MergeSortingTransform::consume(Chunk chunk) max_merged_block_size, limit, quiet, + use_average_block_sizes, have_all_inputs); processors.emplace_back(external_merging_sorted); From af7f3fc25184c30194786580edcb40c43b72806d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Mar 2020 20:31:03 +0300 Subject: [PATCH 047/743] Fix build. --- dbms/src/Processors/Merges/MergingSortedTransform.cpp | 4 ++-- dbms/src/Processors/Merges/MergingSortedTransform.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index 2dc48e4265b..d371d623746 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -21,8 +21,8 @@ MergingSortedTransform::MergingSortedTransform( UInt64 limit_, bool quiet_, bool use_average_block_sizes, - bool have_all_inputs) - : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, have_all_inputs) + bool have_all_inputs_) + : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, have_all_inputs_) , description(std::move(description_)) , limit(limit_) , quiet(quiet_) diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.h b/dbms/src/Processors/Merges/MergingSortedTransform.h index c176c43d0c5..9d6fdda723c 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.h +++ b/dbms/src/Processors/Merges/MergingSortedTransform.h @@ -20,7 +20,7 @@ public: UInt64 limit_ = 0, bool quiet_ = false, bool use_average_block_sizes = false, - bool have_all_inputs = true); + bool have_all_inputs_ = true); String getName() const override { return "MergingSortedTransform"; } void work() override; From d1d54ab1df24de222fd74c9a44ba35277a6607f8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Mar 2020 22:15:14 +0300 Subject: [PATCH 048/743] Try fix tests. --- dbms/src/Processors/Merges/IMergingTransform.cpp | 2 +- dbms/src/Processors/Merges/MergingSortedTransform.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Processors/Merges/IMergingTransform.cpp b/dbms/src/Processors/Merges/IMergingTransform.cpp index 92ad592e11f..ac8f6b947a6 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.cpp +++ b/dbms/src/Processors/Merges/IMergingTransform.cpp @@ -99,7 +99,7 @@ IProcessor::Status IMergingTransform::prepareInitializeInputs() if (input.isFinished()) continue; - if (!input_states[i].is_initialized) + if (input_states[i].is_initialized) { // input.setNotNeeded(); continue; diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index d371d623746..22b8673c4f5 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -116,7 +116,7 @@ void MergingSortedTransform::merge(TSortingHeap & queue) return false; } - return merged_data.hasEnoughRows(); + return !merged_data.hasEnoughRows(); }; /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` @@ -218,7 +218,7 @@ void MergingSortedTransform::onFinish() if (quiet) return; - auto * log = &Logger::get("MergingSortedBlockInputStream"); + auto * log = &Logger::get("MergingSortedTransform"); double seconds = total_stopwatch.elapsedSeconds(); From 96895e063fc2ff2600bd23bffe118f7dfbc01365 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Mar 2020 13:09:42 +0300 Subject: [PATCH 049/743] Try fix tests. --- dbms/src/Processors/Merges/IMergingTransform.cpp | 2 +- dbms/src/Processors/Merges/MergingSortedTransform.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Merges/IMergingTransform.cpp b/dbms/src/Processors/Merges/IMergingTransform.cpp index ac8f6b947a6..2dd6c27fd61 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.cpp +++ b/dbms/src/Processors/Merges/IMergingTransform.cpp @@ -46,7 +46,7 @@ void IMergingTransform::setHaveAllInputs() void IMergingTransform::requestDataForInput(size_t input_number) { - if (!need_data) + if (need_data) throw Exception("Data was requested for several inputs in IMergingTransform:" " " + std::to_string(next_input_to_read) + " and " + std::to_string(input_number), ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index 22b8673c4f5..2971c55a18a 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -184,7 +184,7 @@ void MergingSortedTransform::merge(TSortingHeap & queue) void MergingSortedTransform::insertFromChunk(size_t source_num) { if (source_num >= cursors.size()) - throw Exception("Logical error in MergingSortedTrandform", ErrorCodes::LOGICAL_ERROR); + throw Exception("Logical error in MergingSortedTransform", ErrorCodes::LOGICAL_ERROR); //std::cerr << "copied columns\n"; From 9577ef38d0fb9b297a8df0752f1734cef9d92e93 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Mar 2020 15:04:56 +0300 Subject: [PATCH 050/743] Fix style. --- dbms/src/Processors/Merges/IMergingTransform.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Processors/Merges/IMergingTransform.h b/dbms/src/Processors/Merges/IMergingTransform.h index 5680b92b6d3..58ad3a9b969 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.h +++ b/dbms/src/Processors/Merges/IMergingTransform.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + /// Base class for merging transforms. class IMergingTransform : public IProcessor { From d5ab2e53319331a22acbb147081793f6fabb007b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Mar 2020 20:37:35 +0300 Subject: [PATCH 051/743] Added CollapsingSortedTransform. --- .../Merges/CollapsingSortedTransform.cpp | 231 ++++++++++++++++++ .../Merges/CollapsingSortedTransform.h | 93 +++++++ .../Processors/Merges/IMergingTransform.cpp | 10 +- .../src/Processors/Merges/IMergingTransform.h | 20 +- .../Merges/MergingSortedTransform.cpp | 10 +- .../Merges/MergingSortedTransform.h | 3 +- dbms/src/Processors/Merges/SharedChunk.h | 83 +++++++ .../Transforms/MergeSortingTransform.cpp | 1 + 8 files changed, 429 insertions(+), 22 deletions(-) create mode 100644 dbms/src/Processors/Merges/CollapsingSortedTransform.cpp create mode 100644 dbms/src/Processors/Merges/CollapsingSortedTransform.h create mode 100644 dbms/src/Processors/Merges/SharedChunk.h diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp new file mode 100644 index 00000000000..e95b574ffe7 --- /dev/null +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp @@ -0,0 +1,231 @@ +#include +#include +#include +#include +#include + +/// Maximum number of messages about incorrect data in the log. +#define MAX_ERROR_MESSAGES 10 + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; +} + +CollapsingSortedTransform::CollapsingSortedTransform( + const Block & header, + size_t num_inputs, + SortDescription description_, + const String & sign_column, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_, + bool use_average_block_sizes) + : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, true) + , description(std::move(description_)) + , sign_column_number(header.getPositionByName(sign_column)) + , out_row_sources_buf(out_row_sources_buf_) +{ +} + +void CollapsingSortedTransform::initializeInputs() +{ + queue = SortingHeap(cursors); + is_queue_initialized = true; +} + +void CollapsingSortedTransform::consume(Chunk chunk, size_t input_number) +{ + updateCursor(std::move(chunk), input_number); + + if (is_queue_initialized) + queue.push(cursors[input_number]); +} + +void CollapsingSortedTransform::updateCursor(Chunk chunk, size_t source_num) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); + + auto & source_chunk = source_chunks[source_num]; + + if (source_chunk) + { + source_chunk = new detail::SharedChunk(std::move(chunk)); + cursors[source_num].reset(source_chunk->getColumns(), {}); + } + else + { + if (cursors[source_num].has_collation) + throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); + + source_chunk = new detail::SharedChunk(std::move(chunk)); + cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); + } + + source_chunk->all_columns = cursors[source_num].all_columns; + source_chunk->sort_columns = cursors[source_num].sort_columns; +} + +void CollapsingSortedTransform::reportIncorrectData() +{ + std::stringstream s; + s << "Incorrect data: number of rows with sign = 1 (" << count_positive + << ") differs with number of rows with sign = -1 (" << count_negative + << ") by more than one (for key: "; + + auto & sort_columns = *last_row.sort_columns; + for (size_t i = 0, size = sort_columns.size(); i < size; ++i) + { + if (i != 0) + s << ", "; + s << applyVisitor(FieldVisitorToString(), (*sort_columns[i])[last_row.row_num]); + } + + s << ")."; + + /** Fow now we limit ourselves to just logging such situations, + * since the data is generated by external programs. + * With inconsistent data, this is an unavoidable error that can not be easily corrected by admins. Therefore Warning. + */ + LOG_WARNING(log, s.rdbuf()); +} + +void CollapsingSortedTransform::insertRow(RowRef & row) +{ + merged_data.insertRow(*row.all_columns, row.row_num, row.owned_chunk->getNumRows()); +} + +void CollapsingSortedTransform::insertRows() +{ + if (count_positive == 0 && count_negative == 0) + { + /// No input rows have been read. + return; + } + + if (last_is_positive || count_positive != count_negative) + { + if (count_positive <= count_negative) + { + insertRow(first_negative_row); + + if (out_row_sources_buf) + current_row_sources[first_negative_pos].setSkipFlag(false); + } + + if (count_positive >= count_negative) + { + insertRow(last_positive_row); + + if (out_row_sources_buf) + current_row_sources[last_positive_pos].setSkipFlag(false); + } + + if (!(count_positive == count_negative || count_positive + 1 == count_negative || count_positive == count_negative + 1)) + { + if (count_incorrect_data < MAX_ERROR_MESSAGES) + reportIncorrectData(); + ++count_incorrect_data; + } + } + + if (out_row_sources_buf) + out_row_sources_buf->write( + reinterpret_cast(current_row_sources.data()), + current_row_sources.size() * sizeof(RowSourcePart)); +} + +void CollapsingSortedTransform::work() +{ + merge(); +} + +void CollapsingSortedTransform::merge() +{ + /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` + while (queue.isValid()) + { + auto current = queue.current(); + Int8 sign = assert_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; + + RowRef current_row; + setRowRef(current_row, current); + + if (last_row.empty()) + setRowRef(last_row, current); + + bool key_differs = !last_row.hasEqualSortColumnsWith(current_row); + + /// if there are enough rows and the last one is calculated completely + if (key_differs && merged_data.hasEnoughRows()) + return; + + if (key_differs) + { + /// We write data for the previous primary key. + insertRows(); + + current_row.swap(last_row); + + count_negative = 0; + count_positive = 0; + + current_pos = 0; + first_negative_pos = 0; + last_positive_pos = 0; + current_row_sources.resize(0); + } + + /// Initially, skip all rows. On insert, unskip "corner" rows. + if (out_row_sources_buf) + current_row_sources.emplace_back(current.impl->order, true); + + if (sign == 1) + { + ++count_positive; + last_is_positive = true; + + setRowRef(last_positive_row, current); + last_positive_pos = current_pos; + } + else if (sign == -1) + { + if (!count_negative) + { + setRowRef(first_negative_row, current); + first_negative_pos = current_pos; + } + + ++count_negative; + last_is_positive = false; + } + else + throw Exception("Incorrect data: Sign = " + toString(sign) + " (must be 1 or -1).", + ErrorCodes::INCORRECT_DATA); + + if (!current->isLast()) + { + queue.next(); + } + else + { + /// We take next block from the corresponding source, if there is one. + queue.removeTop(); + requestDataForInput(current.impl->order); + return; + } + } + + insertRows(); + finish(); +} + +} diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.h b/dbms/src/Processors/Merges/CollapsingSortedTransform.h new file mode 100644 index 00000000000..a2240d20e77 --- /dev/null +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.h @@ -0,0 +1,93 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +/** Merges several sorted inputs to one. + * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), + * keeps no more than one row with the value of the column `sign_column = -1` ("negative row") + * and no more than a row with the value of the column `sign_column = 1` ("positive row"). + * That is, it collapses the records from the change log. + * + * If the number of positive and negative rows is the same, and the last row is positive, then the first negative and last positive rows are written. + * If the number of positive and negative rows is the same, and the last line is negative, it writes nothing. + * If the positive by 1 is greater than the negative rows, then only the last positive row is written. + * If negative by 1 is greater than positive rows, then only the first negative row is written. + * Otherwise, a logical error. + */ +class CollapsingSortedTransform final : public IMergingTransform +{ +public: + CollapsingSortedTransform( + const Block & header, + size_t num_inputs, + SortDescription description_, + const String & sign_column, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_ = nullptr, + bool use_average_block_sizes = false); + + String getName() const override { return "CollapsingSortedTransform"; } + void work() override; + +protected: + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + +private: + Logger * log = &Logger::get("CollapsingSortedTransform"); + + /// Settings + SortDescription description; + bool has_collation = false; + + const size_t sign_column_number; + + /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) + /// If it is not nullptr then it should be populated during execution + WriteBuffer * out_row_sources_buf = nullptr; + + /// Chunks currently being merged. + using SourceChunks = std::vector; + SourceChunks source_chunks; + + SortCursorImpls cursors; + + SortingHeap queue; + bool is_queue_initialized = false; + + using RowRef = detail::RowRef; + RowRef first_negative_row; + RowRef last_positive_row; + RowRef last_row; + + size_t count_positive = 0; /// The number of positive rows for the current primary key. + size_t count_negative = 0; /// The number of negative rows for the current primary key. + bool last_is_positive = false; /// true if the last row for the current primary key is positive. + + /// Fields specific for VERTICAL merge algorithm. + /// Row numbers are relative to the start of current primary key. + size_t current_pos = 0; /// Current row number + size_t first_negative_pos = 0; /// Row number of first_negative + size_t last_positive_pos = 0; /// Row number of last_positive + PODArray current_row_sources; /// Sources of rows with the current primary key + + size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log. + + void reportIncorrectData(); + void insertRow(RowRef & row); + void insertRows(); + void merge(); + void updateCursor(Chunk chunk, size_t source_num); + void setRowRef(RowRef & row, SortCursor & cursor) { row.set(cursor, source_chunks[cursor.impl->order]); } +}; + +} diff --git a/dbms/src/Processors/Merges/IMergingTransform.cpp b/dbms/src/Processors/Merges/IMergingTransform.cpp index 2dd6c27fd61..b0226ce0f3d 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.cpp +++ b/dbms/src/Processors/Merges/IMergingTransform.cpp @@ -173,15 +173,7 @@ IProcessor::Status IMergingTransform::prepare() /// Push if has data. bool has_data_to_push = (is_finished && merged_data.mergedRows()) || merged_data.hasEnoughRows(); if (has_data_to_push && !is_port_full) - { - auto chunk = merged_data.pull(); - - ++total_chunks; - total_rows += chunk.getNumRows(); - total_bytes += chunk.allocatedBytes(); - - output.push(std::move(chunk)); - } + output.push(merged_data.pull()); if (!is_initialized) return prepareInitializeInputs(); diff --git a/dbms/src/Processors/Merges/IMergingTransform.h b/dbms/src/Processors/Merges/IMergingTransform.h index 58ad3a9b969..933b300bb96 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.h +++ b/dbms/src/Processors/Merges/IMergingTransform.h @@ -41,7 +41,7 @@ protected: void finish() { is_finished = true; } /// Call it when all data was inserted to merged_data. /// Struct which represents current merging chunk of data. - /// Also it calculates the number of merged rows. + /// Also it calculates the number of merged rows and other profile info. class MergedData { public: @@ -98,6 +98,8 @@ protected: merged_rows = 0; sum_blocks_granularity = 0; + ++total_chunks; + total_allocated_bytes += chunk.allocatedBytes(); return chunk; } @@ -118,28 +120,28 @@ protected: return merged_rows * merged_rows >= sum_blocks_granularity; } - UInt64 totalMergedRows() const { return total_merged_rows; } UInt64 mergedRows() const { return merged_rows; } + UInt64 totalMergedRows() const { return total_merged_rows; } + UInt64 totalChunks() const { return total_chunks; } + UInt64 totalAllocatedBytes() const { return total_allocated_bytes; } private: + MutableColumns columns; + UInt64 sum_blocks_granularity = 0; - UInt64 total_merged_rows = 0; UInt64 merged_rows = 0; + UInt64 total_merged_rows = 0; + UInt64 total_chunks = 0; + UInt64 total_allocated_bytes = 0; const UInt64 max_block_size; const bool use_average_block_size; - - MutableColumns columns; }; MergedData merged_data; -protected: /// Profile info. Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; - UInt64 total_rows = 0; - UInt64 total_bytes = 0; - UInt64 total_chunks = 0; private: /// Processor state. diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index 2971c55a18a..fc07a7c63ea 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -19,6 +19,7 @@ MergingSortedTransform::MergingSortedTransform( SortDescription description_, size_t max_block_size, UInt64 limit_, + WriteBuffer * out_row_sources_buf_, bool quiet_, bool use_average_block_sizes, bool have_all_inputs_) @@ -26,6 +27,7 @@ MergingSortedTransform::MergingSortedTransform( , description(std::move(description_)) , limit(limit_) , quiet(quiet_) + , out_row_sources_buf(out_row_sources_buf_) , source_chunks(num_inputs) , cursors(num_inputs) { @@ -205,6 +207,8 @@ void MergingSortedTransform::insertFromChunk(size_t source_num) source_chunks[source_num] = Chunk(); + /// Write order of rows for other columns + /// this data will be used in gather stream if (out_row_sources_buf) { RowSourcePart row_source(source_num); @@ -224,13 +228,13 @@ void MergingSortedTransform::onFinish() std::stringstream message; message << std::fixed << std::setprecision(2) - << "Merge sorted " << total_chunks << " blocks, " << total_rows << " rows" + << "Merge sorted " << merged_data.totalChunks() << " blocks, " << merged_data.totalMergedRows() << " rows" << " in " << seconds << " sec."; if (seconds != 0) message << ", " - << total_rows / seconds << " rows/sec., " - << total_bytes / 1000000.0 / seconds << " MB/sec."; + << merged_data.totalMergedRows() / seconds << " rows/sec., " + << merged_data.totalAllocatedBytes() / 1000000.0 / seconds << " MB/sec."; LOG_DEBUG(log, message.str()); } diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.h b/dbms/src/Processors/Merges/MergingSortedTransform.h index 9d6fdda723c..f8e3e65951f 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.h +++ b/dbms/src/Processors/Merges/MergingSortedTransform.h @@ -9,7 +9,7 @@ namespace DB { /// Merges several sorted inputs into one sorted output. -class MergingSortedTransform : public IMergingTransform +class MergingSortedTransform final : public IMergingTransform { public: MergingSortedTransform( @@ -18,6 +18,7 @@ public: SortDescription description, size_t max_block_size, UInt64 limit_ = 0, + WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false, bool use_average_block_sizes = false, bool have_all_inputs_ = true); diff --git a/dbms/src/Processors/Merges/SharedChunk.h b/dbms/src/Processors/Merges/SharedChunk.h new file mode 100644 index 00000000000..5846c9d0336 --- /dev/null +++ b/dbms/src/Processors/Merges/SharedChunk.h @@ -0,0 +1,83 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB::detail +{ + +/// Allows you refer to the row in the block and hold the block ownership, +/// and thus avoid creating a temporary row object. +/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`; +/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece; +/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr; +/// The reference counter is not atomic, since it is used from one thread. +struct SharedChunk : Chunk +{ + int refcount = 0; + + ColumnRawPtrs all_columns; + ColumnRawPtrs sort_columns; + + explicit SharedChunk(Chunk && chunk) : Chunk(std::move(chunk)) {} +}; + +inline void intrusive_ptr_add_ref(detail::SharedChunk * ptr) +{ + ++ptr->refcount; +} + +inline void intrusive_ptr_release(detail::SharedChunk * ptr) +{ + if (0 == --ptr->refcount) + delete ptr; +} + +using SharedChunkPtr = boost::intrusive_ptr; + +struct RowRef +{ + detail::SharedChunkPtr owned_chunk; + + ColumnRawPtrs * all_columns = nullptr; + ColumnRawPtrs * sort_columns = nullptr; + UInt64 row_num = 0; + + void swap(RowRef & other) + { + owned_chunk.swap(other.owned_chunk); + std::swap(all_columns, other.all_columns); + std::swap(sort_columns, other.sort_columns); + std::swap(row_num, other.row_num); + } + + bool empty() const { return all_columns == nullptr; } + + void set(SortCursor & cursor, SharedChunkPtr chunk) + { + owned_chunk = std::move(chunk); + row_num = cursor.impl->pos; + all_columns = &owned_chunk->all_columns; + sort_columns = &owned_chunk->sort_columns; + } + + bool hasEqualSortColumnsWith(const RowRef & other) + { + auto size = sort_columns->size(); + for (size_t col_number = 0; col_number < size; ++col_number) + { + auto & cur_column = (*sort_columns)[col_number]; + auto & other_column = (*other.sort_columns)[col_number]; + + if (0 != cur_column->compareAt(row_num, other.row_num, *other_column, 1)) + return false; + } + + return true; + } +}; + +} diff --git a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp index de5b7092024..ee1836a4a14 100644 --- a/dbms/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/dbms/src/Processors/Transforms/MergeSortingTransform.cpp @@ -199,6 +199,7 @@ void MergeSortingTransform::consume(Chunk chunk) description, max_merged_block_size, limit, + nullptr, quiet, use_average_block_sizes, have_all_inputs); From bbecbff5461ae1e50833559f7cd47185279af720 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Mar 2020 20:58:35 +0300 Subject: [PATCH 052/743] Try fix tests. --- dbms/src/Processors/Merges/MergingSortedTransform.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index fc07a7c63ea..52e8a7f5a57 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -122,8 +122,11 @@ void MergingSortedTransform::merge(TSortingHeap & queue) }; /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` - while (queue.isValid() && can_read_another_row()) + while (queue.isValid()) { + if (!can_read_another_row()) + return; + auto current = queue.current(); /** And what if the block is totally less or equal than the rest for the current cursor? From d591cea77ccd19a5ccc14a0f075109a2518ccebc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Mar 2020 21:42:36 +0300 Subject: [PATCH 053/743] Try fix tests. --- dbms/src/Processors/Merges/IMergingTransform.h | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Merges/IMergingTransform.h b/dbms/src/Processors/Merges/IMergingTransform.h index 933b300bb96..d0b18c77480 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.h +++ b/dbms/src/Processors/Merges/IMergingTransform.h @@ -71,7 +71,6 @@ protected: ErrorCodes::LOGICAL_ERROR); auto num_rows = chunk.getNumRows(); - auto block_size = num_rows; columns = chunk.mutateColumns(); if (limit_rows && num_rows > limit_rows) { @@ -80,9 +79,12 @@ protected: column = (*column->cut(0, num_rows)).mutate(); } + was_chunk_inserted = true; total_merged_rows += num_rows; merged_rows = num_rows; - sum_blocks_granularity += block_size * num_rows; + + /// We don't cate about granularity here. Because, for fast-forward optimization, chunk will be moved as-is. + /// sum_blocks_granularity += block_size * num_rows; } Chunk pull() @@ -100,12 +102,18 @@ protected: sum_blocks_granularity = 0; ++total_chunks; total_allocated_bytes += chunk.allocatedBytes(); + was_chunk_inserted = false; return chunk; } bool hasEnoughRows() const { + /// If full chunk was inserted, then we must pull it. + /// It is needed for fast-forward optimization. + if (was_chunk_inserted) + return true; + /// Never return more then max_block_size. if (merged_rows >= max_block_size) return true; @@ -136,6 +144,8 @@ protected: const UInt64 max_block_size; const bool use_average_block_size; + + bool was_chunk_inserted = false; }; MergedData merged_data; From d012266440282465653e7c76eafa63500f56f9a2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 Mar 2020 01:58:03 +0300 Subject: [PATCH 054/743] fix --- dbms/src/Common/rename.cpp | 6 ++++ dbms/src/Databases/DatabaseLazy.cpp | 2 ++ dbms/src/Databases/DatabaseOnDisk.cpp | 4 +-- dbms/src/Databases/DatabaseOrdinary.cpp | 2 ++ dbms/src/Interpreters/DatabaseCatalog.cpp | 30 ++++++++++++++++--- dbms/src/Interpreters/DatabaseCatalog.h | 1 + .../src/Interpreters/InterpreterDropQuery.cpp | 13 ++++---- .../configs/config.d/users.xml | 1 - .../integration/test_merge_tree_s3/test.py | 9 ++++-- 9 files changed, 52 insertions(+), 16 deletions(-) diff --git a/dbms/src/Common/rename.cpp b/dbms/src/Common/rename.cpp index 039c4304f69..72bd5d1bb60 100644 --- a/dbms/src/Common/rename.cpp +++ b/dbms/src/Common/rename.cpp @@ -54,11 +54,17 @@ namespace ErrorCodes #endif +#if !defined(__NR_renameat2) +[[noreturn]] +#endif void renameNoReplace(const std::string & old_path, const std::string & new_path) { renameat2(old_path, new_path, RENAME_NOREPLACE); } +#if !defined(__NR_renameat2) +[[noreturn]] +#endif void renameExchange(const std::string & old_path, const std::string & new_path) { renameat2(old_path, new_path, RENAME_EXCHANGE); diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index b911687586e..026cf3345da 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -38,6 +38,8 @@ void DatabaseLazy::loadStoredObjects( Context & context, bool /* has_force_restore_data_flag */) { + Poco::File(context.getPath() + getDataPath()).createDirectories(); + Poco::File(getMetadataPath()).createDirectories(); iterateMetadataFiles(context, [this](const String & file_name) { const std::string table_name = file_name.substr(0, file_name.size() - 4); diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index fceb9d06fd5..2d3009bd247 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -123,13 +123,11 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) return statement_stream.str(); } -DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context & context) +DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context &) : DatabaseWithOwnTablesBase(name, logger) , metadata_path(metadata_path_) , data_path("data/" + escapeForFileName(database_name) + "/") { - Poco::File(context.getPath() + getDataPath()).createDirectories(); - Poco::File(getMetadataPath()).createDirectories(); } diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 63d9bf644e6..ff98870efa4 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -113,6 +113,8 @@ void DatabaseOrdinary::loadStoredObjects( Context & context, bool has_force_restore_data_flag) { + Poco::File(context.getPath() + getDataPath()).createDirectories(); + Poco::File(getMetadataPath()).createDirectories(); /** Tables load faster if they are loaded in sorted (by name) order. * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, * which does not correspond to order tables creation and does not correspond to order of their location on disk. diff --git a/dbms/src/Interpreters/DatabaseCatalog.cpp b/dbms/src/Interpreters/DatabaseCatalog.cpp index 353cdfb33dd..db87462f952 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.cpp +++ b/dbms/src/Interpreters/DatabaseCatalog.cpp @@ -139,6 +139,12 @@ void DatabaseCatalog::shutdown() for (auto & database : current_databases) database.second->shutdown(); + { + std::lock_guard lock(tables_marked_droped_mutex); + for (auto & elem : tables_marked_droped) + if (elem.need_shutdown) + elem.table->shutdown(); + } std::lock_guard lock(databases_mutex); assert(std::find_if_not(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { return elem.map.empty(); }) == uuid_map.end()); @@ -514,6 +520,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr assert(dropped_metadata_path == getPathForDroppedMetadata(table_id)); time_t drop_time; + bool need_shutdown = true; if (table) drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); else @@ -550,13 +557,14 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr } drop_time = Poco::File(dropped_metadata_path).getLastModified().epochTime(); + need_shutdown = false; } std::lock_guard lock(tables_marked_droped_mutex); if (ignore_delay) - tables_marked_droped.push_front({table_id, table, dropped_metadata_path, 0}); + tables_marked_droped.push_front({table_id, table, dropped_metadata_path, 0, need_shutdown}); else - tables_marked_droped.push_back({table_id, table, dropped_metadata_path, drop_time}); + tables_marked_droped.push_back({table_id, table, dropped_metadata_path, drop_time, need_shutdown}); } void DatabaseCatalog::dropTableDataTask() @@ -573,14 +581,21 @@ void DatabaseCatalog::dropTableDataTask() //LOG_INFO(log, "Check table " + elem.table_id.getNameForLogs() + ": " + // "refcount = " + std::to_string(elem.table.use_count()) + ", " + // "time elapsed = " + std::to_string(current_time - elem.drop_time)); - return (!elem.table || elem.table.unique()) && elem.drop_time + drop_delay_s < current_time; + bool not_in_use = !elem.table || elem.table.unique(); + bool old_enough = elem.drop_time + drop_delay_s < current_time; + return (not_in_use && old_enough) || elem.need_shutdown; }); - if (it != tables_marked_droped.end()) + if (it != tables_marked_droped.end() && !it->need_shutdown) { table = std::move(*it); LOG_INFO(log, "Will try drop " + table.table_id.getNameForLogs()); tables_marked_droped.erase(it); } + else if (it->need_shutdown) + { + table = *it; + it->need_shutdown = false; + } } catch (...) { @@ -589,6 +604,13 @@ void DatabaseCatalog::dropTableDataTask() if (table.table_id) { + if (table.need_shutdown) + { + table.table->shutdown(); + (*drop_task)->scheduleAfter(0); + return; + } + try { dropTableFinally(table); diff --git a/dbms/src/Interpreters/DatabaseCatalog.h b/dbms/src/Interpreters/DatabaseCatalog.h index 8d800faaebc..1e09a23fb40 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.h +++ b/dbms/src/Interpreters/DatabaseCatalog.h @@ -185,6 +185,7 @@ private: StoragePtr table; String metadata_path; time_t drop_time; + bool need_shutdown{true}; }; using TablesMarkedAsDropped = std::list; diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 7705a859938..7c0c4488446 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -92,8 +92,11 @@ BlockIO InterpreterDropQuery::executeToTable( { context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id); table->shutdown(); - /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = table->lockExclusively(context.getCurrentQueryId()); + TableStructureWriteLockHolder table_lock; + if (database->getEngineName() != "Atomic") + { + table_lock = table->lockExclusively(context.getCurrentQueryId()); + } /// Drop table from memory, don't touch data and metadata database->detachTable(table_id.table_name); } @@ -112,12 +115,12 @@ BlockIO InterpreterDropQuery::executeToTable( context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id); table->checkTableCanBeDropped(); - table->shutdown(); - /// If table was already dropped by anyone, an exception will be thrown - TableStructureWriteLockHolder table_lock; if (database->getEngineName() != "Atomic") + { + table->shutdown(); table_lock = table->lockExclusively(context.getCurrentQueryId()); + } database->dropTable(context, table_id.table_name, query.no_delay); } diff --git a/dbms/tests/integration/test_merge_tree_s3/configs/config.d/users.xml b/dbms/tests/integration/test_merge_tree_s3/configs/config.d/users.xml index a13b24b278d..797113053f4 100644 --- a/dbms/tests/integration/test_merge_tree_s3/configs/config.d/users.xml +++ b/dbms/tests/integration/test_merge_tree_s3/configs/config.d/users.xml @@ -1,4 +1,3 @@ - diff --git a/dbms/tests/integration/test_merge_tree_s3/test.py b/dbms/tests/integration/test_merge_tree_s3/test.py index f69c09631e8..43250bede73 100644 --- a/dbms/tests/integration/test_merge_tree_s3/test.py +++ b/dbms/tests/integration/test_merge_tree_s3/test.py @@ -83,7 +83,8 @@ def drop_table(cluster): node = cluster.instances["node"] minio = cluster.minio_client - node.query("DROP TABLE IF EXISTS s3_test") + node.query("DROP TABLE IF EXISTS s3_test NO DELAY") + time.sleep(1) assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 @@ -301,7 +302,8 @@ def test_move_replace_partition_to_another_table(cluster): time.sleep(3) assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD*2 + FILES_OVERHEAD_PER_PART_WIDE*4 - node.query("DROP TABLE s3_clone") + node.query("DROP TABLE s3_clone NO DELAY") + time.sleep(1) assert node.query("SELECT sum(id) FROM s3_test FORMAT Values") == "(0)" assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(16384)" # Data should remain in S3 @@ -311,7 +313,8 @@ def test_move_replace_partition_to_another_table(cluster): # Number S3 objects should be unchanged. assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE*4 - node.query("DROP TABLE s3_test") + node.query("DROP TABLE s3_test NO DELAY") + time.sleep(1) # Backup data should remain in S3. assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD_PER_PART_WIDE*4 From a02eb99f7b1a41f5f9c69ee103f3440707c9516f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Mar 2020 12:23:42 +0300 Subject: [PATCH 055/743] Fix build. --- dbms/src/Processors/Merges/CollapsingSortedTransform.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.h b/dbms/src/Processors/Merges/CollapsingSortedTransform.h index a2240d20e77..985fd8018fe 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.h +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.h @@ -47,8 +47,6 @@ private: /// Settings SortDescription description; - bool has_collation = false; - const size_t sign_column_number; /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) From 968917e99b9364860122e585f0503edc9123ee72 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Mar 2020 12:32:55 +0300 Subject: [PATCH 056/743] Fix tests. --- dbms/src/Processors/Merges/IMergingTransform.h | 13 ++++++++----- .../Processors/Merges/MergingSortedTransform.cpp | 4 +++- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/dbms/src/Processors/Merges/IMergingTransform.h b/dbms/src/Processors/Merges/IMergingTransform.h index d0b18c77480..040263aca36 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.h +++ b/dbms/src/Processors/Merges/IMergingTransform.h @@ -53,6 +53,9 @@ protected: columns.emplace_back(column.type->createColumn()); } + /// Pull will be called at next prepare call. + void flush() { need_flush = true; } + void insertRow(const ColumnRawPtrs & raw_columns, size_t row, size_t block_size) { size_t num_columns = raw_columns.size(); @@ -79,7 +82,7 @@ protected: column = (*column->cut(0, num_rows)).mutate(); } - was_chunk_inserted = true; + need_flush = true; total_merged_rows += num_rows; merged_rows = num_rows; @@ -102,16 +105,16 @@ protected: sum_blocks_granularity = 0; ++total_chunks; total_allocated_bytes += chunk.allocatedBytes(); - was_chunk_inserted = false; + need_flush = false; return chunk; } bool hasEnoughRows() const { - /// If full chunk was inserted, then we must pull it. + /// If full chunk was or is going to be inserted, then we must pull it. /// It is needed for fast-forward optimization. - if (was_chunk_inserted) + if (need_flush) return true; /// Never return more then max_block_size. @@ -145,7 +148,7 @@ protected: const UInt64 max_block_size; const bool use_average_block_size; - bool was_chunk_inserted = false; + bool need_flush = false; }; MergedData merged_data; diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index 52e8a7f5a57..734da2e4bb3 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -138,10 +138,12 @@ void MergingSortedTransform::merge(TSortingHeap & queue) { //std::cerr << "current block is totally less or equals\n"; - /// If there are already data in the current block, we first return it. We'll get here again the next time we call the merge function. + /// If there are already data in the current block, we first return it. + /// We'll get here again the next time we call the merge function. if (merged_data.mergedRows() != 0) { //std::cerr << "merged rows is non-zero\n"; + merged_data.flush(); return; } From d40225330e71f9a88153c0a40d3aab75b11b028d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Mar 2020 13:51:40 +0300 Subject: [PATCH 057/743] Add SharedChunkAllocator. --- .../Merges/CollapsingSortedTransform.cpp | 5 +- .../Merges/CollapsingSortedTransform.h | 5 +- dbms/src/Processors/Merges/RowRef.h | 147 ++++++++++++++++++ dbms/src/Processors/Merges/SharedChunk.h | 83 ---------- 4 files changed, 154 insertions(+), 86 deletions(-) create mode 100644 dbms/src/Processors/Merges/RowRef.h delete mode 100644 dbms/src/Processors/Merges/SharedChunk.h diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp index e95b574ffe7..ee5f79013a9 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp @@ -28,6 +28,7 @@ CollapsingSortedTransform::CollapsingSortedTransform( , description(std::move(description_)) , sign_column_number(header.getPositionByName(sign_column)) , out_row_sources_buf(out_row_sources_buf_) + , chunk_allocator(num_inputs + max_row_refs) { } @@ -58,7 +59,7 @@ void CollapsingSortedTransform::updateCursor(Chunk chunk, size_t source_num) if (source_chunk) { - source_chunk = new detail::SharedChunk(std::move(chunk)); + source_chunk = chunk_allocator.alloc(std::move(chunk)); cursors[source_num].reset(source_chunk->getColumns(), {}); } else @@ -66,7 +67,7 @@ void CollapsingSortedTransform::updateCursor(Chunk chunk, size_t source_num) if (cursors[source_num].has_collation) throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - source_chunk = new detail::SharedChunk(std::move(chunk)); + source_chunk = chunk_allocator.alloc(std::move(chunk)); cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); } diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.h b/dbms/src/Processors/Merges/CollapsingSortedTransform.h index 985fd8018fe..87596d11a29 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.h +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -63,10 +63,13 @@ private: bool is_queue_initialized = false; using RowRef = detail::RowRef; + static constexpr size_t max_row_refs = 4; /// first_negative, last_positive, last, current. RowRef first_negative_row; RowRef last_positive_row; RowRef last_row; + detail::SharedChunkAllocator chunk_allocator; + size_t count_positive = 0; /// The number of positive rows for the current primary key. size_t count_negative = 0; /// The number of negative rows for the current primary key. bool last_is_positive = false; /// true if the last row for the current primary key is positive. diff --git a/dbms/src/Processors/Merges/RowRef.h b/dbms/src/Processors/Merges/RowRef.h new file mode 100644 index 00000000000..ea89b24fbdb --- /dev/null +++ b/dbms/src/Processors/Merges/RowRef.h @@ -0,0 +1,147 @@ +#pragma once + +#include +#include +#include + +#include + + +namespace DB::detail +{ + +class SharedChunkAllocator; + +/// Allows you refer to the row in the block and hold the block ownership, +/// and thus avoid creating a temporary row object. +/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`; +/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece; +/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr; +/// The reference counter is not atomic, since it is used from one thread. +struct SharedChunk : Chunk +{ + ColumnRawPtrs all_columns; + ColumnRawPtrs sort_columns; + + using Chunk::Chunk; + using Chunk::operator=; + +private: + int refcount = 0; + size_t position = 0; + SharedChunkAllocator * allocator = nullptr; + + friend class SharedChunkAllocator; + friend void intrusive_ptr_add_ref(SharedChunk * ptr); + friend void intrusive_ptr_release(SharedChunk * ptr); +}; + +using SharedChunkPtr = boost::intrusive_ptr; + +/// Custom allocator for shared chunk. +/// It helps to avoid explicit new/delete calls if we know maximum required capacity. +/// Thanks to that, SharedChunk does not own any memory. +/// It improves leaks detection, because memory is allocated only once in constructor. +class SharedChunkAllocator +{ +public: + explicit SharedChunkAllocator(size_t max_chunks) + { + chunks.resize(max_chunks); + free_chunks.reserve(max_chunks); + + for (size_t i = 0; i < max_chunks; ++i) + free_chunks.push_back(i); + } + + SharedChunkPtr alloc(Chunk && chunk) + { + if (free_chunks.empty()) + throw Exception("Not enough space in SharedChunkAllocator. " + "Chunks allocated: " + std::to_string(chunks.size()), ErrorCodes::LOGICAL_ERROR); + + auto pos = free_chunks.back(); + free_chunks.pop_back(); + + chunks[pos] = std::move(chunk); + chunks[pos].position = pos; + chunks[pos].allocator = this; + + return SharedChunkPtr(&chunks[pos]); + } + +private: + std::vector chunks; + std::vector free_chunks; + + void release(SharedChunk * ptr) + { + /// Release memory. It is not obligatory. + ptr->clear(); + ptr->all_columns.clear(); + ptr->sort_columns.clear(); + + free_chunks.push_back(ptr->position); + } + + friend void intrusive_ptr_release(SharedChunk * ptr); +}; + +inline void intrusive_ptr_add_ref(SharedChunk * ptr) +{ + ++ptr->refcount; +} + +inline void intrusive_ptr_release(SharedChunk * ptr) +{ + if (0 == --ptr->refcount) + ptr->allocator->release(ptr); +} + +/// This class represents a row in a chunk. +/// RowRef hold shared pointer to this chunk, possibly extending its life time. +/// It is needed, for example, in CollapsingTransform, where we need to store first negative row for current sort key. +/// We do not copy data itself, because it may be potentially changed for each row. Performance for `set` is important. +struct RowRef +{ + detail::SharedChunkPtr owned_chunk; + + ColumnRawPtrs * all_columns = nullptr; + ColumnRawPtrs * sort_columns = nullptr; + UInt64 row_num = 0; + + void swap(RowRef & other) + { + owned_chunk.swap(other.owned_chunk); + std::swap(all_columns, other.all_columns); + std::swap(sort_columns, other.sort_columns); + std::swap(row_num, other.row_num); + } + + bool empty() const { return all_columns == nullptr; } + + void set(SortCursor & cursor, SharedChunkPtr chunk) + { + owned_chunk = std::move(chunk); + row_num = cursor.impl->pos; + all_columns = &owned_chunk->all_columns; + sort_columns = &owned_chunk->sort_columns; + } + + bool hasEqualSortColumnsWith(const RowRef & other) + { + auto size = sort_columns->size(); + for (size_t col_number = 0; col_number < size; ++col_number) + { + auto & cur_column = (*sort_columns)[col_number]; + auto & other_column = (*other.sort_columns)[col_number]; + + if (0 != cur_column->compareAt(row_num, other.row_num, *other_column, 1)) + return false; + } + + return true; + } +}; + +} diff --git a/dbms/src/Processors/Merges/SharedChunk.h b/dbms/src/Processors/Merges/SharedChunk.h deleted file mode 100644 index 5846c9d0336..00000000000 --- a/dbms/src/Processors/Merges/SharedChunk.h +++ /dev/null @@ -1,83 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB::detail -{ - -/// Allows you refer to the row in the block and hold the block ownership, -/// and thus avoid creating a temporary row object. -/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`; -/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece; -/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr; -/// The reference counter is not atomic, since it is used from one thread. -struct SharedChunk : Chunk -{ - int refcount = 0; - - ColumnRawPtrs all_columns; - ColumnRawPtrs sort_columns; - - explicit SharedChunk(Chunk && chunk) : Chunk(std::move(chunk)) {} -}; - -inline void intrusive_ptr_add_ref(detail::SharedChunk * ptr) -{ - ++ptr->refcount; -} - -inline void intrusive_ptr_release(detail::SharedChunk * ptr) -{ - if (0 == --ptr->refcount) - delete ptr; -} - -using SharedChunkPtr = boost::intrusive_ptr; - -struct RowRef -{ - detail::SharedChunkPtr owned_chunk; - - ColumnRawPtrs * all_columns = nullptr; - ColumnRawPtrs * sort_columns = nullptr; - UInt64 row_num = 0; - - void swap(RowRef & other) - { - owned_chunk.swap(other.owned_chunk); - std::swap(all_columns, other.all_columns); - std::swap(sort_columns, other.sort_columns); - std::swap(row_num, other.row_num); - } - - bool empty() const { return all_columns == nullptr; } - - void set(SortCursor & cursor, SharedChunkPtr chunk) - { - owned_chunk = std::move(chunk); - row_num = cursor.impl->pos; - all_columns = &owned_chunk->all_columns; - sort_columns = &owned_chunk->sort_columns; - } - - bool hasEqualSortColumnsWith(const RowRef & other) - { - auto size = sort_columns->size(); - for (size_t col_number = 0; col_number < size; ++col_number) - { - auto & cur_column = (*sort_columns)[col_number]; - auto & other_column = (*other.sort_columns)[col_number]; - - if (0 != cur_column->compareAt(row_num, other.row_num, *other_column, 1)) - return false; - } - - return true; - } -}; - -} From 9dfe4a4c614392ebeb18d7723d5c8145266f328d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Mar 2020 15:06:16 +0300 Subject: [PATCH 058/743] Added ReplacingSortedTransform. --- .../Merges/CollapsingSortedTransform.cpp | 3 + .../Merges/CollapsingSortedTransform.h | 1 - .../Merges/ReplacingSortedTransform.cpp | 144 ++++++++++++++++++ .../Merges/ReplacingSortedTransform.h | 67 ++++++++ dbms/src/Processors/Merges/RowRef.h | 12 +- 5 files changed, 224 insertions(+), 3 deletions(-) create mode 100644 dbms/src/Processors/Merges/ReplacingSortedTransform.cpp create mode 100644 dbms/src/Processors/Merges/ReplacingSortedTransform.h diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp index ee5f79013a9..4bb4eb0af43 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp @@ -138,6 +138,9 @@ void CollapsingSortedTransform::insertRows() } } + first_negative_row.clear(); + last_positive_row.clear(); + if (out_row_sources_buf) out_row_sources_buf->write( reinterpret_cast(current_row_sources.data()), diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.h b/dbms/src/Processors/Merges/CollapsingSortedTransform.h index 87596d11a29..64f9016b90d 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.h +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.h @@ -56,7 +56,6 @@ private: /// Chunks currently being merged. using SourceChunks = std::vector; SourceChunks source_chunks; - SortCursorImpls cursors; SortingHeap queue; diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp new file mode 100644 index 00000000000..9783d23dfc6 --- /dev/null +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp @@ -0,0 +1,144 @@ +#include +#include + +namespace DB +{ + +ReplacingSortedTransform::ReplacingSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description_, const String & version_column, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_, + bool use_average_block_sizes) + : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, true) + , description(std::move(description_)) + , out_row_sources_buf(out_row_sources_buf_) + , chunk_allocator(num_inputs + max_row_refs) +{ + if (!version_column.empty()) + version_column_number = header.getPositionByName(version_column); +} + +void ReplacingSortedTransform::initializeInputs() +{ + queue = SortingHeap(cursors); + is_queue_initialized = true; +} + +void ReplacingSortedTransform::consume(Chunk chunk, size_t input_number) +{ + updateCursor(std::move(chunk), input_number); + + if (is_queue_initialized) + queue.push(cursors[input_number]); +} + +void ReplacingSortedTransform::updateCursor(Chunk chunk, size_t source_num) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); + + auto & source_chunk = source_chunks[source_num]; + + if (source_chunk) + { + source_chunk = chunk_allocator.alloc(std::move(chunk)); + cursors[source_num].reset(source_chunk->getColumns(), {}); + } + else + { + if (cursors[source_num].has_collation) + throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); + + source_chunk = chunk_allocator.alloc(std::move(chunk)); + cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); + } + + source_chunk->all_columns = cursors[source_num].all_columns; + source_chunk->sort_columns = cursors[source_num].sort_columns; +} + +void ReplacingSortedTransform::insertRow() +{ + if (out_row_sources_buf) + { + /// true flag value means "skip row" + current_row_sources[max_pos].setSkipFlag(false); + + out_row_sources_buf->write(reinterpret_cast(current_row_sources.data()), + current_row_sources.size() * sizeof(RowSourcePart)); + current_row_sources.resize(0); + } + + merged_data.insertRow(*selected_row.all_columns, selected_row.row_num, selected_row.owned_chunk->getNumRows()); + selected_row.clear(); +} + +void ReplacingSortedTransform::merge() +{ + /// Take the rows in needed order and put them into `merged_columns` until rows no more than `max_block_size` + while (queue.isValid()) + { + SortCursor current = queue.current(); + + if (last_row.empty()) + setRowRef(last_row, current); + + RowRef current_row; + setRowRef(current_row, current); + + bool key_differs = !current_row.hasEqualSortColumnsWith(last_row); + + /// if there are enough rows and the last one is calculated completely + if (key_differs && merged_data.hasEnoughRows()) + return; + + if (key_differs) + { + /// Write the data for the previous primary key. + insertRow(); + last_row.swap(current_row); + } + + /// Initially, skip all rows. Unskip last on insert. + size_t current_pos = current_row_sources.size(); + if (out_row_sources_buf) + current_row_sources.emplace_back(current.impl->order, true); + + /// A non-strict comparison, since we select the last row for the same version values. + if (version_column_number == -1 + || selected_row.empty() + || current->all_columns[version_column_number]->compareAt( + current->pos, selected_row.row_num, + *(*selected_row.all_columns)[version_column_number], + /* nan_direction_hint = */ 1) >= 0) + { + max_pos = current_pos; + setRowRef(selected_row, current); + } + + if (!current->isLast()) + { + queue.next(); + } + else + { + /// We get the next block from the corresponding source, if there is one. + queue.removeTop(); + requestDataForInput(current.impl->order); + return; + } + } + + /// We will write the data for the last primary key. + if (!selected_row.empty()) + insertRow(); + + finish(); +} + +} diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.h b/dbms/src/Processors/Merges/ReplacingSortedTransform.h new file mode 100644 index 00000000000..19b7a999e9b --- /dev/null +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.h @@ -0,0 +1,67 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +class ReplacingSortedTransform : public IMergingTransform +{ +public: + ReplacingSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description_, const String & version_column, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_ = nullptr, + bool use_average_block_sizes = false); + + String getName() const override { return "ReplacingSorted"; } + +protected: + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + +private: + Logger * log = &Logger::get("ReplacingSortedBlockInputStream"); + + SortDescription description; + ssize_t version_column_number = -1; + + /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) + /// If it is not nullptr then it should be populated during execution + WriteBuffer * out_row_sources_buf = nullptr; + + /// Chunks currently being merged. + using SourceChunks = std::vector; + SourceChunks source_chunks; + SortCursorImpls cursors; + + SortingHeap queue; + bool is_queue_initialized = false; + + using RowRef = detail::RowRef; + static constexpr size_t max_row_refs = 3; /// last, current, selected. + RowRef last_row; + /// RowRef next_key; /// Primary key of next row. + RowRef selected_row; /// Last row with maximum version for current primary key. + size_t max_pos = 0; /// The position (into current_row_sources) of the row with the highest version. + + detail::SharedChunkAllocator chunk_allocator; + + /// Sources of rows with the current primary key. + PODArray current_row_sources; + + void insertRow(); + void merge(); + void updateCursor(Chunk chunk, size_t source_num); + void setRowRef(RowRef & row, SortCursor & cursor) { row.set(cursor, source_chunks[cursor.impl->order]); } +}; + +} diff --git a/dbms/src/Processors/Merges/RowRef.h b/dbms/src/Processors/Merges/RowRef.h index ea89b24fbdb..55d5577107f 100644 --- a/dbms/src/Processors/Merges/RowRef.h +++ b/dbms/src/Processors/Merges/RowRef.h @@ -104,7 +104,7 @@ inline void intrusive_ptr_release(SharedChunk * ptr) /// We do not copy data itself, because it may be potentially changed for each row. Performance for `set` is important. struct RowRef { - detail::SharedChunkPtr owned_chunk; + detail::SharedChunkPtr owned_chunk = nullptr; ColumnRawPtrs * all_columns = nullptr; ColumnRawPtrs * sort_columns = nullptr; @@ -118,7 +118,15 @@ struct RowRef std::swap(row_num, other.row_num); } - bool empty() const { return all_columns == nullptr; } + bool empty() const { return owned_chunk == nullptr; } + + void clear() + { + owned_chunk.reset(); + all_columns = nullptr; + sort_columns = nullptr; + row_num = 0; + } void set(SortCursor & cursor, SharedChunkPtr chunk) { From d8a303dda8088d1c7676e66d200c92e45f66b063 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Mar 2020 15:12:16 +0300 Subject: [PATCH 059/743] Fix style. --- dbms/src/Processors/Merges/RowRef.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Processors/Merges/RowRef.h b/dbms/src/Processors/Merges/RowRef.h index 55d5577107f..67d32d11277 100644 --- a/dbms/src/Processors/Merges/RowRef.h +++ b/dbms/src/Processors/Merges/RowRef.h @@ -6,6 +6,10 @@ #include +namespace DB::ErrorCodes +{ + extern const int LOGICAL_ERROR; +} namespace DB::detail { From 0eb351044c00def206cd39abe9453a320b21da67 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 30 Mar 2020 17:53:05 +0300 Subject: [PATCH 060/743] revert some changes around DDLGuard --- dbms/src/Interpreters/DatabaseCatalog.cpp | 30 ++++--------------- dbms/src/Interpreters/DatabaseCatalog.h | 1 - .../Interpreters/InterpreterCreateQuery.cpp | 18 ++--------- .../src/Interpreters/InterpreterDropQuery.cpp | 10 ++----- 4 files changed, 10 insertions(+), 49 deletions(-) diff --git a/dbms/src/Interpreters/DatabaseCatalog.cpp b/dbms/src/Interpreters/DatabaseCatalog.cpp index db87462f952..81656464957 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.cpp +++ b/dbms/src/Interpreters/DatabaseCatalog.cpp @@ -139,13 +139,6 @@ void DatabaseCatalog::shutdown() for (auto & database : current_databases) database.second->shutdown(); - { - std::lock_guard lock(tables_marked_droped_mutex); - for (auto & elem : tables_marked_droped) - if (elem.need_shutdown) - elem.table->shutdown(); - } - std::lock_guard lock(databases_mutex); assert(std::find_if_not(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { return elem.map.empty(); }) == uuid_map.end()); databases.clear(); @@ -482,7 +475,7 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() String full_path = path + it.name(); Strings name_parts; - boost::split(name_parts, it.name(), boost::is_any_of(".")); + boost::split(name_parts, it.name(), boost::is_any_of(".")); // NOLINT: LLVM Bug 41141 if (name_parts.size() != 4) /// Unexpected file continue; @@ -520,7 +513,6 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr assert(dropped_metadata_path == getPathForDroppedMetadata(table_id)); time_t drop_time; - bool need_shutdown = true; if (table) drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); else @@ -557,14 +549,13 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr } drop_time = Poco::File(dropped_metadata_path).getLastModified().epochTime(); - need_shutdown = false; } std::lock_guard lock(tables_marked_droped_mutex); if (ignore_delay) - tables_marked_droped.push_front({table_id, table, dropped_metadata_path, 0, need_shutdown}); + tables_marked_droped.push_front({table_id, table, dropped_metadata_path, 0}); else - tables_marked_droped.push_back({table_id, table, dropped_metadata_path, drop_time, need_shutdown}); + tables_marked_droped.push_back({table_id, table, dropped_metadata_path, drop_time}); } void DatabaseCatalog::dropTableDataTask() @@ -583,19 +574,14 @@ void DatabaseCatalog::dropTableDataTask() // "time elapsed = " + std::to_string(current_time - elem.drop_time)); bool not_in_use = !elem.table || elem.table.unique(); bool old_enough = elem.drop_time + drop_delay_s < current_time; - return (not_in_use && old_enough) || elem.need_shutdown; + return not_in_use && old_enough; }); - if (it != tables_marked_droped.end() && !it->need_shutdown) + if (it != tables_marked_droped.end()) { table = std::move(*it); LOG_INFO(log, "Will try drop " + table.table_id.getNameForLogs()); tables_marked_droped.erase(it); } - else if (it->need_shutdown) - { - table = *it; - it->need_shutdown = false; - } } catch (...) { @@ -604,12 +590,6 @@ void DatabaseCatalog::dropTableDataTask() if (table.table_id) { - if (table.need_shutdown) - { - table.table->shutdown(); - (*drop_task)->scheduleAfter(0); - return; - } try { diff --git a/dbms/src/Interpreters/DatabaseCatalog.h b/dbms/src/Interpreters/DatabaseCatalog.h index 1e09a23fb40..8d800faaebc 100644 --- a/dbms/src/Interpreters/DatabaseCatalog.h +++ b/dbms/src/Interpreters/DatabaseCatalog.h @@ -185,7 +185,6 @@ private: StoragePtr table; String metadata_path; time_t drop_time; - bool need_shutdown{true}; }; using TablesMarkedAsDropped = std::list; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index beca15e6ca0..d81fff61f83 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -539,26 +539,13 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String current_database = context.getCurrentDatabase(); - String metadata_path_tmp; - bool created_tmp = false; - SCOPE_EXIT({ if (created_tmp) Poco::File(metadata_path_tmp).remove(); }); - // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) { auto database_name = create.database.empty() ? current_database : create.database; auto database = DatabaseCatalog::instance().getDatabase(database_name); - - //TODO do we really need it? refactor it if we do - if (database->getEngineName() == "Atomic") - { - metadata_path_tmp = database->getObjectMetadataPath(create.table) + ".tmp"; - if (!Poco::File(metadata_path_tmp).createFile()) - throw Exception("Cannot attach table because of concurrent query.", ErrorCodes::TABLE_ALREADY_EXISTS); - created_tmp = true; - } - bool if_not_exists = create.if_not_exists; + // Table SQL definition is available even if the table is detached auto query = database->getCreateTableQuery(context, create.table); create = query->as(); // Copy the saved create query, but use ATTACH instead of CREATE @@ -622,8 +609,7 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. */ - if (database->getEngineName() != "Atomic") - guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name); + guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name); /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. if (database->isTableExist(context, table_name)) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 7c0c4488446..e6279de818c 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -75,10 +75,7 @@ BlockIO InterpreterDropQuery::executeToTable( auto table_id = context.resolveStorageID(table_id_, Context::ResolveOrdinary); - std::unique_ptr ddl_guard; - if (auto db = DatabaseCatalog::instance().tryGetDatabase(table_id.database_name)) //FIXME - if (db->getEngineName() != "Atomic") - ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); + auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); auto [database, table] = tryGetDatabaseAndTable(table_id.database_name, table_id.table_name, query.if_exists); @@ -115,12 +112,11 @@ BlockIO InterpreterDropQuery::executeToTable( context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id); table->checkTableCanBeDropped(); + table->shutdown(); + TableStructureWriteLockHolder table_lock; if (database->getEngineName() != "Atomic") - { - table->shutdown(); table_lock = table->lockExclusively(context.getCurrentQueryId()); - } database->dropTable(context, table_id.table_name, query.no_delay); } From c05331656fbbb977787be527a3ec29ef5914ebb1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Mar 2020 19:16:00 +0300 Subject: [PATCH 061/743] Added VersionedCollapsingTransform. --- .../Merges/ReplacingSortedTransform.cpp | 10 + .../Merges/ReplacingSortedTransform.h | 3 +- .../Merges/VersionedCollapsingTransform.cpp | 200 ++++++++++++++++ .../Merges/VersionedCollapsingTransform.h | 225 ++++++++++++++++++ 4 files changed, 437 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp create mode 100644 dbms/src/Processors/Merges/VersionedCollapsingTransform.h diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp index 9783d23dfc6..18fb9f98221 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp @@ -4,6 +4,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + ReplacingSortedTransform::ReplacingSortedTransform( size_t num_inputs, const Block & header, SortDescription description_, const String & version_column, @@ -78,6 +83,11 @@ void ReplacingSortedTransform::insertRow() selected_row.clear(); } +void ReplacingSortedTransform::work() +{ + merge(); +} + void ReplacingSortedTransform::merge() { /// Take the rows in needed order and put them into `merged_columns` until rows no more than `max_block_size` diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.h b/dbms/src/Processors/Merges/ReplacingSortedTransform.h index 19b7a999e9b..85fd0d23fe4 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.h +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.h @@ -23,13 +23,14 @@ public: bool use_average_block_sizes = false); String getName() const override { return "ReplacingSorted"; } + void work() override; protected: void initializeInputs() override; void consume(Chunk chunk, size_t input_number) override; private: - Logger * log = &Logger::get("ReplacingSortedBlockInputStream"); + Logger * log = &Logger::get("ReplacingSortedTransform"); SortDescription description; ssize_t version_column_number = -1; diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp new file mode 100644 index 00000000000..b4caeb22bf8 --- /dev/null +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp @@ -0,0 +1,200 @@ +#include +#include +#include + +namespace DB +{ + +static const size_t MAX_ROWS_IN_MULTIVERSION_QUEUE = 8192; + +VersionedCollapsingTransform::VersionedCollapsingTransform( + size_t num_inputs, const Block & header, + SortDescription description_, const String & sign_column_, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_, + bool use_average_block_sizes) + : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, true) + , description(std::move(description_)) + , out_row_sources_buf(out_row_sources_buf_) + , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 2) + , current_keys(max_rows_in_queue + 1) + , chunk_allocator(num_inputs + max_rows_in_queue + 1) +{ + sign_column_number = header.getPositionByName(sign_column_); +} + +void VersionedCollapsingTransform::initializeInputs() +{ + queue = SortingHeap(cursors); + is_queue_initialized = true; +} + +void VersionedCollapsingTransform::consume(Chunk chunk, size_t input_number) +{ + updateCursor(std::move(chunk), input_number); + + if (is_queue_initialized) + queue.push(cursors[input_number]); +} + +void VersionedCollapsingTransform::updateCursor(Chunk chunk, size_t source_num) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); + + auto & source_chunk = source_chunks[source_num]; + + if (source_chunk) + { + source_chunk = chunk_allocator.alloc(std::move(chunk)); + cursors[source_num].reset(source_chunk->getColumns(), {}); + } + else + { + if (cursors[source_num].has_collation) + throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); + + source_chunk = chunk_allocator.alloc(std::move(chunk)); + cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); + } + + source_chunk->all_columns = cursors[source_num].all_columns; + source_chunk->sort_columns = cursors[source_num].sort_columns; +} + +void VersionedCollapsingTransform::work() +{ + merge(); +} + +inline ALWAYS_INLINE static void writeRowSourcePart(WriteBuffer & buffer, RowSourcePart row_source) +{ + if constexpr (sizeof(RowSourcePart) == 1) + buffer.write(*reinterpret_cast(&row_source)); + else + buffer.write(reinterpret_cast(&row_source), sizeof(RowSourcePart)); +} + +void VersionedCollapsingTransform::insertGap(size_t gap_size) +{ + if (out_row_sources_buf) + { + for (size_t i = 0; i < gap_size; ++i) + { + writeRowSourcePart(*out_row_sources_buf, current_row_sources.front()); + current_row_sources.pop(); + } + } +} + +void VersionedCollapsingTransform::insertRow(size_t skip_rows, const RowRef & row) +{ + merged_data.insertRow(*row.all_columns, row.row_num, row.owned_chunk->getNumRows()); + + insertGap(skip_rows); + + if (out_row_sources_buf) + { + current_row_sources.front().setSkipFlag(false); + writeRowSourcePart(*out_row_sources_buf, current_row_sources.front()); + current_row_sources.pop(); + } +} + +void VersionedCollapsingTransform::merge() +{ + /// Take rows in correct order and put them into `merged_columns` until the rows no more than `max_block_size` + while (queue.isValid()) + { + SortCursor current = queue.current(); + + RowRef current_row; + + Int8 sign = assert_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; + + setRowRef(current_row, current); + + /// At first, let's decide the number of rows needed to insert right now. + size_t num_rows_to_insert = 0; + if (!current_keys.empty()) + { + auto key_differs = !current_row.hasEqualSortColumnsWith(current_keys.back()); + + if (key_differs) /// Flush whole queue + num_rows_to_insert = current_keys.size(); + else if (current_keys.size() >= max_rows_in_queue) /// Flush single row if queue is big + num_rows_to_insert = 1; + } + + /// Insert ready roes if any. + while (num_rows_to_insert) + { + const auto & row = current_keys.front(); + auto gap = current_keys.frontGap(); + + insertRow(gap, row); + + current_keys.popFront(); + + --num_rows_to_insert; + + /// It's ok to return here, because we didn't affect queue. + if (merged_data.hasEnoughRows()) + return; + } + + if (current_keys.empty()) + { + sign_in_queue = sign; + current_keys.pushBack(current_row); + } + else /// If queue is not empty, then current_row has the same key as in current_keys queue + { + if (sign == sign_in_queue) + current_keys.pushBack(current_row); + else + { + current_keys.popBack(); + current_keys.pushGap(2); + } + } + + if (out_row_sources_buf) + current_row_sources.emplace(current->order, true); + + if (!current->isLast()) + { + queue.next(); + } + else + { + /// We take next block from the corresponding source, if there is one. + queue.removeTop(); + requestDataForInput(current.impl->order); + return; + } + } + + while (!current_keys.empty()) + { + const auto & row = current_keys.front(); + auto gap = current_keys.frontGap(); + + insertRow(gap, row); + current_keys.popFront(); + + if (merged_data.hasEnoughRows()) + return; + } + + /// Write information about last collapsed rows. + insertGap(current_keys.frontGap()); + finish(); +} + + +} diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h new file mode 100644 index 00000000000..03b5be9edc6 --- /dev/null +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h @@ -0,0 +1,225 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/* Deque with fixed memory size. Allows pushing gaps. + * frontGap() returns the number of gaps were inserted before front. + * + * This structure may be implemented via std::deque, but + * - Deque uses fixed amount of memory which is allocated in constructor. No more allocations are performed. + * - Gaps are not stored as separate values in queue, which is more memory efficient. + * - Deque is responsible for gaps invariant: after removing element, moves gaps into neighbor cell. + * + * Note: empty deque may have non-zero front gap. + */ +template +class FixedSizeDequeWithGaps +{ +public: + + struct ValueWithGap + { + /// The number of gaps before current element. The number of gaps after last element stores into end cell. + size_t gap; + /// Store char[] instead of T in order to make ValueWithGap POD. + /// Call placement constructors after push and and destructors after pop. + char value[sizeof(T)]; + }; + + explicit FixedSizeDequeWithGaps(size_t size) + { + container.resize_fill(size + 1); + } + + ~FixedSizeDequeWithGaps() + { + auto destruct_range = [this](size_t from, size_t to) + { + for (size_t i = from; i < to; ++i) + destructValue(i); + }; + + if (begin <= end) + destruct_range(begin, end); + else + { + destruct_range(0, end); + destruct_range(begin, container.size()); + } + } + + void pushBack(const T & value) + { + checkEnoughSpaceToInsert(); + constructValue(end, value); + moveRight(end); + container[end].gap = 0; + } + + void pushGap(size_t count) { container[end].gap += count; } + + void popBack() + { + checkHasValuesToRemove(); + size_t curr_gap = container[end].gap; + moveLeft(end); + destructValue(end); + container[end].gap += curr_gap; + } + + void popFront() + { + checkHasValuesToRemove(); + destructValue(begin); + moveRight(begin); + } + + T & front() + { + checkHasValuesToGet(); + return getValue(begin); + } + const T & front() const + { + checkHasValuesToGet(); + return getValue(begin); + } + + const T & back() const + { + size_t ps = end; + moveLeft(ps); + return getValue(ps); + } + + size_t & frontGap() { return container[begin].gap; } + const size_t & frontGap() const { return container[begin].gap; } + + size_t size() const + { + if (begin <= end) + return end - begin; + return end + (container.size() - begin); + } + + bool empty() const { return begin == end; } + +private: + PODArray container; + + size_t gap_before_first = 0; + size_t begin = 0; + size_t end = 0; + + void constructValue(size_t index, const T & value) { new (container[index].value) T(value); } + void destructValue(size_t index) { reinterpret_cast(container[index].value)->~T(); } + + T & getValue(size_t index) { return *reinterpret_cast(container[index].value); } + const T & getValue(size_t index) const { return *reinterpret_cast(container[index].value); } + + void moveRight(size_t & index) const + { + ++index; + + if (index == container.size()) + index = 0; + } + + void moveLeft(size_t & index) const + { + if (index == 0) + index = container.size(); + + --index; + } + + void checkEnoughSpaceToInsert() const + { + if (size() + 1 == container.size()) + throw Exception("Not enough space to insert into FixedSizeDequeWithGaps with capacity " + + toString(container.size() - 1), ErrorCodes::LOGICAL_ERROR); + } + + void checkHasValuesToRemove() const + { + if (empty()) + throw Exception("Cannot remove from empty FixedSizeDequeWithGaps", ErrorCodes::LOGICAL_ERROR); + } + + void checkHasValuesToGet() const + { + if (empty()) + throw Exception("Cannot get value from empty FixedSizeDequeWithGaps", ErrorCodes::LOGICAL_ERROR); + } +}; + +class VersionedCollapsingTransform : public IMergingTransform +{ +public: + /// Don't need version column. It's in primary key. + VersionedCollapsingTransform( + size_t num_inputs, const Block & header, + SortDescription description_, const String & sign_column_, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_ = nullptr, + bool use_average_block_sizes = false); + + String getName() const override { return "VersionedCollapsingTransform"; } + void work() override; + +protected: + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + +private: + Logger * log = &Logger::get("VersionedCollapsingTransform"); + + SortDescription description; + size_t sign_column_number = 0; + + /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) + /// If it is not nullptr then it should be populated during execution + WriteBuffer * out_row_sources_buf = nullptr; + + /// Chunks currently being merged. + using SourceChunks = std::vector; + SourceChunks source_chunks; + SortCursorImpls cursors; + + SortingHeap queue; + bool is_queue_initialized = false; + + using RowRef = detail::RowRef; + const size_t max_rows_in_queue; + /// Rows with the same primary key and sign. + FixedSizeDequeWithGaps current_keys; + Int8 sign_in_queue = 0; + + detail::SharedChunkAllocator chunk_allocator; + + std::queue current_row_sources; /// Sources of rows with the current primary key + + void insertGap(size_t gap_size); + void insertRow(size_t skip_rows, const RowRef & row); + void merge(); + void updateCursor(Chunk chunk, size_t source_num); + void setRowRef(RowRef & row, SortCursor & cursor) { row.set(cursor, source_chunks[cursor.impl->order]); } +}; + +} From d77a907df4f6cefb9c25cc8a1d159f8568c9f696 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Mar 2020 19:42:57 +0300 Subject: [PATCH 062/743] Fix build. --- dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp | 6 +++--- dbms/src/Processors/Merges/VersionedCollapsingTransform.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp index b4caeb22bf8..bc19931e9cc 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp @@ -16,9 +16,9 @@ VersionedCollapsingTransform::VersionedCollapsingTransform( : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, true) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) - , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 2) - , current_keys(max_rows_in_queue + 1) - , chunk_allocator(num_inputs + max_rows_in_queue + 1) + , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 1) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer + , current_keys(max_rows_in_queue) + , chunk_allocator(num_inputs + max_rows_in_queue + 1) /// +1 just in case (for current_row) { sign_column_number = header.getPositionByName(sign_column_); } diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h index 03b5be9edc6..4be0eb41438 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h @@ -153,7 +153,7 @@ private: { if (size() + 1 == container.size()) throw Exception("Not enough space to insert into FixedSizeDequeWithGaps with capacity " - + toString(container.size() - 1), ErrorCodes::LOGICAL_ERROR); + + std::to_string(container.size() - 1), ErrorCodes::LOGICAL_ERROR); } void checkHasValuesToRemove() const From fa9399d82607e3f799bae53a5770a70b7dad22cb Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 30 Mar 2020 21:00:38 +0300 Subject: [PATCH 063/743] [demo] Create unique columns for literals --- dbms/src/Interpreters/ActionsVisitor.cpp | 43 +++++++++++++------ dbms/src/Parsers/ASTLiteral.h | 2 + .../01101_literal_column_clash.reference | 3 ++ .../01101_literal_column_clash.sql | 10 +++++ 4 files changed, 46 insertions(+), 12 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01101_literal_column_clash.reference create mode 100644 dbms/tests/queries/0_stateless/01101_literal_column_clash.sql diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 4e008a81973..b2091465be9 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -197,10 +197,17 @@ SetPtr makeExplicitSet( static String getUniqueName(const Block & block, const String & prefix) { - int i = 1; - while (block.has(prefix + toString(i))) - ++i; - return prefix + toString(i); + auto result = prefix; + + if (block.has(prefix)) + { + int i = 1; + while (block.has(prefix + toString(i))) + ++i; + result = prefix + "_" + toString(i); + } + + return result; } ScopeStack::ScopeStack(const ExpressionActionsPtr & actions, const Context & context_) @@ -431,7 +438,6 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & for (size_t arg = 0; arg < node.arguments->children.size(); ++arg) { auto & child = node.arguments->children[arg]; - auto child_column_name = child->getColumnName(); const auto * lambda = child->as(); const auto * identifier = child->as(); @@ -461,7 +467,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!prepared_set->empty()) column.name = getUniqueName(data.getSampleBlock(), "__set"); else - column.name = child_column_name; + column.name = child->getColumnName(); if (!data.hasColumn(column.name)) { @@ -496,6 +502,18 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & { /// If the argument is not a lambda expression, call it recursively and find out its type. visit(child, data); + + // In the above visit() call, if the argument is a literal, we + // generated a unique column name for it. Use it instead of a generic + // display name. + auto child_column_name = child->getColumnName(); + auto asLiteral = dynamic_cast(child.get()); + if (asLiteral) + { + assert(!asLiteral->unique_column_name.empty()); + child_column_name = asLiteral->unique_column_name; + } + if (data.hasColumn(child_column_name)) { argument_types.push_back(data.getSampleBlock().getByName(child_column_name).type); @@ -587,18 +605,19 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & } } -void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data) +void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, + Data & data) { - CachedColumnName column_name; - if (data.hasColumn(column_name.get(ast))) - return; - DataTypePtr type = applyVisitor(FieldToDataType(), literal.value); ColumnWithTypeAndName column; column.column = type->createColumnConst(1, convertFieldToType(literal.value, *type)); column.type = type; - column.name = column_name.get(ast); + + // Always create columns for literals with a unique name. Otherwise, there + // may be some weird clashes, see 01101_literal_column_clash. + column.name = getUniqueName(data.getSampleBlock(), literal.getColumnName()); + const_cast(literal).unique_column_name = column.name; data.addAction(ExpressionAction::addColumn(column)); } diff --git a/dbms/src/Parsers/ASTLiteral.h b/dbms/src/Parsers/ASTLiteral.h index 552f5da04a2..e9fb3d3b8ec 100644 --- a/dbms/src/Parsers/ASTLiteral.h +++ b/dbms/src/Parsers/ASTLiteral.h @@ -17,6 +17,8 @@ class ASTLiteral : public ASTWithAlias public: Field value; + String unique_column_name; + /// For ConstantExpressionTemplate std::optional begin; std::optional end; diff --git a/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference b/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference new file mode 100644 index 00000000000..0dc94464bfc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference @@ -0,0 +1,3 @@ +1 +7 3 +xyzabc 2 diff --git a/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql b/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql new file mode 100644 index 00000000000..11e3b622277 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql @@ -0,0 +1,10 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/9810 +select cast(1 as String) +from (select 1 as iid) as t1 +join (select '1' as sid) as t2 on t2.sid = cast(t1.iid as String); + +-- even simpler cases +select cast(7 as String), * from (select 3 "'String'"); +SELECT concat('xyz', 'abc'), * FROM (SELECT 2 AS "'xyz'"); + + From 78f0d4c5cbd9ea4b04f31c3f7c20a122796f56d7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 30 Mar 2020 22:15:18 +0300 Subject: [PATCH 064/743] check table finally detached before attaching --- dbms/src/Databases/DatabaseAtomic.cpp | 24 +++++++++++++++++++ dbms/src/Databases/DatabaseAtomic.h | 7 +++--- .../01107_atomic_db_detach_attach.reference | 2 ++ .../01107_atomic_db_detach_attach.sh | 20 ++++++++++++++++ 4 files changed, 50 insertions(+), 3 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01107_atomic_db_detach_attach.reference create mode 100755 dbms/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index 72b7ae05dd8..97f8414ced6 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -52,6 +52,7 @@ void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, { assert(relative_table_path != data_path && !relative_table_path.empty()); std::lock_guard lock(mutex); + assertDetachedTableNotInUse(table->getStorageID().uuid); DatabaseWithDictionaries::attachTableUnlocked(name, table, relative_table_path); table_name_to_path.emplace(std::make_pair(name, relative_table_path)); } @@ -61,6 +62,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) std::lock_guard lock(mutex); auto table = DatabaseWithDictionaries::detachTableUnlocked(name); table_name_to_path.erase(name); + detached_tables.emplace(table->getStorageID().uuid, table); return table; } @@ -138,6 +140,7 @@ void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore void DatabaseAtomic::shutdown() { + DatabaseWithDictionaries::shutdown(); } @@ -148,6 +151,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora try { std::lock_guard lock{mutex}; + assertDetachedTableNotInUse(query.uuid); renameNoReplace(table_metadata_tmp_path, table_metadata_path); attachTableUnlocked(query.table, table, table_data_path); /// Should never throw table_name_to_path.emplace(query.table, table_data_path); @@ -173,5 +177,25 @@ void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & renameExchange(table_metadata_tmp_path, table_metadata_path); } +void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid) +{ + cleenupDetachedTables(); + if (detached_tables.count(uuid)) + throw Exception("Cannot attach table with UUID " + toString(uuid) + + ", because it was detached but still used by come query. Retry later.", ErrorCodes::TABLE_ALREADY_EXISTS); +} + +void DatabaseAtomic::cleenupDetachedTables() +{ + auto it = detached_tables.begin(); + while (it != detached_tables.end()) + { + if (it->second.unique()) + it = detached_tables.erase(it); + else + ++it; + } +} + } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index c1c0a8fc75a..f2d93167cab 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -22,7 +22,6 @@ public: IDatabase & to_database, const String & to_table_name) override; - //void removeTable(const Context & context, const String & table_name) override; void dropTable(const Context & context, const String & table_name, bool no_delay) override; void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; @@ -47,11 +46,13 @@ private: void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path) override; + void assertDetachedTableNotInUse(const UUID & uuid); + void cleenupDetachedTables(); + //TODO store path in DatabaseWithOwnTables::tables std::map table_name_to_path; - - + std::map detached_tables; }; } diff --git a/dbms/tests/queries/0_stateless/01107_atomic_db_detach_attach.reference b/dbms/tests/queries/0_stateless/01107_atomic_db_detach_attach.reference new file mode 100644 index 00000000000..a08773dab53 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01107_atomic_db_detach_attach.reference @@ -0,0 +1,2 @@ +OK +5 10 diff --git a/dbms/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/dbms/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh new file mode 100755 index 00000000000..e0b46c4e7b7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" + +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" & +sleep 1 + +$CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" +$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK" + +sleep 5 +$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" + +$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" From 4e668a5b06974afe45e7c6cbdb74e5c98297fae1 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 30 Mar 2020 23:41:29 +0300 Subject: [PATCH 065/743] fixup --- dbms/src/Interpreters/ActionsVisitor.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index b2091465be9..fd36cd27a0e 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -199,12 +199,15 @@ static String getUniqueName(const Block & block, const String & prefix) { auto result = prefix; - if (block.has(prefix)) + if (block.has(result)) { int i = 1; - while (block.has(prefix + toString(i))) + do + { + result = prefix + "_" + toString(i); ++i; - result = prefix + "_" + toString(i); + } + while (block.has(result)); } return result; From ebb5cc05422d2397c992945b741270ba68729786 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 30 Mar 2020 23:47:28 +0300 Subject: [PATCH 066/743] fixup --- dbms/src/Interpreters/ActionsVisitor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index fd36cd27a0e..eb92ee76973 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -510,11 +510,11 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & // generated a unique column name for it. Use it instead of a generic // display name. auto child_column_name = child->getColumnName(); - auto asLiteral = dynamic_cast(child.get()); - if (asLiteral) + auto as_literal = dynamic_cast(child.get()); + if (as_literal) { - assert(!asLiteral->unique_column_name.empty()); - child_column_name = asLiteral->unique_column_name; + assert(!as_literal->unique_column_name.empty()); + child_column_name = as_literal->unique_column_name; } if (data.hasColumn(child_column_name)) From de36104ab27c6a22a0c55210e08c3505100ddfa4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Mar 2020 13:42:42 +0300 Subject: [PATCH 067/743] Refactor MergingTransforms. --- .../Merges/CollapsingSortedTransform.cpp | 6 +- .../Merges/CollapsingSortedTransform.h | 3 + .../Merges/FixedSizeDequeWithGaps.h | 162 ++++++++++++++++++ .../Processors/Merges/IMergingTransform.cpp | 15 +- .../src/Processors/Merges/IMergingTransform.h | 130 +------------- dbms/src/Processors/Merges/MergedData.h | 122 +++++++++++++ .../Merges/MergingSortedTransform.cpp | 13 +- .../Merges/MergingSortedTransform.h | 7 +- .../Merges/ReplacingSortedTransform.cpp | 6 +- .../Merges/ReplacingSortedTransform.h | 3 + .../Merges/VersionedCollapsingTransform.cpp | 6 +- .../Merges/VersionedCollapsingTransform.h | 160 +---------------- 12 files changed, 336 insertions(+), 297 deletions(-) create mode 100644 dbms/src/Processors/Merges/FixedSizeDequeWithGaps.h create mode 100644 dbms/src/Processors/Merges/MergedData.h diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp index 4bb4eb0af43..13a61d26caa 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp @@ -24,7 +24,8 @@ CollapsingSortedTransform::CollapsingSortedTransform( size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) - : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, true) + : IMergingTransform(num_inputs, header, header, true) + , merged_data(header, use_average_block_sizes, max_block_size) , description(std::move(description_)) , sign_column_number(header.getPositionByName(sign_column)) , out_row_sources_buf(out_row_sources_buf_) @@ -150,6 +151,7 @@ void CollapsingSortedTransform::insertRows() void CollapsingSortedTransform::work() { merge(); + prepareOutputChunk(merged_data); } void CollapsingSortedTransform::merge() @@ -229,7 +231,7 @@ void CollapsingSortedTransform::merge() } insertRows(); - finish(); + is_finished = true; } } diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.h b/dbms/src/Processors/Merges/CollapsingSortedTransform.h index 64f9016b90d..58c97f964bc 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.h +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -45,6 +46,8 @@ protected: private: Logger * log = &Logger::get("CollapsingSortedTransform"); + MergedData merged_data; + /// Settings SortDescription description; const size_t sign_column_number; diff --git a/dbms/src/Processors/Merges/FixedSizeDequeWithGaps.h b/dbms/src/Processors/Merges/FixedSizeDequeWithGaps.h new file mode 100644 index 00000000000..35cfded4214 --- /dev/null +++ b/dbms/src/Processors/Merges/FixedSizeDequeWithGaps.h @@ -0,0 +1,162 @@ +#pragma once + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + +/* Deque with fixed memory size. Allows pushing gaps. + * frontGap() returns the number of gaps were inserted before front. + * + * This structure may be implemented via std::deque, but + * - Deque uses fixed amount of memory which is allocated in constructor. No more allocations are performed. + * - Gaps are not stored as separate values in queue, which is more memory efficient. + * - Deque is responsible for gaps invariant: after removing element, moves gaps into neighbor cell. + * + * Note: empty deque may have non-zero front gap. + */ +template +class FixedSizeDequeWithGaps +{ +public: + + struct ValueWithGap + { + /// The number of gaps before current element. The number of gaps after last element stores into end cell. + size_t gap; + /// Store char[] instead of T in order to make ValueWithGap POD. + /// Call placement constructors after push and and destructors after pop. + char value[sizeof(T)]; + }; + + explicit FixedSizeDequeWithGaps(size_t size) + { + container.resize_fill(size + 1); + } + + ~FixedSizeDequeWithGaps() + { + auto destruct_range = [this](size_t from, size_t to) + { + for (size_t i = from; i < to; ++i) + destructValue(i); + }; + + if (begin <= end) + destruct_range(begin, end); + else + { + destruct_range(0, end); + destruct_range(begin, container.size()); + } + } + + void pushBack(const T & value) + { + checkEnoughSpaceToInsert(); + constructValue(end, value); + moveRight(end); + container[end].gap = 0; + } + + void pushGap(size_t count) { container[end].gap += count; } + + void popBack() + { + checkHasValuesToRemove(); + size_t curr_gap = container[end].gap; + moveLeft(end); + destructValue(end); + container[end].gap += curr_gap; + } + + void popFront() + { + checkHasValuesToRemove(); + destructValue(begin); + moveRight(begin); + } + + T & front() + { + checkHasValuesToGet(); + return getValue(begin); + } + const T & front() const + { + checkHasValuesToGet(); + return getValue(begin); + } + + const T & back() const + { + size_t ps = end; + moveLeft(ps); + return getValue(ps); + } + + size_t & frontGap() { return container[begin].gap; } + const size_t & frontGap() const { return container[begin].gap; } + + size_t size() const + { + if (begin <= end) + return end - begin; + return end + (container.size() - begin); + } + + bool empty() const { return begin == end; } + +private: + PODArray container; + + size_t gap_before_first = 0; + size_t begin = 0; + size_t end = 0; + + void constructValue(size_t index, const T & value) { new (container[index].value) T(value); } + void destructValue(size_t index) { reinterpret_cast(container[index].value)->~T(); } + + T & getValue(size_t index) { return *reinterpret_cast(container[index].value); } + const T & getValue(size_t index) const { return *reinterpret_cast(container[index].value); } + + void moveRight(size_t & index) const + { + ++index; + + if (index == container.size()) + index = 0; + } + + void moveLeft(size_t & index) const + { + if (index == 0) + index = container.size(); + + --index; + } + + void checkEnoughSpaceToInsert() const + { + if (size() + 1 == container.size()) + throw Exception("Not enough space to insert into FixedSizeDequeWithGaps with capacity " + + std::to_string(container.size() - 1), ErrorCodes::LOGICAL_ERROR); + } + + void checkHasValuesToRemove() const + { + if (empty()) + throw Exception("Cannot remove from empty FixedSizeDequeWithGaps", ErrorCodes::LOGICAL_ERROR); + } + + void checkHasValuesToGet() const + { + if (empty()) + throw Exception("Cannot get value from empty FixedSizeDequeWithGaps", ErrorCodes::LOGICAL_ERROR); + } +}; + +} diff --git a/dbms/src/Processors/Merges/IMergingTransform.cpp b/dbms/src/Processors/Merges/IMergingTransform.cpp index b0226ce0f3d..0838dda6323 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.cpp +++ b/dbms/src/Processors/Merges/IMergingTransform.cpp @@ -13,11 +13,8 @@ IMergingTransform::IMergingTransform( size_t num_inputs, const Block & input_header, const Block & output_header, - size_t max_block_size, - bool use_average_block_size, bool have_all_inputs_) : IProcessor(InputPorts(num_inputs, input_header), {output_header}) - , merged_data(output_header, use_average_block_size, max_block_size) , have_all_inputs(have_all_inputs_) { } @@ -55,6 +52,13 @@ void IMergingTransform::requestDataForInput(size_t input_number) next_input_to_read = input_number; } +void IMergingTransform::prepareOutputChunk(MergedData & merged_data) +{ + has_output_chunk = (is_finished && merged_data.mergedRows()) || merged_data.hasEnoughRows(); + if (has_output_chunk) + output_chunk = merged_data.pull(); +} + IProcessor::Status IMergingTransform::prepareSingleInput() { auto & input = inputs.front(); @@ -171,9 +175,8 @@ IProcessor::Status IMergingTransform::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - bool has_data_to_push = (is_finished && merged_data.mergedRows()) || merged_data.hasEnoughRows(); - if (has_data_to_push && !is_port_full) - output.push(merged_data.pull()); + if (has_output_chunk && !is_port_full) + output.push(std::move(output_chunk)); if (!is_initialized) return prepareInitializeInputs(); diff --git a/dbms/src/Processors/Merges/IMergingTransform.h b/dbms/src/Processors/Merges/IMergingTransform.h index 040263aca36..a1046d207ad 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.h +++ b/dbms/src/Processors/Merges/IMergingTransform.h @@ -6,10 +6,7 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} +class MergedData; /// Base class for merging transforms. class IMergingTransform : public IProcessor @@ -19,8 +16,8 @@ public: size_t num_inputs, const Block & input_header, const Block & output_header, - size_t max_block_size, - bool use_average_block_size, /// For adaptive granularity. Return chunks with the same avg size as inputs. + //size_t max_block_size, + //bool use_average_block_size, /// For adaptive granularity. Return chunks with the same avg size as inputs. bool have_all_inputs_); /// Methods to add additional input port. It is possible to do only before the first call of `prepare`. @@ -38,129 +35,18 @@ protected: virtual void onFinish() {} /// Is called when all data is processed. void requestDataForInput(size_t input_number); /// Call it to say that next chunk of data is required for input. - void finish() { is_finished = true; } /// Call it when all data was inserted to merged_data. - - /// Struct which represents current merging chunk of data. - /// Also it calculates the number of merged rows and other profile info. - class MergedData - { - public: - explicit MergedData(const Block & header, bool use_average_block_size_, UInt64 max_block_size_) - : max_block_size(max_block_size_), use_average_block_size(use_average_block_size_) - { - columns.reserve(header.columns()); - for (const auto & column : header) - columns.emplace_back(column.type->createColumn()); - } - - /// Pull will be called at next prepare call. - void flush() { need_flush = true; } - - void insertRow(const ColumnRawPtrs & raw_columns, size_t row, size_t block_size) - { - size_t num_columns = raw_columns.size(); - for (size_t i = 0; i < num_columns; ++i) - columns[i]->insertFrom(*raw_columns[i], row); - - ++total_merged_rows; - ++merged_rows; - sum_blocks_granularity += block_size; - } - - void insertFromChunk(Chunk && chunk, size_t limit_rows) - { - if (merged_rows) - throw Exception("Cannot insert to MergedData from Chunk because MergedData is not empty.", - ErrorCodes::LOGICAL_ERROR); - - auto num_rows = chunk.getNumRows(); - columns = chunk.mutateColumns(); - if (limit_rows && num_rows > limit_rows) - { - num_rows = limit_rows; - for (auto & column : columns) - column = (*column->cut(0, num_rows)).mutate(); - } - - need_flush = true; - total_merged_rows += num_rows; - merged_rows = num_rows; - - /// We don't cate about granularity here. Because, for fast-forward optimization, chunk will be moved as-is. - /// sum_blocks_granularity += block_size * num_rows; - } - - Chunk pull() - { - MutableColumns empty_columns; - empty_columns.reserve(columns.size()); - - for (const auto & column : columns) - empty_columns.emplace_back(column->cloneEmpty()); - - empty_columns.swap(columns); - Chunk chunk(std::move(empty_columns), merged_rows); - - merged_rows = 0; - sum_blocks_granularity = 0; - ++total_chunks; - total_allocated_bytes += chunk.allocatedBytes(); - need_flush = false; - - return chunk; - } - - bool hasEnoughRows() const - { - /// If full chunk was or is going to be inserted, then we must pull it. - /// It is needed for fast-forward optimization. - if (need_flush) - return true; - - /// Never return more then max_block_size. - if (merged_rows >= max_block_size) - return true; - - if (!use_average_block_size) - return false; - - /// Zero rows always not enough. - if (merged_rows == 0) - return false; - - return merged_rows * merged_rows >= sum_blocks_granularity; - } - - UInt64 mergedRows() const { return merged_rows; } - UInt64 totalMergedRows() const { return total_merged_rows; } - UInt64 totalChunks() const { return total_chunks; } - UInt64 totalAllocatedBytes() const { return total_allocated_bytes; } - - private: - MutableColumns columns; - - UInt64 sum_blocks_granularity = 0; - UInt64 merged_rows = 0; - UInt64 total_merged_rows = 0; - UInt64 total_chunks = 0; - UInt64 total_allocated_bytes = 0; - - const UInt64 max_block_size; - const bool use_average_block_size; - - bool need_flush = false; - }; - - MergedData merged_data; + void prepareOutputChunk(MergedData & merged_data); /// Moves chunk from merged_data to output_chunk if needed. /// Profile info. Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + Chunk output_chunk; + bool has_output_chunk = false; + bool is_finished = false; + private: /// Processor state. bool is_initialized = false; - bool is_finished = false; - bool need_data = false; size_t next_input_to_read = 0; diff --git a/dbms/src/Processors/Merges/MergedData.h b/dbms/src/Processors/Merges/MergedData.h new file mode 100644 index 00000000000..6f66d584d91 --- /dev/null +++ b/dbms/src/Processors/Merges/MergedData.h @@ -0,0 +1,122 @@ +#pragma once + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/// Class which represents current merging chunk of data. +/// Also it calculates the number of merged rows and other profile info. +class MergedData +{ +public: + explicit MergedData(const Block & header, bool use_average_block_size_, UInt64 max_block_size_) + : max_block_size(max_block_size_), use_average_block_size(use_average_block_size_) + { + columns.reserve(header.columns()); + for (const auto & column : header) + columns.emplace_back(column.type->createColumn()); + } + + /// Pull will be called at next prepare call. + void flush() { need_flush = true; } + + void insertRow(const ColumnRawPtrs & raw_columns, size_t row, size_t block_size) + { + size_t num_columns = raw_columns.size(); + for (size_t i = 0; i < num_columns; ++i) + columns[i]->insertFrom(*raw_columns[i], row); + + ++total_merged_rows; + ++merged_rows; + sum_blocks_granularity += block_size; + } + + void insertFromChunk(Chunk && chunk, size_t limit_rows) + { + if (merged_rows) + throw Exception("Cannot insert to MergedData from Chunk because MergedData is not empty.", + ErrorCodes::LOGICAL_ERROR); + + auto num_rows = chunk.getNumRows(); + columns = chunk.mutateColumns(); + if (limit_rows && num_rows > limit_rows) + { + num_rows = limit_rows; + for (auto & column : columns) + column = (*column->cut(0, num_rows)).mutate(); + } + + need_flush = true; + total_merged_rows += num_rows; + merged_rows = num_rows; + + /// We don't cate about granularity here. Because, for fast-forward optimization, chunk will be moved as-is. + /// sum_blocks_granularity += block_size * num_rows; + } + + Chunk pull() + { + MutableColumns empty_columns; + empty_columns.reserve(columns.size()); + + for (const auto & column : columns) + empty_columns.emplace_back(column->cloneEmpty()); + + empty_columns.swap(columns); + Chunk chunk(std::move(empty_columns), merged_rows); + + merged_rows = 0; + sum_blocks_granularity = 0; + ++total_chunks; + total_allocated_bytes += chunk.allocatedBytes(); + need_flush = false; + + return chunk; + } + + bool hasEnoughRows() const + { + /// If full chunk was or is going to be inserted, then we must pull it. + /// It is needed for fast-forward optimization. + if (need_flush) + return true; + + /// Never return more then max_block_size. + if (merged_rows >= max_block_size) + return true; + + if (!use_average_block_size) + return false; + + /// Zero rows always not enough. + if (merged_rows == 0) + return false; + + return merged_rows * merged_rows >= sum_blocks_granularity; + } + + UInt64 mergedRows() const { return merged_rows; } + UInt64 totalMergedRows() const { return total_merged_rows; } + UInt64 totalChunks() const { return total_chunks; } + UInt64 totalAllocatedBytes() const { return total_allocated_bytes; } + +private: + MutableColumns columns; + + UInt64 sum_blocks_granularity = 0; + UInt64 merged_rows = 0; + UInt64 total_merged_rows = 0; + UInt64 total_chunks = 0; + UInt64 total_allocated_bytes = 0; + + const UInt64 max_block_size; + const bool use_average_block_size; + + bool need_flush = false; +}; + +} diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index 734da2e4bb3..226b29c36dc 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -23,7 +23,8 @@ MergingSortedTransform::MergingSortedTransform( bool quiet_, bool use_average_block_sizes, bool have_all_inputs_) - : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, have_all_inputs_) + : IMergingTransform(num_inputs, header, header, have_all_inputs_) + , merged_data(header, use_average_block_sizes, max_block_size) , description(std::move(description_)) , limit(limit_) , quiet(quiet_) @@ -103,6 +104,8 @@ void MergingSortedTransform::work() merge(queue_with_collation); else merge(queue_without_collation); + + prepareOutputChunk(merged_data); } template @@ -114,7 +117,7 @@ void MergingSortedTransform::merge(TSortingHeap & queue) if (limit && merged_data.totalMergedRows() >= limit) { //std::cerr << "Limit reached\n"; - finish(); + is_finished = true; return false; } @@ -179,13 +182,13 @@ void MergingSortedTransform::merge(TSortingHeap & queue) requestDataForInput(current.impl->order); if (limit && merged_data.totalMergedRows() >= limit) - finish(); + is_finished = true; return; } } - finish(); + is_finished = true; } void MergingSortedTransform::insertFromChunk(size_t source_num) @@ -202,7 +205,7 @@ void MergingSortedTransform::insertFromChunk(size_t source_num) { num_rows = total_merged_rows_after_insertion - limit; merged_data.insertFromChunk(std::move(source_chunks[source_num]), num_rows); - finish(); + is_finished = true; } else { diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.h b/dbms/src/Processors/Merges/MergingSortedTransform.h index f8e3e65951f..49bfe228106 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.h +++ b/dbms/src/Processors/Merges/MergingSortedTransform.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -26,13 +27,15 @@ public: String getName() const override { return "MergingSortedTransform"; } void work() override; -private: - +protected: void onNewInput() override; void initializeInputs() override; void consume(Chunk chunk, size_t input_number) override; void onFinish() override; +private: + MergedData merged_data; + /// Settings SortDescription description; UInt64 limit; diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp index 18fb9f98221..2de67707d6d 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp @@ -15,7 +15,8 @@ ReplacingSortedTransform::ReplacingSortedTransform( size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) - : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, true) + : IMergingTransform(num_inputs, header, header, true) + , merged_data(header, use_average_block_sizes, max_block_size) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) , chunk_allocator(num_inputs + max_row_refs) @@ -86,6 +87,7 @@ void ReplacingSortedTransform::insertRow() void ReplacingSortedTransform::work() { merge(); + prepareOutputChunk(merged_data); } void ReplacingSortedTransform::merge() @@ -148,7 +150,7 @@ void ReplacingSortedTransform::merge() if (!selected_row.empty()) insertRow(); - finish(); + is_finished = true; } } diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.h b/dbms/src/Processors/Merges/ReplacingSortedTransform.h index 85fd0d23fe4..4f4b71c5b13 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.h +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -32,6 +33,8 @@ protected: private: Logger * log = &Logger::get("ReplacingSortedTransform"); + MergedData merged_data; + SortDescription description; ssize_t version_column_number = -1; diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp index bc19931e9cc..9191123c878 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp @@ -13,7 +13,8 @@ VersionedCollapsingTransform::VersionedCollapsingTransform( size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) - : IMergingTransform(num_inputs, header, header, max_block_size, use_average_block_sizes, true) + : IMergingTransform(num_inputs, header, header, true) + , merged_data(header, use_average_block_sizes, max_block_size) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 1) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer @@ -69,6 +70,7 @@ void VersionedCollapsingTransform::updateCursor(Chunk chunk, size_t source_num) void VersionedCollapsingTransform::work() { merge(); + prepareOutputChunk(merged_data); } inline ALWAYS_INLINE static void writeRowSourcePart(WriteBuffer & buffer, RowSourcePart row_source) @@ -193,7 +195,7 @@ void VersionedCollapsingTransform::merge() /// Write information about last collapsed rows. insertGap(current_keys.frontGap()); - finish(); + is_finished = true; } diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h index 4be0eb41438..722bd30feca 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include #include @@ -13,162 +15,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -/* Deque with fixed memory size. Allows pushing gaps. - * frontGap() returns the number of gaps were inserted before front. - * - * This structure may be implemented via std::deque, but - * - Deque uses fixed amount of memory which is allocated in constructor. No more allocations are performed. - * - Gaps are not stored as separate values in queue, which is more memory efficient. - * - Deque is responsible for gaps invariant: after removing element, moves gaps into neighbor cell. - * - * Note: empty deque may have non-zero front gap. - */ -template -class FixedSizeDequeWithGaps -{ -public: - - struct ValueWithGap - { - /// The number of gaps before current element. The number of gaps after last element stores into end cell. - size_t gap; - /// Store char[] instead of T in order to make ValueWithGap POD. - /// Call placement constructors after push and and destructors after pop. - char value[sizeof(T)]; - }; - - explicit FixedSizeDequeWithGaps(size_t size) - { - container.resize_fill(size + 1); - } - - ~FixedSizeDequeWithGaps() - { - auto destruct_range = [this](size_t from, size_t to) - { - for (size_t i = from; i < to; ++i) - destructValue(i); - }; - - if (begin <= end) - destruct_range(begin, end); - else - { - destruct_range(0, end); - destruct_range(begin, container.size()); - } - } - - void pushBack(const T & value) - { - checkEnoughSpaceToInsert(); - constructValue(end, value); - moveRight(end); - container[end].gap = 0; - } - - void pushGap(size_t count) { container[end].gap += count; } - - void popBack() - { - checkHasValuesToRemove(); - size_t curr_gap = container[end].gap; - moveLeft(end); - destructValue(end); - container[end].gap += curr_gap; - } - - void popFront() - { - checkHasValuesToRemove(); - destructValue(begin); - moveRight(begin); - } - - T & front() - { - checkHasValuesToGet(); - return getValue(begin); - } - const T & front() const - { - checkHasValuesToGet(); - return getValue(begin); - } - - const T & back() const - { - size_t ps = end; - moveLeft(ps); - return getValue(ps); - } - - size_t & frontGap() { return container[begin].gap; } - const size_t & frontGap() const { return container[begin].gap; } - - size_t size() const - { - if (begin <= end) - return end - begin; - return end + (container.size() - begin); - } - - bool empty() const { return begin == end; } - -private: - PODArray container; - - size_t gap_before_first = 0; - size_t begin = 0; - size_t end = 0; - - void constructValue(size_t index, const T & value) { new (container[index].value) T(value); } - void destructValue(size_t index) { reinterpret_cast(container[index].value)->~T(); } - - T & getValue(size_t index) { return *reinterpret_cast(container[index].value); } - const T & getValue(size_t index) const { return *reinterpret_cast(container[index].value); } - - void moveRight(size_t & index) const - { - ++index; - - if (index == container.size()) - index = 0; - } - - void moveLeft(size_t & index) const - { - if (index == 0) - index = container.size(); - - --index; - } - - void checkEnoughSpaceToInsert() const - { - if (size() + 1 == container.size()) - throw Exception("Not enough space to insert into FixedSizeDequeWithGaps with capacity " - + std::to_string(container.size() - 1), ErrorCodes::LOGICAL_ERROR); - } - - void checkHasValuesToRemove() const - { - if (empty()) - throw Exception("Cannot remove from empty FixedSizeDequeWithGaps", ErrorCodes::LOGICAL_ERROR); - } - - void checkHasValuesToGet() const - { - if (empty()) - throw Exception("Cannot get value from empty FixedSizeDequeWithGaps", ErrorCodes::LOGICAL_ERROR); - } -}; - class VersionedCollapsingTransform : public IMergingTransform { public: @@ -190,6 +36,8 @@ protected: private: Logger * log = &Logger::get("VersionedCollapsingTransform"); + MergedData merged_data; + SortDescription description; size_t sign_column_number = 0; From 67073f8b100032969deaa335deb94db7ac2fd8ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Mar 2020 13:44:06 +0300 Subject: [PATCH 068/743] Fix build. --- dbms/src/Processors/Merges/IMergingTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Processors/Merges/IMergingTransform.cpp b/dbms/src/Processors/Merges/IMergingTransform.cpp index 0838dda6323..7c1b58d30f5 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.cpp +++ b/dbms/src/Processors/Merges/IMergingTransform.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { From 5f5eb58abe767938c32eeda8a36c933d9a991ddf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Mar 2020 17:11:58 +0300 Subject: [PATCH 069/743] Add SummingSortedTransform [part 1] --- dbms/src/Processors/Merges/MergedData.h | 2 +- .../Merges/SummingSortedTransform.cpp | 264 ++++++++++++++++++ .../Merges/SummingSortedTransform.h | 101 +++++++ 3 files changed, 366 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Processors/Merges/SummingSortedTransform.cpp create mode 100644 dbms/src/Processors/Merges/SummingSortedTransform.h diff --git a/dbms/src/Processors/Merges/MergedData.h b/dbms/src/Processors/Merges/MergedData.h index 6f66d584d91..e5a8a541aa5 100644 --- a/dbms/src/Processors/Merges/MergedData.h +++ b/dbms/src/Processors/Merges/MergedData.h @@ -104,7 +104,7 @@ public: UInt64 totalChunks() const { return total_chunks; } UInt64 totalAllocatedBytes() const { return total_allocated_bytes; } -private: +protected: MutableColumns columns; UInt64 sum_blocks_granularity = 0; diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp new file mode 100644 index 00000000000..0c4052e821d --- /dev/null +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -0,0 +1,264 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ +bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number) +{ + for (auto & desc : description) + if (desc.column_name == name || (desc.column_name.empty() && desc.column_number == number)) + return true; + + return false; +} + +/// Returns true if merge result is not empty +bool mergeMap(const SummingSortedTransform::MapDescription & desc, Row & row, SortCursor & cursor) +{ + /// Strongly non-optimal. + + Row & left = row; + Row right(left.size()); + + for (size_t col_num : desc.key_col_nums) + right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); + + for (size_t col_num : desc.val_col_nums) + right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); + + auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & + { + return matrix[i].get()[j]; + }; + + auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array + { + size_t size = col_nums.size(); + Array res(size); + for (size_t col_num_index = 0; col_num_index < size; ++col_num_index) + res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j); + return res; + }; + + std::map merged; + + auto accumulate = [](Array & dst, const Array & src) + { + bool has_non_zero = false; + size_t size = dst.size(); + for (size_t i = 0; i < size; ++i) + if (applyVisitor(FieldVisitorSum(src[i]), dst[i])) + has_non_zero = true; + return has_non_zero; + }; + + auto merge = [&](const Row & matrix) + { + size_t rows = matrix[desc.key_col_nums[0]].get().size(); + + for (size_t j = 0; j < rows; ++j) + { + Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j); + Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j); + + auto it = merged.find(key); + if (merged.end() == it) + merged.emplace(std::move(key), std::move(value)); + else + { + if (!accumulate(it->second, value)) + merged.erase(it); + } + } + }; + + merge(left); + merge(right); + + for (size_t col_num : desc.key_col_nums) + row[col_num] = Array(merged.size()); + for (size_t col_num : desc.val_col_nums) + row[col_num] = Array(merged.size()); + + size_t row_num = 0; + for (const auto & key_value : merged) + { + for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) + row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; + + for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) + row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; + + ++row_num; + } + + return row_num != 0; +} +} + +SummingSortedTransform::SummingSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description, + /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. + const Names & column_names_to_sum, + size_t max_block_size) + : IMergingTransform(num_inputs, header, header, true) +{ + size_t num_columns = header.columns(); + current_row.resize(num_columns); + + /// name of nested structure -> the column numbers that refer to it. + std::unordered_map> discovered_maps; + + /** Fill in the column numbers, which must be summed. + * This can only be numeric columns that are not part of the sort key. + * If a non-empty column_names_to_sum is specified, then we only take these columns. + * Some columns from column_names_to_sum may not be found. This is ignored. + */ + for (size_t i = 0; i < num_columns; ++i) + { + const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + + /// Discover nested Maps and find columns for summation + if (typeid_cast(column.type.get())) + { + const auto map_name = Nested::extractTableName(column.name); + /// if nested table name ends with `Map` it is a possible candidate for special handling + if (map_name == column.name || !endsWith(map_name, "Map")) + { + column_numbers_not_to_aggregate.push_back(i); + continue; + } + + discovered_maps[map_name].emplace_back(i); + } + else + { + bool is_agg_func = WhichDataType(column.type).isAggregateFunction(); + + /// There are special const columns for example after prewere sections. + if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column)) + { + column_numbers_not_to_aggregate.push_back(i); + continue; + } + + /// Are they inside the PK? + if (isInPrimaryKey(description, column.name, i)) + { + column_numbers_not_to_aggregate.push_back(i); + continue; + } + + if (column_names_to_sum.empty() + || column_names_to_sum.end() != + std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name)) + { + // Create aggregator to sum this column + AggregateDescription desc; + desc.is_agg_func_type = is_agg_func; + desc.column_numbers = {i}; + + if (!is_agg_func) + { + desc.init("sumWithOverflow", {column.type}); + } + + columns_to_aggregate.emplace_back(std::move(desc)); + } + else + { + // Column is not going to be summed, use last value + column_numbers_not_to_aggregate.push_back(i); + } + } + } + + /// select actual nested Maps from list of candidates + for (const auto & map : discovered_maps) + { + /// map should contain at least two elements (key -> value) + if (map.second.size() < 2) + { + for (auto col : map.second) + column_numbers_not_to_aggregate.push_back(col); + continue; + } + + /// no elements of map could be in primary key + auto column_num_it = map.second.begin(); + for (; column_num_it != map.second.end(); ++column_num_it) + if (isInPrimaryKey(description, header.safeGetByPosition(*column_num_it).name, *column_num_it)) + break; + if (column_num_it != map.second.end()) + { + for (auto col : map.second) + column_numbers_not_to_aggregate.push_back(col); + continue; + } + + DataTypes argument_types; + AggregateDescription desc; + MapDescription map_desc; + + column_num_it = map.second.begin(); + for (; column_num_it != map.second.end(); ++column_num_it) + { + const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it); + const String & name = key_col.name; + const IDataType & nested_type = *static_cast(key_col.type.get())->getNestedType(); + + if (column_num_it == map.second.begin() + || endsWith(name, "ID") + || endsWith(name, "Key") + || endsWith(name, "Type")) + { + if (!nested_type.isValueRepresentedByInteger() && !isStringOrFixedString(nested_type)) + break; + + map_desc.key_col_nums.push_back(*column_num_it); + } + else + { + if (!nested_type.isSummable()) + break; + + map_desc.val_col_nums.push_back(*column_num_it); + } + + // Add column to function arguments + desc.column_numbers.push_back(*column_num_it); + argument_types.push_back(key_col.type); + } + + if (column_num_it != map.second.end()) + { + for (auto col : map.second) + column_numbers_not_to_aggregate.push_back(col); + continue; + } + + if (map_desc.key_col_nums.size() == 1) + { + // Create summation for all value columns in the map + desc.init("sumMapWithOverflow", argument_types); + columns_to_aggregate.emplace_back(std::move(desc)); + } + else + { + // Fall back to legacy mergeMaps for composite keys + for (auto col : map.second) + column_numbers_not_to_aggregate.push_back(col); + maps_to_sum.emplace_back(std::move(map_desc)); + } + } +} + +} diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.h b/dbms/src/Processors/Merges/SummingSortedTransform.h new file mode 100644 index 00000000000..f82c1f9e6db --- /dev/null +++ b/dbms/src/Processors/Merges/SummingSortedTransform.h @@ -0,0 +1,101 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + + + +class SummingSortedTransform : public IMergingTransform +{ +public: + + SummingSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description, + /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. + const Names & column_names_to_sum, + size_t max_block_size); + + /// Stores aggregation function, state, and columns to be used as function arguments + struct AggregateDescription + { + /// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing. + AggregateFunctionPtr function; + IAggregateFunction::AddFunc add_function = nullptr; + std::vector column_numbers; + IColumn * merged_column = nullptr; + AlignedBuffer state; + bool created = false; + + /// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above. + bool is_agg_func_type = false; + + void init(const char * function_name, const DataTypes & argument_types) + { + function = AggregateFunctionFactory::instance().get(function_name, argument_types); + add_function = function->getAddressOfAddFunction(); + state.reset(function->sizeOfData(), function->alignOfData()); + } + + void createState() + { + if (created) + return; + if (is_agg_func_type) + merged_column->insertDefault(); + else + function->create(state.data()); + created = true; + } + + void destroyState() + { + if (!created) + return; + if (!is_agg_func_type) + function->destroy(state.data()); + created = false; + } + + /// Explicitly destroy aggregation state if the stream is terminated + ~AggregateDescription() + { + destroyState(); + } + + AggregateDescription() = default; + AggregateDescription(AggregateDescription &&) = default; + AggregateDescription(const AggregateDescription &) = delete; + }; + + struct SummingMergedData : public MergedData + { + public: + + }; + + /// Stores numbers of key-columns and value-columns. + struct MapDescription + { + std::vector key_col_nums; + std::vector val_col_nums; + }; + +private: + /// Columns with which values should be summed. + ColumnNumbers column_numbers_not_to_aggregate; + + std::vector columns_to_aggregate; + std::vector maps_to_sum; +}; + +} From f3743552cedad4cd75207ed8b10b3d508e68ae59 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Mar 2020 22:58:27 +0300 Subject: [PATCH 070/743] Add SummingSortedTransform [part 2] --- .../Merges/CollapsingSortedTransform.cpp | 2 +- dbms/src/Processors/Merges/MergedData.h | 7 +- .../Merges/MergingSortedTransform.cpp | 2 +- .../Merges/ReplacingSortedTransform.cpp | 2 +- .../Merges/SummingSortedTransform.cpp | 509 ++++++++++-------- .../Merges/SummingSortedTransform.h | 54 +- .../Merges/VersionedCollapsingTransform.cpp | 2 +- 7 files changed, 328 insertions(+), 250 deletions(-) diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp index 13a61d26caa..9fcb7cea116 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp @@ -25,7 +25,7 @@ CollapsingSortedTransform::CollapsingSortedTransform( WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) : IMergingTransform(num_inputs, header, header, true) - , merged_data(header, use_average_block_sizes, max_block_size) + , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , description(std::move(description_)) , sign_column_number(header.getPositionByName(sign_column)) , out_row_sources_buf(out_row_sources_buf_) diff --git a/dbms/src/Processors/Merges/MergedData.h b/dbms/src/Processors/Merges/MergedData.h index e5a8a541aa5..37dd3c62587 100644 --- a/dbms/src/Processors/Merges/MergedData.h +++ b/dbms/src/Processors/Merges/MergedData.h @@ -13,12 +13,9 @@ namespace ErrorCodes class MergedData { public: - explicit MergedData(const Block & header, bool use_average_block_size_, UInt64 max_block_size_) - : max_block_size(max_block_size_), use_average_block_size(use_average_block_size_) + explicit MergedData(MutableColumns columns_, bool use_average_block_size_, UInt64 max_block_size_) + : columns(std::move(columns_)), max_block_size(max_block_size_), use_average_block_size(use_average_block_size_) { - columns.reserve(header.columns()); - for (const auto & column : header) - columns.emplace_back(column.type->createColumn()); } /// Pull will be called at next prepare call. diff --git a/dbms/src/Processors/Merges/MergingSortedTransform.cpp b/dbms/src/Processors/Merges/MergingSortedTransform.cpp index 226b29c36dc..7b7e4fcf62c 100644 --- a/dbms/src/Processors/Merges/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/MergingSortedTransform.cpp @@ -24,7 +24,7 @@ MergingSortedTransform::MergingSortedTransform( bool use_average_block_sizes, bool have_all_inputs_) : IMergingTransform(num_inputs, header, header, have_all_inputs_) - , merged_data(header, use_average_block_sizes, max_block_size) + , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , description(std::move(description_)) , limit(limit_) , quiet(quiet_) diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp index 2de67707d6d..65654a98764 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp @@ -16,7 +16,7 @@ ReplacingSortedTransform::ReplacingSortedTransform( WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) : IMergingTransform(num_inputs, header, header, true) - , merged_data(header, use_average_block_sizes, max_block_size) + , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) , chunk_allocator(num_inputs + max_row_refs) diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index 0c4052e821d..45f3f9b71c9 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -4,261 +4,302 @@ #include #include #include +#include namespace DB { namespace { -bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number) -{ - for (auto & desc : description) - if (desc.column_name == name || (desc.column_name.empty() && desc.column_number == number)) - return true; - - return false; -} - -/// Returns true if merge result is not empty -bool mergeMap(const SummingSortedTransform::MapDescription & desc, Row & row, SortCursor & cursor) -{ - /// Strongly non-optimal. - - Row & left = row; - Row right(left.size()); - - for (size_t col_num : desc.key_col_nums) - right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); - - for (size_t col_num : desc.val_col_nums) - right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); - - auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & + bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number) { - return matrix[i].get()[j]; - }; + for (auto & desc : description) + if (desc.column_name == name || (desc.column_name.empty() && desc.column_number == number)) + return true; - auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array - { - size_t size = col_nums.size(); - Array res(size); - for (size_t col_num_index = 0; col_num_index < size; ++col_num_index) - res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j); - return res; - }; - - std::map merged; - - auto accumulate = [](Array & dst, const Array & src) - { - bool has_non_zero = false; - size_t size = dst.size(); - for (size_t i = 0; i < size; ++i) - if (applyVisitor(FieldVisitorSum(src[i]), dst[i])) - has_non_zero = true; - return has_non_zero; - }; - - auto merge = [&](const Row & matrix) - { - size_t rows = matrix[desc.key_col_nums[0]].get().size(); - - for (size_t j = 0; j < rows; ++j) - { - Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j); - Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j); - - auto it = merged.find(key); - if (merged.end() == it) - merged.emplace(std::move(key), std::move(value)); - else - { - if (!accumulate(it->second, value)) - merged.erase(it); - } - } - }; - - merge(left); - merge(right); - - for (size_t col_num : desc.key_col_nums) - row[col_num] = Array(merged.size()); - for (size_t col_num : desc.val_col_nums) - row[col_num] = Array(merged.size()); - - size_t row_num = 0; - for (const auto & key_value : merged) - { - for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; - - for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; - - ++row_num; + return false; } - return row_num != 0; -} + /// Returns true if merge result is not empty + bool mergeMap(const SummingSortedTransform::MapDescription & desc, Row & row, SortCursor & cursor) + { + /// Strongly non-optimal. + + Row & left = row; + Row right(left.size()); + + for (size_t col_num : desc.key_col_nums) + right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); + + for (size_t col_num : desc.val_col_nums) + right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); + + auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & + { + return matrix[i].get()[j]; + }; + + auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array + { + size_t size = col_nums.size(); + Array res(size); + for (size_t col_num_index = 0; col_num_index < size; ++col_num_index) + res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j); + return res; + }; + + std::map merged; + + auto accumulate = [](Array & dst, const Array & src) + { + bool has_non_zero = false; + size_t size = dst.size(); + for (size_t i = 0; i < size; ++i) + if (applyVisitor(FieldVisitorSum(src[i]), dst[i])) + has_non_zero = true; + return has_non_zero; + }; + + auto merge = [&](const Row & matrix) + { + size_t rows = matrix[desc.key_col_nums[0]].get().size(); + + for (size_t j = 0; j < rows; ++j) + { + Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j); + Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j); + + auto it = merged.find(key); + if (merged.end() == it) + merged.emplace(std::move(key), std::move(value)); + else + { + if (!accumulate(it->second, value)) + merged.erase(it); + } + } + }; + + merge(left); + merge(right); + + for (size_t col_num : desc.key_col_nums) + row[col_num] = Array(merged.size()); + for (size_t col_num : desc.val_col_nums) + row[col_num] = Array(merged.size()); + + size_t row_num = 0; + for (const auto & key_value : merged) + { + for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) + row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; + + for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) + row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; + + ++row_num; + } + + return row_num != 0; + } + + SummingSortedTransform::ColumnsDefinition defineColumns( + const Block & header, + const SortDescription & description, + const Names & column_names_to_sum) + { + size_t num_columns = header.columns(); + SummingSortedTransform::ColumnsDefinition def; + + /// name of nested structure -> the column numbers that refer to it. + std::unordered_map> discovered_maps; + + /** Fill in the column numbers, which must be summed. + * This can only be numeric columns that are not part of the sort key. + * If a non-empty column_names_to_sum is specified, then we only take these columns. + * Some columns from column_names_to_sum may not be found. This is ignored. + */ + for (size_t i = 0; i < num_columns; ++i) + { + const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + + /// Discover nested Maps and find columns for summation + if (typeid_cast(column.type.get())) + { + const auto map_name = Nested::extractTableName(column.name); + /// if nested table name ends with `Map` it is a possible candidate for special handling + if (map_name == column.name || !endsWith(map_name, "Map")) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + discovered_maps[map_name].emplace_back(i); + } + else + { + bool is_agg_func = WhichDataType(column.type).isAggregateFunction(); + + /// There are special const columns for example after prewhere sections. + if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column)) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + /// Are they inside the PK? + if (isInPrimaryKey(description, column.name, i)) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + if (column_names_to_sum.empty() + || column_names_to_sum.end() != + std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name)) + { + // Create aggregator to sum this column + SummingSortedTransform::AggregateDescription desc; + desc.is_agg_func_type = is_agg_func; + desc.column_numbers = {i}; + + if (!is_agg_func) + { + desc.init("sumWithOverflow", {column.type}); + } + + def.columns_to_aggregate.emplace_back(std::move(desc)); + } + else + { + // Column is not going to be summed, use last value + def.column_numbers_not_to_aggregate.push_back(i); + } + } + } + + /// select actual nested Maps from list of candidates + for (const auto & map : discovered_maps) + { + /// map should contain at least two elements (key -> value) + if (map.second.size() < 2) + { + for (auto col : map.second) + def.column_numbers_not_to_aggregate.push_back(col); + continue; + } + + /// no elements of map could be in primary key + auto column_num_it = map.second.begin(); + for (; column_num_it != map.second.end(); ++column_num_it) + if (isInPrimaryKey(description, header.safeGetByPosition(*column_num_it).name, *column_num_it)) + break; + if (column_num_it != map.second.end()) + { + for (auto col : map.second) + def.column_numbers_not_to_aggregate.push_back(col); + continue; + } + + DataTypes argument_types; + SummingSortedTransform::AggregateDescription desc; + SummingSortedTransform::MapDescription map_desc; + + column_num_it = map.second.begin(); + for (; column_num_it != map.second.end(); ++column_num_it) + { + const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it); + const String & name = key_col.name; + const IDataType & nested_type = *assert_cast(*key_col.type).getNestedType(); + + if (column_num_it == map.second.begin() + || endsWith(name, "ID") + || endsWith(name, "Key") + || endsWith(name, "Type")) + { + if (!nested_type.isValueRepresentedByInteger() && !isStringOrFixedString(nested_type)) + break; + + map_desc.key_col_nums.push_back(*column_num_it); + } + else + { + if (!nested_type.isSummable()) + break; + + map_desc.val_col_nums.push_back(*column_num_it); + } + + // Add column to function arguments + desc.column_numbers.push_back(*column_num_it); + argument_types.push_back(key_col.type); + } + + if (column_num_it != map.second.end()) + { + for (auto col : map.second) + def.column_numbers_not_to_aggregate.push_back(col); + continue; + } + + if (map_desc.key_col_nums.size() == 1) + { + // Create summation for all value columns in the map + desc.init("sumMapWithOverflow", argument_types); + def.columns_to_aggregate.emplace_back(std::move(desc)); + } + else + { + // Fall back to legacy mergeMaps for composite keys + for (auto col : map.second) + def.column_numbers_not_to_aggregate.push_back(col); + def.maps_to_sum.emplace_back(std::move(map_desc)); + } + } + } + + MutableColumns getMergedDataColumns( + const Block & header, + const SummingSortedTransform::ColumnsDefinition & columns_definition) + { + MutableColumns columns; + columns.reserve(columns_definition.getNumColumns()); + + for (auto & desc : columns_definition.columns_to_aggregate) + { + // Wrap aggregated columns in a tuple to match function signature + if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) + { + size_t tuple_size = desc.column_numbers.size(); + MutableColumns tuple_columns(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + tuple_columns[i] = header.safeGetByPosition(desc.column_numbers[i]).column->cloneEmpty(); + + columns.emplace_back(ColumnTuple::create(std::move(tuple_columns))); + } + else + columns.emplace_back(header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty()); + } + + for (auto & column_number : columns_definition.column_numbers_not_to_aggregate) + columns.emplace_back(header.safeGetByPosition(column_number).type->createColumn()); + + return columns; + } } SummingSortedTransform::SummingSortedTransform( size_t num_inputs, const Block & header, - SortDescription description, + SortDescription description_, /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. const Names & column_names_to_sum, size_t max_block_size) : IMergingTransform(num_inputs, header, header, true) + , columns_definition(defineColumns(header, description_, column_names_to_sum)) + , merged_data(getMergedDataColumns(header, columns_definition), false, max_block_size) { size_t num_columns = header.columns(); current_row.resize(num_columns); - - /// name of nested structure -> the column numbers that refer to it. - std::unordered_map> discovered_maps; - - /** Fill in the column numbers, which must be summed. - * This can only be numeric columns that are not part of the sort key. - * If a non-empty column_names_to_sum is specified, then we only take these columns. - * Some columns from column_names_to_sum may not be found. This is ignored. - */ - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnWithTypeAndName & column = header.safeGetByPosition(i); - - /// Discover nested Maps and find columns for summation - if (typeid_cast(column.type.get())) - { - const auto map_name = Nested::extractTableName(column.name); - /// if nested table name ends with `Map` it is a possible candidate for special handling - if (map_name == column.name || !endsWith(map_name, "Map")) - { - column_numbers_not_to_aggregate.push_back(i); - continue; - } - - discovered_maps[map_name].emplace_back(i); - } - else - { - bool is_agg_func = WhichDataType(column.type).isAggregateFunction(); - - /// There are special const columns for example after prewere sections. - if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column)) - { - column_numbers_not_to_aggregate.push_back(i); - continue; - } - - /// Are they inside the PK? - if (isInPrimaryKey(description, column.name, i)) - { - column_numbers_not_to_aggregate.push_back(i); - continue; - } - - if (column_names_to_sum.empty() - || column_names_to_sum.end() != - std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name)) - { - // Create aggregator to sum this column - AggregateDescription desc; - desc.is_agg_func_type = is_agg_func; - desc.column_numbers = {i}; - - if (!is_agg_func) - { - desc.init("sumWithOverflow", {column.type}); - } - - columns_to_aggregate.emplace_back(std::move(desc)); - } - else - { - // Column is not going to be summed, use last value - column_numbers_not_to_aggregate.push_back(i); - } - } - } - - /// select actual nested Maps from list of candidates - for (const auto & map : discovered_maps) - { - /// map should contain at least two elements (key -> value) - if (map.second.size() < 2) - { - for (auto col : map.second) - column_numbers_not_to_aggregate.push_back(col); - continue; - } - - /// no elements of map could be in primary key - auto column_num_it = map.second.begin(); - for (; column_num_it != map.second.end(); ++column_num_it) - if (isInPrimaryKey(description, header.safeGetByPosition(*column_num_it).name, *column_num_it)) - break; - if (column_num_it != map.second.end()) - { - for (auto col : map.second) - column_numbers_not_to_aggregate.push_back(col); - continue; - } - - DataTypes argument_types; - AggregateDescription desc; - MapDescription map_desc; - - column_num_it = map.second.begin(); - for (; column_num_it != map.second.end(); ++column_num_it) - { - const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it); - const String & name = key_col.name; - const IDataType & nested_type = *static_cast(key_col.type.get())->getNestedType(); - - if (column_num_it == map.second.begin() - || endsWith(name, "ID") - || endsWith(name, "Key") - || endsWith(name, "Type")) - { - if (!nested_type.isValueRepresentedByInteger() && !isStringOrFixedString(nested_type)) - break; - - map_desc.key_col_nums.push_back(*column_num_it); - } - else - { - if (!nested_type.isSummable()) - break; - - map_desc.val_col_nums.push_back(*column_num_it); - } - - // Add column to function arguments - desc.column_numbers.push_back(*column_num_it); - argument_types.push_back(key_col.type); - } - - if (column_num_it != map.second.end()) - { - for (auto col : map.second) - column_numbers_not_to_aggregate.push_back(col); - continue; - } - - if (map_desc.key_col_nums.size() == 1) - { - // Create summation for all value columns in the map - desc.init("sumMapWithOverflow", argument_types); - columns_to_aggregate.emplace_back(std::move(desc)); - } - else - { - // Fall back to legacy mergeMaps for composite keys - for (auto col : map.second) - column_numbers_not_to_aggregate.push_back(col); - maps_to_sum.emplace_back(std::move(map_desc)); - } - } } } diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.h b/dbms/src/Processors/Merges/SummingSortedTransform.h index f82c1f9e6db..20b49aa7ac8 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.h +++ b/dbms/src/Processors/Merges/SummingSortedTransform.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -20,7 +21,7 @@ public: SummingSortedTransform( size_t num_inputs, const Block & header, - SortDescription description, + SortDescription description_, /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. const Names & column_names_to_sum, size_t max_block_size); @@ -80,7 +81,7 @@ public: struct SummingMergedData : public MergedData { public: - + using MergedData::MergedData; }; /// Stores numbers of key-columns and value-columns. @@ -90,12 +91,51 @@ public: std::vector val_col_nums; }; -private: - /// Columns with which values should be summed. - ColumnNumbers column_numbers_not_to_aggregate; + struct ColumnsDefinition + { + /// Columns with which values should be summed. + ColumnNumbers column_numbers_not_to_aggregate; + /// Columns which should be aggregated. + std::vector columns_to_aggregate; + /// Mapping for nested columns. + std::vector maps_to_sum; - std::vector columns_to_aggregate; - std::vector maps_to_sum; + size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } + }; + +private: + Row current_row; + bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. + + ColumnsDefinition columns_definition; + SummingMergedData merged_data; + + SortDescription description; + + /// Chunks currently being merged. + std::vector source_chunks; + SortCursorImpls cursors; + + /// In merging algorithm, we need to compare current sort key with the last one. + /// So, sorting columns for last row needed to be stored. + /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). + Chunk last_chunk; + ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. + + struct RowRef + { + ColumnRawPtrs * sort_columns = nullptr; /// Point to sort_columns from SortCursor or last_chunk_sort_columns. + UInt64 row_number = 0; + }; + + RowRef last_row; + + SortingHeap queue; + bool is_queue_initialized = false; + + void insertRow(); + void merge(); + void updateCursor(Chunk chunk, size_t source_num); }; } diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp index 9191123c878..4042c146724 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp @@ -14,7 +14,7 @@ VersionedCollapsingTransform::VersionedCollapsingTransform( WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) : IMergingTransform(num_inputs, header, header, true) - , merged_data(header, use_average_block_sizes, max_block_size) + , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 1) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer From 61d6c61757be99add871888341c3ee3cbc0a154a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Mar 2020 23:12:04 +0300 Subject: [PATCH 071/743] Try fix tests. --- dbms/src/Processors/Merges/IMergingTransform.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Processors/Merges/IMergingTransform.cpp b/dbms/src/Processors/Merges/IMergingTransform.cpp index 7c1b58d30f5..2c5a7affd40 100644 --- a/dbms/src/Processors/Merges/IMergingTransform.cpp +++ b/dbms/src/Processors/Merges/IMergingTransform.cpp @@ -177,7 +177,10 @@ IProcessor::Status IMergingTransform::prepare() /// Push if has data. if (has_output_chunk && !is_port_full) + { output.push(std::move(output_chunk)); + has_output_chunk = false; + } if (!is_initialized) return prepareInitializeInputs(); From 04eb205cd7a9f13550e993d7185443ae2699f74e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 31 Mar 2020 23:38:05 +0300 Subject: [PATCH 072/743] implement EXCHANGE TABLES --- dbms/src/Databases/DatabaseAtomic.cpp | 92 +++++++++++++------ dbms/src/Databases/DatabaseAtomic.h | 3 +- dbms/src/Databases/DatabaseLazy.cpp | 5 +- dbms/src/Databases/DatabaseLazy.h | 3 +- dbms/src/Databases/DatabaseOnDisk.cpp | 6 +- dbms/src/Databases/DatabaseOnDisk.h | 3 +- dbms/src/Databases/IDatabase.h | 3 +- .../Interpreters/InterpreterRenameQuery.cpp | 6 +- dbms/src/Parsers/ASTRenameQuery.h | 7 +- dbms/src/Parsers/ParserRenameQuery.cpp | 19 +++- .../01109_exchange_tables.reference | 18 ++++ .../0_stateless/01109_exchange_tables.sql | 49 ++++++++++ 12 files changed, 173 insertions(+), 41 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01109_exchange_tables.reference create mode 100644 dbms/tests/queries/0_stateless/01109_exchange_tables.sql diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index 97f8414ced6..a9da008e419 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -63,6 +63,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) auto table = DatabaseWithDictionaries::detachTableUnlocked(name); table_name_to_path.erase(name); detached_tables.emplace(table->getStorageID().uuid, table); + cleenupDetachedTables(); return table; } @@ -83,54 +84,90 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool } void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name) + const String & to_table_name, bool exchange) { if (typeid(*this) != typeid(to_database)) { if (!typeid_cast(&to_database)) throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); - /// Allow moving tables from Atomic to Ordinary (with table lock) - DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name); + /// Allow moving tables between Atomic and Ordinary (with table lock) + DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange); return; } + auto & other_db = dynamic_cast(to_database); StoragePtr table = tryGetTable(context, table_name); + StoragePtr other_table; + if (exchange) + other_table = other_db.tryGetTable(context, to_table_name); if (!table) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + if (exchange && !other_table) + throw Exception("Table " + backQuote(other_db.getDatabaseName()) + "." + backQuote(to_table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); String old_metadata_path = getObjectMetadataPath(table_name); String new_metadata_path = to_database.getObjectMetadataPath(to_table_name); - if (this == &to_database) + auto detach = [](DatabaseAtomic & db, const String & table_name_) { - std::lock_guard lock(mutex); - renameNoReplace(old_metadata_path, new_metadata_path); - auto table_data_path = table_name_to_path.find(table_name)->second; - tables.erase(table_name); - table_name_to_path.erase(table_name); - table->renameInMemory(to_database.getDatabaseName(), to_table_name); - tables.emplace(to_table_name, table); - table_name_to_path.emplace(to_table_name, table_data_path); + auto table_data_path_ = db.table_name_to_path.find(table_name_)->second; + db.tables.erase(table_name_); + db.table_name_to_path.erase(table_name_); + return table_data_path_; + }; + + auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_) + { + db.tables.emplace(table_name_, table_); + db.table_name_to_path.emplace(table_name_, table_data_path_); + }; + + String table_data_path; + String other_table_data_path; + + bool inside_database = this == &other_db; + if (inside_database && table_name == to_table_name) + return; + + std::unique_lock db_lock; + std::unique_lock other_db_lock; + if (inside_database) + db_lock = std::unique_lock{mutex}; + else if (this < &other_db) + { + db_lock = std::unique_lock{mutex}; + other_db_lock = std::unique_lock{other_db.mutex}; } else { - String table_data_path; - { - std::lock_guard lock(mutex); - renameNoReplace(old_metadata_path, new_metadata_path); - table_data_path = table_name_to_path.find(table_name)->second; - tables.erase(table_name); - table_name_to_path.erase(table_name); - DatabaseCatalog::instance().updateUUIDMapping(table->getStorageID().uuid, to_database.shared_from_this(), table); - } - table->renameInMemory(to_database.getDatabaseName(), to_table_name); - auto & to_atomic_db = dynamic_cast(to_database); - - std::lock_guard lock(to_atomic_db.mutex); - to_atomic_db.tables.emplace(to_table_name, table); - to_atomic_db.table_name_to_path.emplace(to_table_name, table_data_path); + other_db_lock = std::unique_lock{other_db.mutex}; + db_lock = std::unique_lock{mutex}; } + + if (exchange) + renameExchange(old_metadata_path, new_metadata_path); + else + renameNoReplace(old_metadata_path, new_metadata_path); + + table_data_path = detach(*this, table_name); + if (exchange) + other_table_data_path = detach(other_db, to_table_name); + + table->renameInMemory(other_db.getDatabaseName(), to_table_name); + if (exchange) + other_table->renameInMemory(getDatabaseName(), table_name); + + if (!inside_database) + { + DatabaseCatalog::instance().updateUUIDMapping(table->getStorageID().uuid, other_db.shared_from_this(), table); + if (exchange) + DatabaseCatalog::instance().updateUUIDMapping(other_table->getStorageID().uuid, shared_from_this(), other_table); + } + + attach(other_db, to_table_name, table_data_path, table); + if (exchange) + attach(*this, table_name, other_table_data_path, other_table); } void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) @@ -140,7 +177,6 @@ void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore void DatabaseAtomic::shutdown() { - DatabaseWithDictionaries::shutdown(); } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index f2d93167cab..f02f4438f5d 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -20,7 +20,8 @@ public: const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name) override; + const String & to_table_name, + bool exchange) override; void dropTable(const Context & context, const String & table_name, bool no_delay) override; diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 026cf3345da..683d315d012 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -79,10 +79,11 @@ void DatabaseLazy::renameTable( const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name) + const String & to_table_name, + bool exchange) { SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name); + DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange); } diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index ff09d5a5668..99bdfea11ae 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -41,7 +41,8 @@ public: const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name) override; + const String & to_table_name, + bool exchange) override; void alterTable( const Context & context, diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 2d3009bd247..2d13d6e13ab 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -242,8 +242,12 @@ void DatabaseOnDisk::renameTable( const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name) + const String & to_table_name, + bool exchange) { + if (exchange) + throw Exception("Tables can be exchanged only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED); + bool from_ordinary_to_atomic = false; bool from_atomic_to_ordinary = false; if (typeid(*this) != typeid(to_database)) diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index c2d3f5392d2..b7fa0b1b015 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -47,7 +47,8 @@ public: const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name) override; + const String & to_table_name, + bool exchange) override; ASTPtr getCreateDatabaseQuery(const Context & context) const override; diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 5bb3e3259ba..44ade7542bf 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -216,7 +216,8 @@ public: const Context & /*context*/, const String & /*name*/, IDatabase & /*to_database*/, - const String & /*to_name*/) + const String & /*to_name*/, + bool /*exchange*/) { throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index 9995eebcc98..f7cbd2a9678 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -92,13 +92,15 @@ BlockIO InterpreterRenameQuery::execute() for (auto & elem : descriptions) { - database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name)); + if (!rename.exchange) + database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name)); database_catalog.getDatabase(elem.from_database_name)->renameTable( context, elem.from_table_name, *database_catalog.getDatabase(elem.to_database_name), - elem.to_table_name); + elem.to_table_name, + rename.exchange); } return {}; diff --git a/dbms/src/Parsers/ASTRenameQuery.h b/dbms/src/Parsers/ASTRenameQuery.h index 4cf007d3b36..9f91faffa50 100644 --- a/dbms/src/Parsers/ASTRenameQuery.h +++ b/dbms/src/Parsers/ASTRenameQuery.h @@ -29,6 +29,8 @@ public: using Elements = std::vector; Elements elements; + bool exchange{false}; /// For EXCHANGE TABLES + /** Get the text that identifies this element. */ String getID(char) const override { return "Rename"; } @@ -59,7 +61,8 @@ public: protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME TABLE " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") + << (exchange ? "EXCHANGE TABLES " : "RENAME TABLE ") << (settings.hilite ? hilite_none : ""); for (auto it = elements.cbegin(); it != elements.cend(); ++it) { @@ -67,7 +70,7 @@ protected: settings.ostr << ", "; settings.ostr << (!it->from.database.empty() ? backQuoteIfNeed(it->from.database) + "." : "") << backQuoteIfNeed(it->from.table) - << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "") + << (settings.hilite ? hilite_keyword : "") << (exchange ? " AND " : " TO ") << (settings.hilite ? hilite_none : "") << (!it->to.database.empty() ? backQuoteIfNeed(it->to.database) + "." : "") << backQuoteIfNeed(it->to.table); } diff --git a/dbms/src/Parsers/ParserRenameQuery.cpp b/dbms/src/Parsers/ParserRenameQuery.cpp index 9323ca16ee4..abb08b503cd 100644 --- a/dbms/src/Parsers/ParserRenameQuery.cpp +++ b/dbms/src/Parsers/ParserRenameQuery.cpp @@ -41,13 +41,27 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_rename_table("RENAME TABLE"); ParserKeyword s_to("TO"); + ParserKeyword s_exchange_tables("EXCHANGE TABLES"); + ParserKeyword s_and("AND"); ParserToken s_comma(TokenType::Comma); + bool exchange = false; + if (!s_rename_table.ignore(pos, expected)) - return false; + { + if (s_exchange_tables.ignore(pos, expected)) + exchange = true; + else + return false; + } ASTRenameQuery::Elements elements; + auto ignore_delim = [&]() + { + return exchange ? s_and.ignore(pos) : s_to.ignore(pos); + }; + while (true) { if (!elements.empty() && !s_comma.ignore(pos)) @@ -56,7 +70,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) elements.push_back(ASTRenameQuery::Element()); if (!parseDatabaseAndTable(elements.back().from, pos, expected) - || !s_to.ignore(pos) + || !ignore_delim() || !parseDatabaseAndTable(elements.back().to, pos, expected)) return false; } @@ -73,6 +87,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) node = query; query->elements = elements; + query->exchange = exchange; return true; } diff --git a/dbms/tests/queries/0_stateless/01109_exchange_tables.reference b/dbms/tests/queries/0_stateless/01109_exchange_tables.reference new file mode 100644 index 00000000000..9c2f4af4d3d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01109_exchange_tables.reference @@ -0,0 +1,18 @@ +0 exchange +1 tables +2 hello +3 world +2 hello +3 world +0 exchange +1 tables +0 exchange +1 tables +2 hello +3 world +4 another +5 db +2 hello +3 world +0 exchange +1 tables diff --git a/dbms/tests/queries/0_stateless/01109_exchange_tables.sql b/dbms/tests/queries/0_stateless/01109_exchange_tables.sql new file mode 100644 index 00000000000..5114aa7e347 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01109_exchange_tables.sql @@ -0,0 +1,49 @@ +DROP DATABASE IF EXISTS test_01109; +CREATE DATABASE test_01109 ENGINE=Atomic; + +USE test_01109; + +CREATE TABLE t1 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables']))); +CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t1), * FROM (SELECT arrayJoin(['hello', 'world'])); + +EXCHANGE TABLES t1 AND t3; -- { serverError 60 } +EXCHANGE TABLES t4 AND t2; -- { serverError 60 } +SELECT * FROM t1; +SELECT * FROM t2; + +EXCHANGE TABLES t1 AND t2; +SELECT * FROM t1; +SELECT * FROM t2; + +RENAME TABLE t1 TO t1tmp, t2 TO t2tmp; +RENAME TABLE t1tmp TO t2, t2tmp TO t1; +SELECT * FROM t1; +SELECT * FROM t2; + +DROP DATABASE IF EXISTS test_01109_other_atomic; +DROP DATABASE IF EXISTS test_01109_ordinary; +SET default_database_engine='Atomic'; +CREATE DATABASE test_01109_other_atomic; +SET default_database_engine='Ordinary'; +CREATE DATABASE test_01109_ordinary; + +CREATE TABLE test_01109_other_atomic.t3 ENGINE=MergeTree() ORDER BY tuple() + AS SELECT rowNumberInAllBlocks() + (SELECT max((*,*).1.1) + 1 FROM (SELECT (*,) FROM t1 UNION ALL SELECT (*,) FROM t2)), * + FROM (SELECT arrayJoin(['another', 'db'])); + +CREATE TABLE test_01109_ordinary.t4 AS t1; + +EXCHANGE TABLES test_01109_other_atomic.t3 AND test_01109_ordinary.t4; -- { serverError 48 } +EXCHANGE TABLES test_01109_ordinary.t4 AND test_01109_other_atomic.t3; -- { serverError 48 } +EXCHANGE TABLES test_01109_ordinary.t4 AND test_01109_ordinary.t4; -- { serverError 48 } + +EXCHANGE TABLES t1 AND test_01109_other_atomic.t3; +EXCHANGE TABLES t2 AND t2; +SELECT * FROM t1; +SELECT * FROM t2; +SELECT * FROM test_01109_other_atomic.t3; +SELECT * FROM test_01109_ordinary.t4; + +DROP DATABASE test_01109; +DROP DATABASE test_01109_other_atomic; +DROP DATABASE test_01109_ordinary; From 2ecbf0b0bb2431593f05853ca6aa10ce74518f82 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Apr 2020 14:45:02 +0300 Subject: [PATCH 073/743] Add SummingSortedTransform [part 3] --- .../Merges/SummingSortedTransform.cpp | 184 ++++++++++++++++++ .../Merges/SummingSortedTransform.h | 35 +++- 2 files changed, 217 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index 45f3f9b71c9..7ae127c2314 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -286,6 +286,43 @@ namespace return columns; } + + void finalizeChunk( + Chunk & chunk, size_t num_result_columns, + const SummingSortedTransform::ColumnsDefinition & columns_definition) + { + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + Columns res_columns(num_result_columns); + size_t next_column = 0; + + for (auto & desc : columns_definition.columns_to_aggregate) + { + auto column = std::move(columns[next_column]); + ++next_column; + + if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) + { + /// Unpack tuple into block. + size_t tuple_size = desc.column_numbers.size(); + for (size_t i = 0; i < tuple_size; ++i) + res_columns[desc.column_numbers[i]] = assert_cast(*column).getColumnPtr(i); + } + else + res_columns[desc.column_numbers[0]] = std::move(column); + } + + for (auto column_number : columns_definition.column_numbers_not_to_aggregate) + { + auto column = std::move(columns[next_column]); + ++next_column; + + res_columns[column_number] = std::move(column); + } + + chunk.setColumns(std::move(res_columns), num_rows); + } } SummingSortedTransform::SummingSortedTransform( @@ -302,4 +339,151 @@ SummingSortedTransform::SummingSortedTransform( current_row.resize(num_columns); } +void SummingSortedTransform::initializeInputs() +{ + queue = SortingHeap(cursors); + is_queue_initialized = true; +} + +void SummingSortedTransform::consume(Chunk chunk, size_t input_number) +{ + updateCursor(std::move(chunk), input_number); + + if (is_queue_initialized) + queue.push(cursors[input_number]); +} + +void SummingSortedTransform::updateCursor(Chunk chunk, size_t source_num) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); + + auto & source_chunk = source_chunks[source_num]; + + if (source_chunk) + { + /// Extend lifetime of last chunk. + last_chunk = std::move(source_chunk); + last_chunk_sort_columns = std::move(cursors[source_num].all_columns); + + source_chunk = std::move(chunk); + cursors[source_num].reset(source_chunk.getColumns(), {}); + } + else + { + if (cursors[source_num].has_collation) + throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); + + source_chunk = std::move(chunk); + cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num); + } +} + +void SummingSortedTransform::work() +{ + merge(); + prepareOutputChunk(merged_data); + + if (has_output_chunk) + finalizeChunk(output_chunk, getOutputs().back().getHeader().columns(), columns_definition); +} + +void SummingSortedTransform::merge() +{ + /// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size` + while (queue.isValid()) + { + bool key_differs; + bool has_previous_group = !last_key.empty(); + + SortCursor current = queue.current(); + + { + RowRef current_key; + current_key.set(current); + + if (!has_previous_group) /// The first key encountered. + { + key_differs = true; + current_row_is_zero = true; + } + else + key_differs = !last_key.hasEqualSortColumnsWith(current_key); + + last_key = current_key; + last_chunk_sort_columns.clear(); + } + + if (key_differs) + { + if (has_previous_group) + /// Write the data for the previous group. + insertCurrentRowIfNeeded(); + + if (merged_data.hasEnoughRows()) + { + /// The block is now full and the last row is calculated completely. + last_key.reset(); + return; + } + + setRow(current_row, current); + + /// Reset aggregation states for next row + for (auto & desc : columns_definition.columns_to_aggregate) + desc.createState(); + + // Start aggregations with current row + addRow(current); + + if (columns_definition.maps_to_sum.empty()) + { + /// We have only columns_to_aggregate. The status of current row will be determined + /// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions. + current_row_is_zero = true; // NOLINT + } + else + { + /// We have complex maps that will be summed with 'mergeMap' method. + /// The single row is considered non zero, and the status after merging with other rows + /// will be determined in the branch below (when key_differs == false). + current_row_is_zero = false; // NOLINT + } + } + else + { + addRow(current); + + // Merge maps only for same rows + for (const auto & desc : columns_definition.maps_to_sum) + if (mergeMap(desc, current_row, current)) + current_row_is_zero = false; + } + + if (!current->isLast()) + { + queue.next(); + } + else + { + /// We get the next block from the corresponding source, if there is one. + queue.removeTop(); + requestDataForInput(current.impl->order); + return; + } + } + + /// We will write the data for the last group, if it is non-zero. + /// If it is zero, and without it the output stream will be empty, we will write it anyway. + insertCurrentRowIfNeeded(); + last_chunk_sort_columns.clear(); + is_finished = true; +} + + + } diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.h b/dbms/src/Processors/Merges/SummingSortedTransform.h index 20b49aa7ac8..662cc65d95d 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.h +++ b/dbms/src/Processors/Merges/SummingSortedTransform.h @@ -103,6 +103,13 @@ public: size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } }; + String getName() const override { return "SummingSortedTransform"; } + void work() override; + +protected: + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + private: Row current_row; bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. @@ -125,10 +132,34 @@ private: struct RowRef { ColumnRawPtrs * sort_columns = nullptr; /// Point to sort_columns from SortCursor or last_chunk_sort_columns. - UInt64 row_number = 0; + UInt64 row_num = 0; + + bool empty() const { return sort_columns == nullptr; } + void reset() { sort_columns = nullptr; } + + void set(SortCursor & cursor) + { + sort_columns = &cursor.impl->sort_columns; + row_num = cursor.impl->pos; + } + + bool hasEqualSortColumnsWith(const RowRef & other) + { + auto size = sort_columns->size(); + for (size_t col_number = 0; col_number < size; ++col_number) + { + auto & cur_column = (*sort_columns)[col_number]; + auto & other_column = (*other.sort_columns)[col_number]; + + if (0 != cur_column->compareAt(row_num, other.row_num, *other_column, 1)) + return false; + } + + return true; + } }; - RowRef last_row; + RowRef last_key; SortingHeap queue; bool is_queue_initialized = false; From 8e18f56d4f5eb233db7225623fd7d4b86d66797b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Apr 2020 14:53:09 +0300 Subject: [PATCH 074/743] Try fix perftest distributed_aggregation.xml --- dbms/tests/performance/distributed_aggregation.xml | 12 ------------ .../config.d/perf-comparison-tweaks-config.xml | 2 ++ 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/dbms/tests/performance/distributed_aggregation.xml b/dbms/tests/performance/distributed_aggregation.xml index 6fdd9fda7c5..25ba4169b9e 100644 --- a/dbms/tests/performance/distributed_aggregation.xml +++ b/dbms/tests/performance/distributed_aggregation.xml @@ -1,18 +1,6 @@ - - - 10 - 1000 - - - 50 - 60000 - - - select count() from (select sipHash64(zero) from zeros_mt(100000000) union all select sipHash64(zero) from zeros_mt(100000000)) select count(sipHash64(zero)) from remote('127.0.0.{{1,1}}', zeros_mt(100000000)) select count(sipHash64(zero)) from remote('127.0.0.{{1,2}}', zeros_mt(100000000)) select count(sipHash64(zero)) from remote('127.0.0.{{2,3}}', zeros_mt(100000000)) - diff --git a/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml index 863a40718d9..a6d59fc3b4c 100644 --- a/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml @@ -1,4 +1,6 @@ + :: + true From 5b22a9596d6698ed8f794f25fa6604c2c6a99558 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 1 Apr 2020 15:16:39 +0300 Subject: [PATCH 075/743] fixup --- dbms/src/Columns/IColumn.cpp | 5 ++ dbms/src/Columns/IColumn.h | 1 + dbms/src/Core/Block.cpp | 16 +++--- dbms/src/Core/Block.h | 17 +++++- dbms/src/Interpreters/ActionsVisitor.cpp | 60 +++++++++++++++----- dbms/src/Interpreters/ActionsVisitor.h | 11 +++- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 30 +++++++++- 7 files changed, 115 insertions(+), 25 deletions(-) diff --git a/dbms/src/Columns/IColumn.cpp b/dbms/src/Columns/IColumn.cpp index a3ed0885651..df56823b6aa 100644 --- a/dbms/src/Columns/IColumn.cpp +++ b/dbms/src/Columns/IColumn.cpp @@ -9,6 +9,11 @@ namespace DB { +Field IColumn::get(size_t n) const +{ + return (*this)[n]; +} + String IColumn::dumpStructure() const { WriteBufferFromOwnString res; diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 090537d6770..aa9455fc6cc 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -70,6 +70,7 @@ public: /// Returns value of n-th element in universal Field representation. /// Is used in rare cases, since creation of Field instance is expensive usually. virtual Field operator[](size_t n) const = 0; + Field get(size_t n) const; /// Like the previous one, but avoids extra copying if Field is in a container, for example. virtual void get(size_t n, Field & res) const = 0; diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index a18d34af994..bc29a74f4eb 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -181,25 +181,25 @@ const ColumnWithTypeAndName & Block::safeGetByPosition(size_t position) const } -ColumnWithTypeAndName & Block::getByName(const std::string & name) +const ColumnWithTypeAndName * Block::findByName(const std::string & name) const { auto it = index_by_name.find(name); if (index_by_name.end() == it) - throw Exception("Not found column " + name + " in block. There are only columns: " + dumpNames() - , ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); - - return data[it->second]; + { + return nullptr; + } + return &data[it->second]; } const ColumnWithTypeAndName & Block::getByName(const std::string & name) const { - auto it = index_by_name.find(name); - if (index_by_name.end() == it) + auto * result = findByName(name); + if (!result) throw Exception("Not found column " + name + " in block. There are only columns: " + dumpNames() , ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); - return data[it->second]; + return *result; } diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 82b60c83efb..ce804ddc0b5 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -28,7 +28,7 @@ class Block { private: using Container = ColumnsWithTypeAndName; - using IndexByName = std::map; + using IndexByName = std::unordered_map; Container data; IndexByName index_by_name; @@ -64,7 +64,20 @@ public: ColumnWithTypeAndName & safeGetByPosition(size_t position); const ColumnWithTypeAndName & safeGetByPosition(size_t position) const; - ColumnWithTypeAndName & getByName(const std::string & name); + ColumnWithTypeAndName* findByName(const std::string & name) + { + return const_cast( + const_cast(this)->findByName(name)); + } + + const ColumnWithTypeAndName* findByName(const std::string & name) const; + + ColumnWithTypeAndName & getByName(const std::string & name) + { + return const_cast( + const_cast(this)->getByName(name)); + } + const ColumnWithTypeAndName & getByName(const std::string & name) const; Container::iterator begin() { return data.begin(); } diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index eb92ee76973..94c6cc000f9 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -195,17 +195,17 @@ SetPtr makeExplicitSet( return set; } -static String getUniqueName(const Block & block, const String & prefix) +static String getUniqueName(ActionsVisitor::Data & data, const String & prefix) { + auto & block = data.getSampleBlock(); auto result = prefix; if (block.has(result)) { - int i = 1; do { - result = prefix + "_" + toString(i); - ++i; + result = prefix + "_" + toString(data.next_unique_suffix); + ++data.next_unique_suffix; } while (block.has(result)); } @@ -468,7 +468,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). if (!prepared_set->empty()) - column.name = getUniqueName(data.getSampleBlock(), "__set"); + column.name = getUniqueName(data, "__set"); else column.name = child->getColumnName(); @@ -496,7 +496,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & ColumnWithTypeAndName column( ColumnConst::create(std::move(column_string), 1), std::make_shared(), - getUniqueName(data.getSampleBlock(), "__joinGet")); + getUniqueName(data, "__joinGet")); data.addAction(ExpressionAction::addColumn(column)); argument_types.push_back(column.type); argument_names.push_back(column.name); @@ -577,7 +577,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// We can not name `getColumnName()`, /// because it does not uniquely define the expression (the types of arguments can be different). - String lambda_name = getUniqueName(data.getSampleBlock(), "__lambda"); + String lambda_name = getUniqueName(data, "__lambda"); auto function_capture = std::make_unique( lambda_actions, captured, lambda_arguments, result_type, result_name); @@ -612,16 +612,50 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, Data & data) { DataTypePtr type = applyVisitor(FieldToDataType(), literal.value); + const auto value = convertFieldToType(literal.value, *type); + + // FIXME why do we have a second pass with a clean sample block over the same + // AST here? Anyway, do not modify the column name if it is set already. + if (literal.unique_column_name.empty()) + { + const auto default_name = literal.getColumnName(); + auto & block = data.getSampleBlock(); + auto * existing_column = block.findByName(default_name); + + /* + * To approximate CSE, build all identical literals to a single temporary + * columns. We try to find the column by its default name, but after that + * we have to check that it contains the correct data. This might not be + * the case if it is a user-supplied column, or it is from under a join, + * etc. + * Overall, this is a hack around a generally poor name-based notion of + * column identity we currently use. + */ + if (existing_column + && existing_column->column + && isColumnConst(*existing_column->column) + && existing_column->column->size() == 1 + && existing_column->column->get(0) == value) + { + const_cast(literal).unique_column_name = default_name; + } + else + { + const_cast(literal).unique_column_name + = getUniqueName(data, default_name); + } + } + + if (data.hasColumn(literal.unique_column_name)) + { + return; + } ColumnWithTypeAndName column; - column.column = type->createColumnConst(1, convertFieldToType(literal.value, *type)); + column.name = literal.unique_column_name; + column.column = type->createColumnConst(1, value); column.type = type; - // Always create columns for literals with a unique name. Otherwise, there - // may be some weird clashes, see 01101_literal_column_clash. - column.name = getUniqueName(data.getSampleBlock(), literal.getColumnName()); - const_cast(literal).unique_column_name = column.name; - data.addAction(ExpressionAction::addColumn(column)); } diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index f6db551ff33..e67e181e009 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -42,6 +42,7 @@ struct ScopeStack const Context & context; +public: ScopeStack(const ExpressionActionsPtr & actions, const Context & context_); void pushLevel(const NamesAndTypesList & input_columns); @@ -80,6 +81,13 @@ public: size_t visit_depth; ScopeStack actions_stack; + /* + * Remember the last unique column suffix to avoid quadratic behavior + * when we add lots of column with same prefix. One counter for all + * prefixes is good enough. + */ + int next_unique_suffix; + Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, @@ -95,7 +103,8 @@ public: only_consts(only_consts_), no_storage_or_local(no_storage_or_local_), visit_depth(0), - actions_stack(actions, context) + actions_stack(actions, context), + next_unique_suffix(actions_stack.getSampleBlock().columns() + 1) {} void updateActions(ExpressionActionsPtr & actions) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index cad4b3bd188..dc362542df9 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -855,7 +855,35 @@ void SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & String result_name = ast->getAliasOrColumnName(); if (required_result_columns.empty() || required_result_columns.count(result_name)) { - result_columns.emplace_back(ast->getColumnName(), result_name); + std::string source_name = ast->getColumnName(); + + /* + * For temporary columns created by ExpressionAnalyzer for literals, + * use the correct source column. Using the default display name + * returned by getColumnName is not enough, and we have to use the + * column id set by EA. In principle, this logic applies to all kinds + * of columns, not only literals. Literals are especially problematic + * for two reasons: + * 1) confusing different literal columns leads to weird side + * effects (see 01101_literal_columns_clash); + * 2) the disambiguation mechanism in SyntaxAnalyzer, that, among + * other things, creates unique aliases for columns with same + * names from different tables, is applied before these temporary + * columns are created by ExpressionAnalyzer. + * Similar problems should also manifest for function columns, which + * are likewise created at a later stage by EA. + * In general, we need to have explicit separation between display + * names and identifiers for columns. This code is a workaround for + * a particular subclass of problems, and not a proper solution. + */ + if (auto as_literal = dynamic_cast(ast.get()); + as_literal) + { + source_name = as_literal->unique_column_name; + assert(!source_name.empty()); + } + + result_columns.emplace_back(source_name, result_name); step.required_output.push_back(result_columns.back().second); } } From ed5c6bff5eca46c8a0050b089759d80d814021d0 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 1 Apr 2020 15:21:13 +0300 Subject: [PATCH 076/743] fixpu --- dbms/src/Interpreters/ActionsVisitor.cpp | 2 +- .../queries/0_stateless/01101_literal_column_clash.reference | 2 ++ dbms/tests/queries/0_stateless/01101_literal_column_clash.sql | 3 +++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 94c6cc000f9..30f3fc754b4 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -623,7 +623,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, auto * existing_column = block.findByName(default_name); /* - * To approximate CSE, build all identical literals to a single temporary + * To approximate CSE, bind all identical literals to a single temporary * columns. We try to find the column by its default name, but after that * we have to check that it contains the correct data. This might not be * the case if it is a user-supplied column, or it is from under a join, diff --git a/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference b/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference index 0dc94464bfc..5cbe1f1eea3 100644 --- a/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference +++ b/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference @@ -1,3 +1,5 @@ 1 7 3 xyzabc 2 +1 3 +1 2 0 0 diff --git a/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql b/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql index 11e3b622277..3a2a71f126a 100644 --- a/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql +++ b/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql @@ -6,5 +6,8 @@ join (select '1' as sid) as t2 on t2.sid = cast(t1.iid as String); -- even simpler cases select cast(7 as String), * from (select 3 "'String'"); SELECT concat('xyz', 'abc'), * FROM (SELECT 2 AS "'xyz'"); +with 3 as "1" select 1, "1"; +-- https://github.com/ClickHouse/ClickHouse/issues/9953 +select 1, * from (select 2 x) a left join (select 1, 3 y) b on y = x; From f4e4aeda7fb7f47586ee53cad9fb3dae0b0969cf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Apr 2020 17:08:48 +0300 Subject: [PATCH 077/743] Add SummingSortedTransform [part 4] --- .../Merges/CollapsingSortedTransform.cpp | 2 + .../Merges/ReplacingSortedTransform.cpp | 2 + .../Merges/SummingSortedTransform.cpp | 171 ++++++++++++++++-- .../Merges/SummingSortedTransform.h | 141 +++++++++------ .../Merges/VersionedCollapsingTransform.cpp | 2 + 5 files changed, 241 insertions(+), 77 deletions(-) diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp index 9fcb7cea116..009aed0983f 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.cpp @@ -29,6 +29,8 @@ CollapsingSortedTransform::CollapsingSortedTransform( , description(std::move(description_)) , sign_column_number(header.getPositionByName(sign_column)) , out_row_sources_buf(out_row_sources_buf_) + , source_chunks(num_inputs) + , cursors(num_inputs) , chunk_allocator(num_inputs + max_row_refs) { } diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp index 65654a98764..e39b33a5a46 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp @@ -19,6 +19,8 @@ ReplacingSortedTransform::ReplacingSortedTransform( , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) + , source_chunks(num_inputs) + , cursors(num_inputs) , chunk_allocator(num_inputs + max_row_refs) { if (!version_column.empty()) diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index 7ae127c2314..87bf533d5c5 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -1,14 +1,23 @@ #include + #include #include -#include -#include -#include +#include #include +#include +#include +#include +#include namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int CORRUPTED_DATA; +} + namespace { bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number) @@ -159,7 +168,7 @@ namespace std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name)) { // Create aggregator to sum this column - SummingSortedTransform::AggregateDescription desc; + detail::AggregateDescription desc; desc.is_agg_func_type = is_agg_func; desc.column_numbers = {i}; @@ -202,7 +211,7 @@ namespace } DataTypes argument_types; - SummingSortedTransform::AggregateDescription desc; + detail::AggregateDescription desc; SummingSortedTransform::MapDescription map_desc; column_num_it = map.second.begin(); @@ -323,20 +332,52 @@ namespace chunk.setColumns(std::move(res_columns), num_rows); } + + void setRow(Row & row, SortCursor & cursor, const Block & header) + { + size_t num_columns = row.size(); + for (size_t i = 0; i < num_columns; ++i) + { + try + { + cursor->all_columns[i]->get(cursor->pos, row[i]); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + + /// Find out the name of the column and throw more informative exception. + + String column_name; + if (i < header.columns()) + { + column_name = header.safeGetByPosition(i).name; + break; + } + + throw Exception("MergingSortedBlockInputStream failed to read row " + toString(cursor->pos) + + " of column " + toString(i) + (column_name.empty() ? "" : " (" + column_name + ")"), + ErrorCodes::CORRUPTED_DATA); + } + } + } } SummingSortedTransform::SummingSortedTransform( - size_t num_inputs, const Block & header, - SortDescription description_, - /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. - const Names & column_names_to_sum, - size_t max_block_size) - : IMergingTransform(num_inputs, header, header, true) - , columns_definition(defineColumns(header, description_, column_names_to_sum)) - , merged_data(getMergedDataColumns(header, columns_definition), false, max_block_size) + size_t num_inputs, const Block & header, + SortDescription description_, + /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. + const Names & column_names_to_sum, + size_t max_block_size) + : IMergingTransform(num_inputs, header, header, true) + , columns_definition(defineColumns(header, description_, column_names_to_sum)) + , merged_data(getMergedDataColumns(header, columns_definition), false, max_block_size) + , description(std::move(description_)) + , source_chunks(num_inputs) + , cursors(num_inputs) { - size_t num_columns = header.columns(); - current_row.resize(num_columns); + current_row.resize(header.columns()); + merged_data.initAggregateDescription(columns_definition.columns_to_aggregate); } void SummingSortedTransform::initializeInputs() @@ -389,7 +430,103 @@ void SummingSortedTransform::work() prepareOutputChunk(merged_data); if (has_output_chunk) + { finalizeChunk(output_chunk, getOutputs().back().getHeader().columns(), columns_definition); + merged_data.initAggregateDescription(columns_definition.columns_to_aggregate); + } +} + +void SummingSortedTransform::insertCurrentRowIfNeeded() +{ + /// We have nothing to aggregate. It means that it could be non-zero, because we have columns_not_to_aggregate. + if (columns_definition.columns_to_aggregate.empty()) + current_row_is_zero = false; + + for (auto & desc : columns_definition.columns_to_aggregate) + { + // Do not insert if the aggregation state hasn't been created + if (desc.created) + { + if (desc.is_agg_func_type) + { + current_row_is_zero = false; + } + else + { + try + { + desc.function->insertResultInto(desc.state.data(), *desc.merged_column); + + /// Update zero status of current row + if (desc.column_numbers.size() == 1) + { + // Flag row as non-empty if at least one column number if non-zero + current_row_is_zero = current_row_is_zero && desc.merged_column->isDefaultAt(desc.merged_column->size() - 1); + } + else + { + /// It is sumMapWithOverflow aggregate function. + /// Assume that the row isn't empty in this case (just because it is compatible with previous version) + current_row_is_zero = false; + } + } + catch (...) + { + desc.destroyState(); + throw; + } + } + desc.destroyState(); + } + else + desc.merged_column->insertDefault(); + } + + /// If it is "zero" row, then rollback the insertion + /// (at this moment we need rollback only cols from columns_to_aggregate) + if (current_row_is_zero) + { + for (auto & desc : columns_definition.columns_to_aggregate) + desc.merged_column->popBack(1); + + return; + } + + merged_data.insertRow(current_row, columns_definition.column_numbers_not_to_aggregate); +} + +void SummingSortedTransform::addRow(SortCursor & cursor) +{ + for (auto & desc : columns_definition.columns_to_aggregate) + { + if (!desc.created) + throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR); + + if (desc.is_agg_func_type) + { + // desc.state is not used for AggregateFunction types + auto & col = cursor->all_columns[desc.column_numbers[0]]; + assert_cast(*desc.merged_column).insertMergeFrom(*col, cursor->pos); + } + else + { + // Specialized case for unary functions + if (desc.column_numbers.size() == 1) + { + auto & col = cursor->all_columns[desc.column_numbers[0]]; + desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr); + } + else + { + // Gather all source columns into a vector + ColumnRawPtrs columns(desc.column_numbers.size()); + for (size_t i = 0; i < desc.column_numbers.size(); ++i) + columns[i] = cursor->all_columns[desc.column_numbers[i]]; + + desc.add_function(desc.function.get(), desc.state.data(), columns.data(), cursor->pos, nullptr); + } + } + } } void SummingSortedTransform::merge() @@ -403,7 +540,7 @@ void SummingSortedTransform::merge() SortCursor current = queue.current(); { - RowRef current_key; + detail::RowRef current_key; current_key.set(current); if (!has_previous_group) /// The first key encountered. @@ -431,7 +568,7 @@ void SummingSortedTransform::merge() return; } - setRow(current_row, current); + setRow(current_row, current, getInputs().front().getHeader()); /// Reset aggregation states for next row for (auto & desc : columns_definition.columns_to_aggregate) diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.h b/dbms/src/Processors/Merges/SummingSortedTransform.h index 662cc65d95d..678ff6587a7 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.h +++ b/dbms/src/Processors/Merges/SummingSortedTransform.h @@ -13,20 +13,9 @@ namespace DB { - - -class SummingSortedTransform : public IMergingTransform +namespace detail { -public: - - SummingSortedTransform( - size_t num_inputs, const Block & header, - SortDescription description_, - /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. - const Names & column_names_to_sum, - size_t max_block_size); - - /// Stores aggregation function, state, and columns to be used as function arguments + /// Stores aggregation function, state, and columns to be used as function arguments. struct AggregateDescription { /// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing. @@ -78,57 +67,31 @@ public: AggregateDescription(const AggregateDescription &) = delete; }; + /// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns. struct SummingMergedData : public MergedData { public: using MergedData::MergedData; + + void insertRow(const Row & row, const ColumnNumbers & column_numbers) + { + for (auto column_number :column_numbers) + columns[column_number]->insert(row[column_number]); + + ++total_merged_rows; + ++merged_rows; + /// TODO: sum_blocks_granularity += block_size; + } + + /// Initialize aggregate descriptions with columns. + void initAggregateDescription(std::vector & columns_to_aggregate) + { + size_t num_columns = columns_to_aggregate.size(); + for (size_t column_number = 0; column_number < num_columns; ++column_number) + columns_to_aggregate[column_number].merged_column = columns[column_number].get(); + } }; - /// Stores numbers of key-columns and value-columns. - struct MapDescription - { - std::vector key_col_nums; - std::vector val_col_nums; - }; - - struct ColumnsDefinition - { - /// Columns with which values should be summed. - ColumnNumbers column_numbers_not_to_aggregate; - /// Columns which should be aggregated. - std::vector columns_to_aggregate; - /// Mapping for nested columns. - std::vector maps_to_sum; - - size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } - }; - - String getName() const override { return "SummingSortedTransform"; } - void work() override; - -protected: - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; - -private: - Row current_row; - bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. - - ColumnsDefinition columns_definition; - SummingMergedData merged_data; - - SortDescription description; - - /// Chunks currently being merged. - std::vector source_chunks; - SortCursorImpls cursors; - - /// In merging algorithm, we need to compare current sort key with the last one. - /// So, sorting columns for last row needed to be stored. - /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). - Chunk last_chunk; - ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. - struct RowRef { ColumnRawPtrs * sort_columns = nullptr; /// Point to sort_columns from SortCursor or last_chunk_sort_columns. @@ -158,15 +121,73 @@ private: return true; } }; +} - RowRef last_key; +class SummingSortedTransform : public IMergingTransform +{ +public: + + SummingSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description_, + /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. + const Names & column_names_to_sum, + size_t max_block_size); + + /// Stores numbers of key-columns and value-columns. + struct MapDescription + { + std::vector key_col_nums; + std::vector val_col_nums; + }; + + struct ColumnsDefinition + { + /// Columns with which values should be summed. + ColumnNumbers column_numbers_not_to_aggregate; + /// Columns which should be aggregated. + std::vector columns_to_aggregate; + /// Mapping for nested columns. + std::vector maps_to_sum; + + size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } + }; + + String getName() const override { return "SummingSortedTransform"; } + void work() override; + +protected: + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + +private: + Row current_row; + bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. + + ColumnsDefinition columns_definition; + detail::SummingMergedData merged_data; + + SortDescription description; + + /// Chunks currently being merged. + std::vector source_chunks; + SortCursorImpls cursors; + + /// In merging algorithm, we need to compare current sort key with the last one. + /// So, sorting columns for last row needed to be stored. + /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). + Chunk last_chunk; + ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. + + detail::RowRef last_key; SortingHeap queue; bool is_queue_initialized = false; - void insertRow(); void merge(); void updateCursor(Chunk chunk, size_t source_num); + void addRow(SortCursor & cursor); + void insertCurrentRowIfNeeded(); }; } diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp index 4042c146724..8b8b2bfa063 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp @@ -17,6 +17,8 @@ VersionedCollapsingTransform::VersionedCollapsingTransform( , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) + , source_chunks(num_inputs) + , cursors(num_inputs) , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 1) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer , current_keys(max_rows_in_queue) , chunk_allocator(num_inputs + max_rows_in_queue + 1) /// +1 just in case (for current_row) From 77e6714031c1221f3f18c9be608c44adf96133a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Apr 2020 17:28:20 +0300 Subject: [PATCH 078/743] Fix build. --- dbms/src/Processors/Merges/SummingSortedTransform.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index 87bf533d5c5..0741dc1cd10 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -265,6 +265,8 @@ namespace def.maps_to_sum.emplace_back(std::move(map_desc)); } } + + return def; } MutableColumns getMergedDataColumns( From a0f163c7618c6f90534a20eafc1573b82d23ef01 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Apr 2020 21:00:26 +0300 Subject: [PATCH 079/743] Add AggregatingSortedTransform. --- .../Merges/AggregatingSortedTransform.cpp | 252 ++++++++++++++++++ .../Merges/AggregatingSortedTransform.h | 162 +++++++++++ .../Merges/CollapsingSortedTransform.h | 2 +- .../Merges/ReplacingSortedTransform.h | 4 +- dbms/src/Processors/Merges/RowRef.h | 56 ++-- .../Merges/SummingSortedTransform.cpp | 4 +- .../Merges/SummingSortedTransform.h | 217 +++++++-------- .../Merges/VersionedCollapsingTransform.h | 4 +- 8 files changed, 556 insertions(+), 145 deletions(-) create mode 100644 dbms/src/Processors/Merges/AggregatingSortedTransform.cpp create mode 100644 dbms/src/Processors/Merges/AggregatingSortedTransform.h diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp new file mode 100644 index 00000000000..23524dfc395 --- /dev/null +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -0,0 +1,252 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + AggregatingSortedTransform::ColumnsDefinition defineColumns( + const Block & header, const SortDescription & description) + { + AggregatingSortedTransform::ColumnsDefinition def = {}; + size_t num_columns = header.columns(); + + /// Fill in the column numbers that need to be aggregated. + for (size_t i = 0; i < num_columns; ++i) + { + const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + + /// We leave only states of aggregate functions. + if (!dynamic_cast(column.type.get()) + && !dynamic_cast(column.type->getCustomName())) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + /// Included into PK? + auto it = description.begin(); + for (; it != description.end(); ++it) + if (it->column_name == column.name || (it->column_name.empty() && it->column_number == i)) + break; + + if (it != description.end()) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + if (auto simple_aggr = dynamic_cast(column.type->getCustomName())) + { + auto type = recursiveRemoveLowCardinality(column.type); + if (type.get() == column.type.get()) + type = nullptr; + + // simple aggregate function + AggregatingSortedTransform::SimpleAggregateDescription desc(simple_aggr->getFunction(), i, type); + if (desc.function->allocatesMemoryInArena()) + def.allocates_memory_in_arena = true; + + def.columns_to_simple_aggregate.emplace_back(std::move(desc)); + } + else + { + // standard aggregate function + def.columns_to_aggregate.emplace_back(i); + } + } + } +} + +AggregatingSortedTransform::AggregatingSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description_, size_t max_block_size) + : IMergingTransform(num_inputs, header, header, true) + , columns_definition(defineColumns(header, description_)) + , merged_data(header.cloneEmptyColumns(), false, max_block_size) + , description(std::move(description_)) + , source_chunks(num_inputs) + , cursors(num_inputs) +{ + merged_data.initAggregateDescription(columns_definition); +} + +void AggregatingSortedTransform::initializeInputs() +{ + queue = SortingHeap(cursors); + is_queue_initialized = true; +} + +void AggregatingSortedTransform::consume(Chunk chunk, size_t input_number) +{ + updateCursor(std::move(chunk), input_number); + + if (is_queue_initialized) + queue.push(cursors[input_number]); +} + +void AggregatingSortedTransform::updateCursor(Chunk chunk, size_t source_num) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + for (auto & desc : columns_definition.columns_to_simple_aggregate) + if (desc.type_to_convert) + columns[desc.column_number] = recursiveRemoveLowCardinality(columns[desc.column_number]); + + chunk.setColumns(std::move(columns), num_rows); + + auto & source_chunk = source_chunks[source_num]; + + if (source_chunk) + { + /// Extend lifetime of last chunk. + last_chunk = std::move(source_chunk); + last_chunk_sort_columns = std::move(cursors[source_num].all_columns); + + source_chunk = std::move(chunk); + cursors[source_num].reset(source_chunk.getColumns(), {}); + } + else + { + if (cursors[source_num].has_collation) + throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); + + source_chunk = std::move(chunk); + cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num); + } +} + +void AggregatingSortedTransform::work() +{ + merge(); + prepareOutputChunk(merged_data); + + if (has_output_chunk) + { + size_t num_rows = output_chunk.getNumRows(); + auto columns = output_chunk.detachColumns(); + auto & header = getOutputs().back().getHeader(); + + for (auto & desc : columns_definition.columns_to_simple_aggregate) + { + if (desc.type_to_convert) + { + auto & from_type = header.getByPosition(desc.column_number).type; + auto & to_type = desc.type_to_convert; + columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type); + } + } + + output_chunk.setColumns(std::move(columns), num_rows); + + merged_data.initAggregateDescription(columns_definition); + } +} + +void AggregatingSortedTransform::merge() +{ + /// We take the rows in the correct order and put them in `merged_block`, while the rows are no more than `max_block_size` + while (queue.isValid()) + { + bool key_differs; + bool has_previous_group = !last_key.empty(); + + SortCursor current = queue.current(); + + { + detail::RowRef current_key; + current_key.set(current); + + if (!has_previous_group) /// The first key encountered. + key_differs = true; + else + key_differs = !last_key.hasEqualSortColumnsWith(current_key); + + last_key = current_key; + last_chunk_sort_columns.clear(); + } + + if (key_differs) + { + /// if there are enough rows accumulated and the last one is calculated completely + if (merged_data.hasEnoughRows()) + { + /// Write the simple aggregation result for the previous group. + insertSimpleAggregationResult(); + return; + } + + /// We will write the data for the group. We copy the values of ordinary columns. + merged_data.insertRow(current->all_columns, current->pos, + columns_definition.column_numbers_not_to_aggregate); + + /// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function. + for (auto & column_to_aggregate : columns_definition.columns_to_aggregate) + column_to_aggregate.column->insertDefault(); + + /// Write the simple aggregation result for the previous group. + if (merged_data.mergedRows() > 0) + insertSimpleAggregationResult(); + + /// Reset simple aggregation states for next row + for (auto & desc : columns_definition.columns_to_simple_aggregate) + desc.createState(); + + if (columns_definition.allocates_memory_in_arena) + arena = std::make_unique(); + } + + addRow(current); + + if (!current->isLast()) + { + queue.next(); + } + else + { + /// We get the next block from the corresponding source, if there is one. + queue.removeTop(); + requestDataForInput(current.impl->order); + return; + } + } + + /// Write the simple aggregation result for the previous group. + if (merged_data.mergedRows() > 0) + insertSimpleAggregationResult(); + + last_chunk_sort_columns.clear(); + is_finished = true; +} + +void AggregatingSortedTransform::addRow(SortCursor & cursor) +{ + for (auto & desc : columns_definition.columns_to_aggregate) + desc.column->insertMergeFrom(*cursor->all_columns[desc.column_number], cursor->pos); + + for (auto & desc : columns_definition.columns_to_simple_aggregate) + { + auto & col = cursor->all_columns[desc.column_number]; + desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get()); + } +} + +void AggregatingSortedTransform::insertSimpleAggregationResult() +{ + for (auto & desc : columns_definition.columns_to_simple_aggregate) + { + desc.function->insertResultInto(desc.state.data(), *desc.column); + desc.destroyState(); + } +} + +} diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.h b/dbms/src/Processors/Merges/AggregatingSortedTransform.h new file mode 100644 index 00000000000..613ac0baa58 --- /dev/null +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.h @@ -0,0 +1,162 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +class ColumnAggregateFunction; + +class AggregatingSortedTransform : public IMergingTransform +{ +public: + AggregatingSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description_, size_t max_block_size); + + struct SimpleAggregateDescription; + + struct ColumnsDefinition + { + struct AggregateDescription + { + ColumnAggregateFunction * column = nullptr; + const size_t column_number = 0; + + AggregateDescription() = default; + explicit AggregateDescription(size_t col_number) : column_number(col_number) {} + }; + + /// Columns with which numbers should not be aggregated. + ColumnNumbers column_numbers_not_to_aggregate; + std::vector columns_to_aggregate; + std::vector columns_to_simple_aggregate; + + /// Does SimpleAggregateFunction allocates memory in arena? + bool allocates_memory_in_arena = false; + }; + + String getName() const override { return "AggregatingSortedTransform"; } + void work() override; + +protected: + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + +private: + + /// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns. + struct AggregatingMergedData : public MergedData + { + public: + using MergedData::MergedData; + + void insertRow(const ColumnRawPtrs & raw_columns, size_t row, const ColumnNumbers & column_numbers) + { + for (auto column_number :column_numbers) + columns[column_number]->insertFrom(*raw_columns[column_number], row); + + ++total_merged_rows; + ++merged_rows; + /// TODO: sum_blocks_granularity += block_size; + } + + /// Initialize aggregate descriptions with columns. + void initAggregateDescription(ColumnsDefinition & def) + { + for (auto & desc : def.columns_to_simple_aggregate) + desc.column = columns[desc.column_number].get(); + + for (auto & desc : def.columns_to_aggregate) + desc.column = typeid_cast(columns[desc.column_number].get()); + } + }; + + ColumnsDefinition columns_definition; + AggregatingMergedData merged_data; + + SortDescription description; + + /// Chunks currently being merged. + std::vector source_chunks; + SortCursorImpls cursors; + + /// In merging algorithm, we need to compare current sort key with the last one. + /// So, sorting columns for last row needed to be stored. + /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). + Chunk last_chunk; + ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. + + detail::RowRef last_key; + + SortingHeap queue; + bool is_queue_initialized = false; + + /// Memory pool for SimpleAggregateFunction + /// (only when allocates_memory_in_arena == true). + std::unique_ptr arena; + + void merge(); + void updateCursor(Chunk chunk, size_t source_num); + void addRow(SortCursor & cursor); + void insertSimpleAggregationResult(); + +public: + /// Stores information for aggregation of SimpleAggregateFunction columns + struct SimpleAggregateDescription + { + /// An aggregate function 'anyLast', 'sum'... + AggregateFunctionPtr function; + IAggregateFunction::AddFunc add_function = nullptr; + + size_t column_number = 0; + IColumn * column = nullptr; + const DataTypePtr type_to_convert; + + AlignedBuffer state; + bool created = false; + + SimpleAggregateDescription(AggregateFunctionPtr function_, const size_t column_number_, DataTypePtr type) + : function(std::move(function_)), column_number(column_number_), type_to_convert(std::move(type)) + { + add_function = function->getAddressOfAddFunction(); + state.reset(function->sizeOfData(), function->alignOfData()); + } + + void createState() + { + if (created) + return; + function->create(state.data()); + created = true; + } + + void destroyState() + { + if (!created) + return; + function->destroy(state.data()); + created = false; + } + + /// Explicitly destroy aggregation state if the stream is terminated + ~SimpleAggregateDescription() + { + destroyState(); + } + + SimpleAggregateDescription() = default; + SimpleAggregateDescription(SimpleAggregateDescription &&) = default; + SimpleAggregateDescription(const SimpleAggregateDescription &) = delete; + }; +}; + +} diff --git a/dbms/src/Processors/Merges/CollapsingSortedTransform.h b/dbms/src/Processors/Merges/CollapsingSortedTransform.h index 58c97f964bc..46e3fb2e693 100644 --- a/dbms/src/Processors/Merges/CollapsingSortedTransform.h +++ b/dbms/src/Processors/Merges/CollapsingSortedTransform.h @@ -64,7 +64,7 @@ private: SortingHeap queue; bool is_queue_initialized = false; - using RowRef = detail::RowRef; + using RowRef = detail::RowRefWithOwnedChunk; static constexpr size_t max_row_refs = 4; /// first_negative, last_positive, last, current. RowRef first_negative_row; RowRef last_positive_row; diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.h b/dbms/src/Processors/Merges/ReplacingSortedTransform.h index 4f4b71c5b13..d28bd239cfe 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.h +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.h @@ -13,7 +13,7 @@ namespace DB { -class ReplacingSortedTransform : public IMergingTransform +class ReplacingSortedTransform final : public IMergingTransform { public: ReplacingSortedTransform( @@ -50,7 +50,7 @@ private: SortingHeap queue; bool is_queue_initialized = false; - using RowRef = detail::RowRef; + using RowRef = detail::RowRefWithOwnedChunk; static constexpr size_t max_row_refs = 3; /// last, current, selected. RowRef last_row; /// RowRef next_key; /// Primary key of next row. diff --git a/dbms/src/Processors/Merges/RowRef.h b/dbms/src/Processors/Merges/RowRef.h index 67d32d11277..ac4be79f560 100644 --- a/dbms/src/Processors/Merges/RowRef.h +++ b/dbms/src/Processors/Merges/RowRef.h @@ -103,10 +103,46 @@ inline void intrusive_ptr_release(SharedChunk * ptr) } /// This class represents a row in a chunk. -/// RowRef hold shared pointer to this chunk, possibly extending its life time. +struct RowRef +{ + ColumnRawPtrs * sort_columns = nullptr; /// Point to sort_columns from SortCursor or last_chunk_sort_columns. + UInt64 row_num = 0; + + bool empty() const { return sort_columns == nullptr; } + void reset() { sort_columns = nullptr; } + + void set(SortCursor & cursor) + { + sort_columns = &cursor.impl->sort_columns; + row_num = cursor.impl->pos; + } + + static bool checkEquals(const ColumnRawPtrs * left, size_t left_row, const ColumnRawPtrs * right, size_t right_row) + { + auto size = left->size(); + for (size_t col_number = 0; col_number < size; ++col_number) + { + auto & cur_column = (*left)[col_number]; + auto & other_column = (*right)[col_number]; + + if (0 != cur_column->compareAt(left_row, right_row, *other_column, 1)) + return false; + } + + return true; + } + + bool hasEqualSortColumnsWith(const RowRef & other) + { + return checkEquals(sort_columns, row_num, other.sort_columns, other.row_num); + } +}; + +/// This class also represents a row in a chunk. +/// RowRefWithOwnedChunk hold shared pointer to this chunk, possibly extending its life time. /// It is needed, for example, in CollapsingTransform, where we need to store first negative row for current sort key. /// We do not copy data itself, because it may be potentially changed for each row. Performance for `set` is important. -struct RowRef +struct RowRefWithOwnedChunk { detail::SharedChunkPtr owned_chunk = nullptr; @@ -114,7 +150,7 @@ struct RowRef ColumnRawPtrs * sort_columns = nullptr; UInt64 row_num = 0; - void swap(RowRef & other) + void swap(RowRefWithOwnedChunk & other) { owned_chunk.swap(other.owned_chunk); std::swap(all_columns, other.all_columns); @@ -140,19 +176,9 @@ struct RowRef sort_columns = &owned_chunk->sort_columns; } - bool hasEqualSortColumnsWith(const RowRef & other) + bool hasEqualSortColumnsWith(const RowRefWithOwnedChunk & other) { - auto size = sort_columns->size(); - for (size_t col_number = 0; col_number < size; ++col_number) - { - auto & cur_column = (*sort_columns)[col_number]; - auto & other_column = (*other.sort_columns)[col_number]; - - if (0 != cur_column->compareAt(row_num, other.row_num, *other_column, 1)) - return false; - } - - return true; + return RowRef::checkEquals(sort_columns, row_num, other.sort_columns, other.row_num); } }; diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index 0741dc1cd10..99008025232 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -168,7 +168,7 @@ namespace std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name)) { // Create aggregator to sum this column - detail::AggregateDescription desc; + SummingSortedTransform::AggregateDescription desc; desc.is_agg_func_type = is_agg_func; desc.column_numbers = {i}; @@ -211,7 +211,7 @@ namespace } DataTypes argument_types; - detail::AggregateDescription desc; + SummingSortedTransform::AggregateDescription desc; SummingSortedTransform::MapDescription map_desc; column_num_it = map.second.begin(); diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.h b/dbms/src/Processors/Merges/SummingSortedTransform.h index 678ff6587a7..e7915cd3c8c 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.h +++ b/dbms/src/Processors/Merges/SummingSortedTransform.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -13,8 +14,100 @@ namespace DB { -namespace detail +class SummingSortedTransform final : public IMergingTransform { +public: + + SummingSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description_, + /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. + const Names & column_names_to_sum, + size_t max_block_size); + + struct AggregateDescription; + + /// Stores numbers of key-columns and value-columns. + struct MapDescription + { + std::vector key_col_nums; + std::vector val_col_nums; + }; + + struct ColumnsDefinition + { + /// Columns with which values should be summed. + ColumnNumbers column_numbers_not_to_aggregate; + /// Columns which should be aggregated. + std::vector columns_to_aggregate; + /// Mapping for nested columns. + std::vector maps_to_sum; + + size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } + }; + + /// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns. + struct SummingMergedData : public MergedData + { + public: + using MergedData::MergedData; + + void insertRow(const Row & row, const ColumnNumbers & column_numbers) + { + for (auto column_number :column_numbers) + columns[column_number]->insert(row[column_number]); + + ++total_merged_rows; + ++merged_rows; + /// TODO: sum_blocks_granularity += block_size; + } + + /// Initialize aggregate descriptions with columns. + void initAggregateDescription(std::vector & columns_to_aggregate) + { + size_t num_columns = columns_to_aggregate.size(); + for (size_t column_number = 0; column_number < num_columns; ++column_number) + columns_to_aggregate[column_number].merged_column = columns[column_number].get(); + } + }; + + String getName() const override { return "SummingSortedTransform"; } + void work() override; + +protected: + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + +private: + Row current_row; + bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. + + ColumnsDefinition columns_definition; + SummingMergedData merged_data; + + SortDescription description; + + /// Chunks currently being merged. + std::vector source_chunks; + SortCursorImpls cursors; + + /// In merging algorithm, we need to compare current sort key with the last one. + /// So, sorting columns for last row needed to be stored. + /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). + Chunk last_chunk; + ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. + + detail::RowRef last_key; + + SortingHeap queue; + bool is_queue_initialized = false; + + void merge(); + void updateCursor(Chunk chunk, size_t source_num); + void addRow(SortCursor & cursor); + void insertCurrentRowIfNeeded(); + +public: /// Stores aggregation function, state, and columns to be used as function arguments. struct AggregateDescription { @@ -66,128 +159,6 @@ namespace detail AggregateDescription(AggregateDescription &&) = default; AggregateDescription(const AggregateDescription &) = delete; }; - - /// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns. - struct SummingMergedData : public MergedData - { - public: - using MergedData::MergedData; - - void insertRow(const Row & row, const ColumnNumbers & column_numbers) - { - for (auto column_number :column_numbers) - columns[column_number]->insert(row[column_number]); - - ++total_merged_rows; - ++merged_rows; - /// TODO: sum_blocks_granularity += block_size; - } - - /// Initialize aggregate descriptions with columns. - void initAggregateDescription(std::vector & columns_to_aggregate) - { - size_t num_columns = columns_to_aggregate.size(); - for (size_t column_number = 0; column_number < num_columns; ++column_number) - columns_to_aggregate[column_number].merged_column = columns[column_number].get(); - } - }; - - struct RowRef - { - ColumnRawPtrs * sort_columns = nullptr; /// Point to sort_columns from SortCursor or last_chunk_sort_columns. - UInt64 row_num = 0; - - bool empty() const { return sort_columns == nullptr; } - void reset() { sort_columns = nullptr; } - - void set(SortCursor & cursor) - { - sort_columns = &cursor.impl->sort_columns; - row_num = cursor.impl->pos; - } - - bool hasEqualSortColumnsWith(const RowRef & other) - { - auto size = sort_columns->size(); - for (size_t col_number = 0; col_number < size; ++col_number) - { - auto & cur_column = (*sort_columns)[col_number]; - auto & other_column = (*other.sort_columns)[col_number]; - - if (0 != cur_column->compareAt(row_num, other.row_num, *other_column, 1)) - return false; - } - - return true; - } - }; -} - -class SummingSortedTransform : public IMergingTransform -{ -public: - - SummingSortedTransform( - size_t num_inputs, const Block & header, - SortDescription description_, - /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. - const Names & column_names_to_sum, - size_t max_block_size); - - /// Stores numbers of key-columns and value-columns. - struct MapDescription - { - std::vector key_col_nums; - std::vector val_col_nums; - }; - - struct ColumnsDefinition - { - /// Columns with which values should be summed. - ColumnNumbers column_numbers_not_to_aggregate; - /// Columns which should be aggregated. - std::vector columns_to_aggregate; - /// Mapping for nested columns. - std::vector maps_to_sum; - - size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } - }; - - String getName() const override { return "SummingSortedTransform"; } - void work() override; - -protected: - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; - -private: - Row current_row; - bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. - - ColumnsDefinition columns_definition; - detail::SummingMergedData merged_data; - - SortDescription description; - - /// Chunks currently being merged. - std::vector source_chunks; - SortCursorImpls cursors; - - /// In merging algorithm, we need to compare current sort key with the last one. - /// So, sorting columns for last row needed to be stored. - /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). - Chunk last_chunk; - ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. - - detail::RowRef last_key; - - SortingHeap queue; - bool is_queue_initialized = false; - - void merge(); - void updateCursor(Chunk chunk, size_t source_num); - void addRow(SortCursor & cursor); - void insertCurrentRowIfNeeded(); }; } diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h index 722bd30feca..0dbdf8e2a40 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h @@ -15,7 +15,7 @@ namespace DB { -class VersionedCollapsingTransform : public IMergingTransform +class VersionedCollapsingTransform final : public IMergingTransform { public: /// Don't need version column. It's in primary key. @@ -53,7 +53,7 @@ private: SortingHeap queue; bool is_queue_initialized = false; - using RowRef = detail::RowRef; + using RowRef = detail::RowRefWithOwnedChunk; const size_t max_rows_in_queue; /// Rows with the same primary key and sign. FixedSizeDequeWithGaps current_keys; From 5b83ca2fb69ae5a5ad8cb00f6c9e839e09fad506 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Apr 2020 21:06:55 +0300 Subject: [PATCH 080/743] Fix build. --- dbms/src/Processors/Merges/AggregatingSortedTransform.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 23524dfc395..1c94bca4e3a 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -60,6 +60,8 @@ namespace def.columns_to_aggregate.emplace_back(i); } } + + return def; } } From 1176b5777e5b43473bd98b341ac798d89774a39b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Thu, 2 Apr 2020 00:57:00 +0300 Subject: [PATCH 081/743] added settings parsing and initialisation to DictionaryStructure --- dbms/src/Dictionaries/DictionaryStructure.cpp | 32 + dbms/src/Dictionaries/DictionaryStructure.h | 7 + dbms/src/Dictionaries/DirectDictionary.cpp | 582 ------------------ dbms/src/Dictionaries/DirectDictionary.h | 294 --------- dbms/src/Dictionaries/DirectDictionary.inc.h | 406 ------------ 5 files changed, 39 insertions(+), 1282 deletions(-) delete mode 100644 dbms/src/Dictionaries/DirectDictionary.cpp delete mode 100644 dbms/src/Dictionaries/DirectDictionary.h delete mode 100644 dbms/src/Dictionaries/DirectDictionary.inc.h diff --git a/dbms/src/Dictionaries/DictionaryStructure.cpp b/dbms/src/Dictionaries/DictionaryStructure.cpp index f8b8fbd6aab..132b9effcb7 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.cpp +++ b/dbms/src/Dictionaries/DictionaryStructure.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -20,6 +21,7 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int TYPE_MISMATCH; extern const int BAD_ARGUMENTS; + extern const int NO_ELEMENTS_IN_CONFIG; } namespace @@ -193,6 +195,10 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration } attributes = getAttributes(config, config_prefix); + + settings = Settings(); + getSettings(config, config_prefix, settings); + if (attributes.empty()) throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS}; } @@ -354,4 +360,30 @@ std::vector DictionaryStructure::getAttributes( return res_attributes; } + +void DictionaryStructure::getSettings( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Settings & settings) +{ + Poco::Util::AbstractConfiguration::Keys config_elems; + config.keys(config_prefix, config_elems); + + + for (const auto & config_elem : config_elems) + { + if (startsWith(config_elem, "settings")) + { + /* i won't do break after this if in case there can be multiple settings sections */ + + const auto prefix = config_prefix + '.' + config_elem; + Poco::Util::AbstractConfiguration::Keys setting_keys; + config.keys(prefix, setting_keys); + settings.loadSettingsFromConfig(prefix, config); + + } + } + +} + } diff --git a/dbms/src/Dictionaries/DictionaryStructure.h b/dbms/src/Dictionaries/DictionaryStructure.h index 2893dea2e4f..9a7428959a6 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.h +++ b/dbms/src/Dictionaries/DictionaryStructure.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -101,6 +102,7 @@ struct DictionaryStructure final std::optional id; std::optional> key; std::vector attributes; + Settings settings; std::optional range_min; std::optional range_max; bool has_expressions = false; @@ -118,6 +120,11 @@ private: const std::string & config_prefix, const bool hierarchy_allowed = true, const bool allow_null_values = true); + + void getSettings( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Settings & settings); }; } diff --git a/dbms/src/Dictionaries/DirectDictionary.cpp b/dbms/src/Dictionaries/DirectDictionary.cpp deleted file mode 100644 index 9e0a77ebc91..00000000000 --- a/dbms/src/Dictionaries/DirectDictionary.cpp +++ /dev/null @@ -1,582 +0,0 @@ -#include "DirectDictionary.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "DirectDictionary.inc.h" -#include "DictionaryBlockInputStream.h" -#include "DictionaryFactory.h" - - -/* - * - * TODO: CHANGE EVENTS TO DIRECT DICTIONARY EVENTS (WTF? WHERE R THEY DECLARED????) - * -*/ - -namespace ProfileEvents -{ - extern const Event DictCacheKeysRequested; - extern const Event DictCacheKeysRequestedMiss; - extern const Event DictCacheKeysRequestedFound; - extern const Event DictCacheKeysExpired; - extern const Event DictCacheKeysNotFound; - extern const Event DictCacheKeysHit; - extern const Event DictCacheRequestTimeNs; - extern const Event DictCacheRequests; - extern const Event DictCacheLockWriteNs; - extern const Event DictCacheLockReadNs; -} - -namespace CurrentMetrics -{ - extern const Metric DictCacheRequests; -} - - -namespace DB -{ - namespace ErrorCodes - { - extern const int TYPE_MISMATCH; - extern const int BAD_ARGUMENTS; - extern const int UNSUPPORTED_METHOD; - extern const int LOGICAL_ERROR; - extern const int TOO_SMALL_BUFFER_SIZE; - } - - /* - * deleted inline size_t DirectDictionary::getCellIdx(const Key id) const - * - */ - - - DirectDictionary::DirectDictionary( - const std::string & name_, - const DictionaryStructure & dict_struct_, - DictionarySourcePtr source_ptr_, - const DictionaryLifetime dict_lifetime_ - ) - : name{name_} - , dict_struct(dict_struct_) - , source_ptr{std::move(source_ptr_)} - , dict_lifetime(dict_lifetime_) - , log(&Logger::get("ExternalDictionaries")) - , rnd_engine(randomSeed()) - { - if (!this->source_ptr->supportsSelectiveLoad()) - throw Exception{name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD}; - - createAttributes(); - } - - - void DirectDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const - { - const auto null_value = std::get(hierarchical_attribute->null_values); - - getItemsNumberImpl(*hierarchical_attribute, ids, out, [&](const size_t) { return null_value; }); - } - - -/// Allow to use single value in same way as array. - static inline DirectDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) - { - return arr[idx]; - } - static inline DirectDictionary::Key getAt(const DirectDictionary::Key & value, const size_t) - { - return value; - } - - - template - void DirectDictionary::isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const - { - /// Transform all children to parents until ancestor id or null_value will be reached. - - size_t out_size = out.size(); - memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated" - - const auto null_value = std::get(hierarchical_attribute->null_values); - - PaddedPODArray children(out_size, 0); - PaddedPODArray parents(child_ids.begin(), child_ids.end()); - - while (true) - { - size_t out_idx = 0; - size_t parents_idx = 0; - size_t new_children_idx = 0; - - while (out_idx < out_size) - { - /// Already calculated - if (out[out_idx] != 0xFF) - { - ++out_idx; - continue; - } - - /// No parent - if (parents[parents_idx] == null_value) - { - out[out_idx] = 0; - } - /// Found ancestor - else if (parents[parents_idx] == getAt(ancestor_ids, parents_idx)) - { - out[out_idx] = 1; - } - /// Loop detected - else if (children[new_children_idx] == parents[parents_idx]) - { - out[out_idx] = 1; - } - /// Found intermediate parent, add this value to search at next loop iteration - else - { - children[new_children_idx] = parents[parents_idx]; - ++new_children_idx; - } - - ++out_idx; - ++parents_idx; - } - - if (new_children_idx == 0) - break; - - /// Transform all children to its parents. - children.resize(new_children_idx); - parents.resize(new_children_idx); - - toParent(children, parents); - } - } - - void DirectDictionary::isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const - { - isInImpl(child_ids, ancestor_ids, out); - } - - void DirectDictionary::isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const - { - isInImpl(child_ids, ancestor_id, out); - } - - void DirectDictionary::isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const - { - /// Special case with single child value. - - const auto null_value = std::get(hierarchical_attribute->null_values); - - PaddedPODArray child(1, child_id); - PaddedPODArray parent(1); - std::vector ancestors(1, child_id); - - /// Iteratively find all ancestors for child. - while (true) - { - toParent(child, parent); - - if (parent[0] == null_value) - break; - - child[0] = parent[0]; - ancestors.push_back(parent[0]); - } - - /// Assuming short hierarchy, so linear search is Ok. - for (size_t i = 0, out_size = out.size(); i < out_size; ++i) - out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end(); - } - - void DirectDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const - { - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); - - const auto null_value = StringRef{std::get(attribute.null_values)}; - - getItemsString(attribute, ids, out, [&](const size_t) { return null_value; }); - } - - void DirectDictionary::getString( - const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const - { - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); - - getItemsString(attribute, ids, out, [&](const size_t row) { return def->getDataAt(row); }); - } - - void DirectDictionary::getString( - const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const - { - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); - - getItemsString(attribute, ids, out, [&](const size_t) { return StringRef{def}; }); - } - - -/// returns cell_idx (always valid for replacing), 'cell is valid' flag, 'cell is outdated' flag -/// true false found and valid -/// false true not found (something outdated, maybe our cell) -/// false false not found (other id stored with valid data) -/// true true impossible -/// -/// todo: split this func to two: find_for_get and find_for_set - DirectDictionary::FindResult DirectDictionary::findCellIdx(const Key & id, const CellMetadata::time_point_t now) const - { - auto pos = getCellIdx(id); - auto oldest_id = pos; - auto oldest_time = CellMetadata::time_point_t::max(); - const auto stop = pos + max_collision_length; - for (; pos < stop; ++pos) - { - const auto cell_idx = pos & size_overlap_mask; - const auto & cell = cells[cell_idx]; - - if (cell.id != id) - { - /// maybe we already found nearest expired cell (try minimize collision_length on insert) - if (oldest_time > now && oldest_time > cell.expiresAt()) - { - oldest_time = cell.expiresAt(); - oldest_id = cell_idx; - } - continue; - } - - if (cell.expiresAt() < now) - { - return {cell_idx, false, true}; - } - - return {cell_idx, true, false}; - } - - return {oldest_id, false, false}; - } - - - /* - * deleted most part of has, that stood for - * looking for a key in cache - * - * TODO: check whether we need last two arguments - * in update function (seems like no) - * - */ - - void DirectDictionary::has(const PaddedPODArray & ids, PaddedPODArray & out) const - { - std::vector required_ids(ids.size()); - std::copy(std::begin(ids), std::end(ids), std::begin(required_ids)); - - /// request new values - update( - required_ids, - [&](const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - out[row] = true; - }, - [&](const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - out[row] = false; - }); - } - - - void DirectDictionary::createAttributes() - { - const auto attributes_size = dict_struct.attributes.size(); - attributes.reserve(attributes_size); - - bytes_allocated += size * sizeof(CellMetadata); - bytes_allocated += attributes_size * sizeof(attributes.front()); - - for (const auto & attribute : dict_struct.attributes) - { - attribute_index_by_name.emplace(attribute.name, attributes.size()); - attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); - - if (attribute.hierarchical) - { - hierarchical_attribute = &attributes.back(); - - if (hierarchical_attribute->type != AttributeUnderlyingType::utUInt64) - throw Exception{name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH}; - } - } - } - - DirectDictionary::Attribute DirectDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) - { - Attribute attr{type, {}, {}}; - - switch (type) - { -#define DISPATCH(TYPE) \ - case AttributeUnderlyingType::ut##TYPE: \ - attr.null_values = TYPE(null_value.get>()); \ - attr.arrays = std::make_unique>(size); \ - bytes_allocated += size * sizeof(TYPE); \ - break; - DISPATCH(UInt8) - DISPATCH(UInt16) - DISPATCH(UInt32) - DISPATCH(UInt64) - DISPATCH(UInt128) - DISPATCH(Int8) - DISPATCH(Int16) - DISPATCH(Int32) - DISPATCH(Int64) - DISPATCH(Decimal32) - DISPATCH(Decimal64) - DISPATCH(Decimal128) - DISPATCH(Float32) - DISPATCH(Float64) -#undef DISPATCH - case AttributeUnderlyingType::utString: - attr.null_values = null_value.get(); - attr.arrays = std::make_unique>(size); - bytes_allocated += size * sizeof(StringRef); - if (!string_arena) - string_arena = std::make_unique(); - break; - } - - return attr; - } - - void DirectDictionary::setDefaultAttributeValue(Attribute & attribute, const Key idx) const - { - switch (attribute.type) - { - case AttributeUnderlyingType::utUInt8: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utUInt16: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utUInt32: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utUInt64: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utUInt128: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utInt8: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utInt16: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utInt32: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utInt64: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utFloat32: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utFloat64: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - - case AttributeUnderlyingType::utDecimal32: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utDecimal64: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - case AttributeUnderlyingType::utDecimal128: - std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); - break; - - case AttributeUnderlyingType::utString: - { - const auto & null_value_ref = std::get(attribute.null_values); - auto & string_ref = std::get>(attribute.arrays)[idx]; - - if (string_ref.data != null_value_ref.data()) - { - if (string_ref.data) - string_arena->free(const_cast(string_ref.data), string_ref.size); - - string_ref = StringRef{null_value_ref}; - } - - break; - } - } - } - - void DirectDictionary::setAttributeValue(Attribute & attribute, const Key idx, const Field & value) const - { - switch (attribute.type) - { - case AttributeUnderlyingType::utUInt8: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utUInt16: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utUInt32: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utUInt64: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utUInt128: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utInt8: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utInt16: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utInt32: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utInt64: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utFloat32: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utFloat64: - std::get>(attribute.arrays)[idx] = value.get(); - break; - - case AttributeUnderlyingType::utDecimal32: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utDecimal64: - std::get>(attribute.arrays)[idx] = value.get(); - break; - case AttributeUnderlyingType::utDecimal128: - std::get>(attribute.arrays)[idx] = value.get(); - break; - - case AttributeUnderlyingType::utString: - { - const auto & string = value.get(); - auto & string_ref = std::get>(attribute.arrays)[idx]; - const auto & null_value_ref = std::get(attribute.null_values); - - /// free memory unless it points to a null_value - if (string_ref.data && string_ref.data != null_value_ref.data()) - string_arena->free(const_cast(string_ref.data), string_ref.size); - - const auto str_size = string.size(); - if (str_size != 0) - { - auto string_ptr = string_arena->alloc(str_size + 1); - std::copy(string.data(), string.data() + str_size + 1, string_ptr); - string_ref = StringRef{string_ptr, str_size}; - } - else - string_ref = {}; - - break; - } - } - } - - DirectDictionary::Attribute & DirectDictionary::getAttribute(const std::string & attribute_name) const - { - const auto it = attribute_index_by_name.find(attribute_name); - if (it == std::end(attribute_index_by_name)) - throw Exception{name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; - - return attributes[it->second]; - } - - /* - * I've deleted: - * bool CacheDictionary::isEmptyCell(const UInt64 idx) const - * and - * PaddedPODArray CacheDictionary::getCachedIds() const - */ - - BlockInputStreamPtr DirectDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const - { - using BlockInputStreamType = DictionaryBlockInputStream; - - /* deleted pre-last argument getCachedIds() from this return (will something break then?) */ - return std::make_shared(shared_from_this(), max_block_size, column_names); - } - - std::exception_ptr DirectDictionary::getLastException() const - { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - return last_exception; - } - - void registerDictionaryDirect(DictionaryFactory & factory) - { - auto create_layout = [=](const std::string & name, - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - DictionarySourcePtr source_ptr) -> DictionaryPtr - { - if (dict_struct.key) - throw Exception{"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD}; - - if (dict_struct.range_min || dict_struct.range_max) - throw Exception{name - + ": elements .structure.range_min and .structure.range_max should be defined only " - "for a dictionary of layout 'range_hashed'", - ErrorCodes::BAD_ARGUMENTS}; - const auto & layout_prefix = config_prefix + ".layout"; - - /* - * - * seems like this stands only for cache dictionaries - * - const auto size = config.getInt(layout_prefix + ".cache.size_in_cells"); - if (size == 0) - throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; - - */ - - const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - if (require_nonempty) - throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", - ErrorCodes::BAD_ARGUMENTS}; - - const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; - - /* deleted last argument (size) in this return */ - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime); - }; - factory.registerLayout("direct", create_layout, false); - } - - -} diff --git a/dbms/src/Dictionaries/DirectDictionary.h b/dbms/src/Dictionaries/DirectDictionary.h deleted file mode 100644 index 1431adccf02..00000000000 --- a/dbms/src/Dictionaries/DirectDictionary.h +++ /dev/null @@ -1,294 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "DictionaryStructure.h" -#include "IDictionary.h" -#include "IDictionarySource.h" - - -namespace DB -{ - class DirectDictionary final : public IDictionary - { - public: - /* Removed last argument (size_) */ - DirectDictionary( - const std::string & name_, - const DictionaryStructure & dict_struct_, - DictionarySourcePtr source_ptr_, - const DictionaryLifetime dict_lifetime_ - ); - - std::string getName() const override { return name; } - - std::string getTypeName() const override { return "Cache"; } - - size_t getBytesAllocated() const override { return bytes_allocated + (string_arena ? string_arena->size() : 0); } - - size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } - - double getHitRate() const override - { - return static_cast(hit_count.load(std::memory_order_acquire)) / query_count.load(std::memory_order_relaxed); - } - - size_t getElementCount() const override { return element_count.load(std::memory_order_relaxed); } - - double getLoadFactor() const override { return static_cast(element_count.load(std::memory_order_relaxed)) / size; } - - bool isCached() const override { return false; } - - std::shared_ptr clone() const override - { - return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, size); - } - - const IDictionarySource * getSource() const override { return source_ptr.get(); } - - const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } - - const DictionaryStructure & getStructure() const override { return dict_struct; } - - bool isInjective(const std::string & attribute_name) const override - { - return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective; - } - - bool hasHierarchy() const override { return hierarchical_attribute; } - - void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const override; - - void isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; - void isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const override; - void isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; - - std::exception_ptr getLastException() const override; - - template - using ResultArrayType = std::conditional_t, DecimalPaddedPODArray, PaddedPODArray>; - -#define DECLARE(TYPE) \ - void get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const; - DECLARE(UInt8) - DECLARE(UInt16) - DECLARE(UInt32) - DECLARE(UInt64) - DECLARE(UInt128) - DECLARE(Int8) - DECLARE(Int16) - DECLARE(Int32) - DECLARE(Int64) - DECLARE(Float32) - DECLARE(Float64) - DECLARE(Decimal32) - DECLARE(Decimal64) - DECLARE(Decimal128) -#undef DECLARE - - void getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const; - -#define DECLARE(TYPE) \ - void get##TYPE( \ - const std::string & attribute_name, \ - const PaddedPODArray & ids, \ - const PaddedPODArray & def, \ - ResultArrayType & out) const; - DECLARE(UInt8) - DECLARE(UInt16) - DECLARE(UInt32) - DECLARE(UInt64) - DECLARE(UInt128) - DECLARE(Int8) - DECLARE(Int16) - DECLARE(Int32) - DECLARE(Int64) - DECLARE(Float32) - DECLARE(Float64) - DECLARE(Decimal32) - DECLARE(Decimal64) - DECLARE(Decimal128) -#undef DECLARE - - void - getString(const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) - const; - -#define DECLARE(TYPE) \ - void get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const; - DECLARE(UInt8) - DECLARE(UInt16) - DECLARE(UInt32) - DECLARE(UInt64) - DECLARE(UInt128) - DECLARE(Int8) - DECLARE(Int16) - DECLARE(Int32) - DECLARE(Int64) - DECLARE(Float32) - DECLARE(Float64) - DECLARE(Decimal32) - DECLARE(Decimal64) - DECLARE(Decimal128) -#undef DECLARE - - void getString(const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const; - - void has(const PaddedPODArray & ids, PaddedPODArray & out) const override; - - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; - - private: - template - using ContainerType = Value[]; - template - using ContainerPtrType = std::unique_ptr>; - - struct CellMetadata final - { - using time_point_t = std::chrono::system_clock::time_point; - using time_point_rep_t = time_point_t::rep; - using time_point_urep_t = std::make_unsigned_t; - - static constexpr UInt64 EXPIRES_AT_MASK = std::numeric_limits::max(); - static constexpr UInt64 IS_DEFAULT_MASK = ~EXPIRES_AT_MASK; - - UInt64 id; - /// Stores both expiration time and `is_default` flag in the most significant bit - time_point_urep_t data; - - /// Sets expiration time, resets `is_default` flag to false - time_point_t expiresAt() const { return ext::safe_bit_cast(data & EXPIRES_AT_MASK); } - void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast(t); } - - bool isDefault() const { return (data & IS_DEFAULT_MASK) == IS_DEFAULT_MASK; } - void setDefault() { data |= IS_DEFAULT_MASK; } - }; - - struct Attribute final - { - AttributeUnderlyingType type; - std::variant< - UInt8, - UInt16, - UInt32, - UInt64, - UInt128, - Int8, - Int16, - Int32, - Int64, - Decimal32, - Decimal64, - Decimal128, - Float32, - Float64, - String> - null_values; - std::variant< - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType> - arrays; - }; - - void createAttributes(); - - Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value); - - template - void getItemsNumberImpl( - Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const; - - template - void getItemsString(Attribute & attribute, const PaddedPODArray & ids, ColumnString * out, DefaultGetter && get_default) const; - - template - void update(const std::vector & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const; - - PaddedPODArray getCachedIds() const; - - bool isEmptyCell(const UInt64 idx) const; - - size_t getCellIdx(const Key id) const; - - void setDefaultAttributeValue(Attribute & attribute, const Key idx) const; - - void setAttributeValue(Attribute & attribute, const Key idx, const Field & value) const; - - Attribute & getAttribute(const std::string & attribute_name) const; - - struct FindResult - { - const size_t cell_idx; - const bool valid; - const bool outdated; - }; - - FindResult findCellIdx(const Key & id, const CellMetadata::time_point_t now) const; - - template - void isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - - const std::string name; - const DictionaryStructure dict_struct; - mutable DictionarySourcePtr source_ptr; - const DictionaryLifetime dict_lifetime; - Logger * const log; - - mutable std::shared_mutex rw_lock; - - /// Actual size will be increased to match power of 2 - const size_t size; - - /// all bits to 1 mask (size - 1) (0b1000 - 1 = 0b111) - const size_t size_overlap_mask; - - /// Max tries to find cell, overlaped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3 - static constexpr size_t max_collision_length = 10; - - const size_t zero_cell_idx{getCellIdx(0)}; - std::map attribute_index_by_name; - mutable std::vector attributes; - mutable std::vector cells; - Attribute * hierarchical_attribute = nullptr; - std::unique_ptr string_arena; - - mutable std::exception_ptr last_exception; - mutable size_t error_count = 0; - mutable std::chrono::system_clock::time_point backoff_end_time; - - mutable pcg64 rnd_engine; - - mutable size_t bytes_allocated = 0; - mutable std::atomic element_count{0}; - mutable std::atomic hit_count{0}; - mutable std::atomic query_count{0}; - }; - -} diff --git a/dbms/src/Dictionaries/DirectDictionary.inc.h b/dbms/src/Dictionaries/DirectDictionary.inc.h deleted file mode 100644 index 68010b0fe19..00000000000 --- a/dbms/src/Dictionaries/DirectDictionary.inc.h +++ /dev/null @@ -1,406 +0,0 @@ -#include "CacheDictionary.h" - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace ProfileEvents -{ - extern const Event DictCacheKeysRequested; - extern const Event DictCacheKeysRequestedMiss; - extern const Event DictCacheKeysRequestedFound; - extern const Event DictCacheKeysExpired; - extern const Event DictCacheKeysNotFound; - extern const Event DictCacheKeysHit; - extern const Event DictCacheRequestTimeNs; - extern const Event DictCacheRequests; - extern const Event DictCacheLockWriteNs; - extern const Event DictCacheLockReadNs; -} - -namespace CurrentMetrics -{ - extern const Metric DictCacheRequests; -} - -namespace DB -{ - namespace ErrorCodes - { - extern const int TYPE_MISMATCH; - } - - template - void CacheDictionary::getItemsNumberImpl( - Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const - { - /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } - std::unordered_map> outdated_ids; - auto & attribute_array = std::get>(attribute.arrays); - const auto rows = ext::size(ids); - - size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; - - { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - - const auto now = std::chrono::system_clock::now(); - /// fetch up-to-date values, decide which ones require update - for (const auto row : ext::range(0, rows)) - { - const auto id = ids[row]; - - /** cell should be updated if either: - * 1. ids do not match, - * 2. cell has expired, - * 3. explicit defaults were specified and cell was set default. */ - - const auto find_result = findCellIdx(id, now); - if (!find_result.valid) - { - outdated_ids[id].push_back(row); - if (find_result.outdated) - ++cache_expired; - else - ++cache_not_found; - } - else - { - ++cache_hit; - const auto & cell_idx = find_result.cell_idx; - const auto & cell = cells[cell_idx]; - out[row] = cell.isDefault() ? get_default(row) : static_cast(attribute_array[cell_idx]); - } - } - } - - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); - - query_count.fetch_add(rows, std::memory_order_relaxed); - hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); - - if (outdated_ids.empty()) - return; - - std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); - - /// request new values - update( - required_ids, - [&](const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; - - for (const size_t row : outdated_ids[id]) - out[row] = static_cast(attribute_value); - }, - [&](const auto id, const auto) - { - for (const size_t row : outdated_ids[id]) - out[row] = get_default(row); - }); - } - - template - void CacheDictionary::getItemsString( - Attribute & attribute, const PaddedPODArray & ids, ColumnString * out, DefaultGetter && get_default) const - { - const auto rows = ext::size(ids); - - /// save on some allocations - out->getOffsets().reserve(rows); - - auto & attribute_array = std::get>(attribute.arrays); - - auto found_outdated_values = false; - - /// perform optimistic version, fallback to pessimistic if failed - { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - - const auto now = std::chrono::system_clock::now(); - /// fetch up-to-date values, discard on fail - for (const auto row : ext::range(0, rows)) - { - const auto id = ids[row]; - - const auto find_result = findCellIdx(id, now); - if (!find_result.valid) - { - found_outdated_values = true; - break; - } - else - { - const auto & cell_idx = find_result.cell_idx; - const auto & cell = cells[cell_idx]; - const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; - out->insertData(string_ref.data, string_ref.size); - } - } - } - - /// optimistic code completed successfully - if (!found_outdated_values) - { - query_count.fetch_add(rows, std::memory_order_relaxed); - hit_count.fetch_add(rows, std::memory_order_release); - return; - } - - /// now onto the pessimistic one, discard possible partial results from the optimistic path - out->getChars().resize_assume_reserved(0); - out->getOffsets().resize_assume_reserved(0); - - /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } - std::unordered_map> outdated_ids; - /// we are going to store every string separately - std::unordered_map map; - - size_t total_length = 0; - size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; - { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - - const auto now = std::chrono::system_clock::now(); - for (const auto row : ext::range(0, ids.size())) - { - const auto id = ids[row]; - - const auto find_result = findCellIdx(id, now); - if (!find_result.valid) - { - outdated_ids[id].push_back(row); - if (find_result.outdated) - ++cache_expired; - else - ++cache_not_found; - } - else - { - ++cache_hit; - const auto & cell_idx = find_result.cell_idx; - const auto & cell = cells[cell_idx]; - const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; - - if (!cell.isDefault()) - map[id] = String{string_ref}; - - total_length += string_ref.size + 1; - } - } - } - - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); - - query_count.fetch_add(rows, std::memory_order_relaxed); - hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); - - /// request new values - if (!outdated_ids.empty()) - { - std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); - - update( - required_ids, - [&](const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; - - map[id] = String{attribute_value}; - total_length += (attribute_value.size + 1) * outdated_ids[id].size(); - }, - [&](const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - total_length += get_default(row).size + 1; - }); - } - - out->getChars().reserve(total_length); - - for (const auto row : ext::range(0, ext::size(ids))) - { - const auto id = ids[row]; - const auto it = map.find(id); - - const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row); - out->insertData(string_ref.data, string_ref.size); - } - } - - template - void CacheDictionary::update( - const std::vector & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const - { - CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests}; - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size()); - - std::unordered_map remaining_ids{requested_ids.size()}; - for (const auto id : requested_ids) - remaining_ids.insert({id, 0}); - - const auto now = std::chrono::system_clock::now(); - - const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; - - if (now > backoff_end_time) - { - try - { - if (error_count) - { - /// Recover after error: we have to clone the source here because - /// it could keep connections which should be reset after error. - source_ptr = source_ptr->clone(); - } - - Stopwatch watch; - auto stream = source_ptr->loadIds(requested_ids); - stream->readPrefix(); - - while (const auto block = stream->read()) - { - const auto id_column = typeid_cast(block.safeGetByPosition(0).column.get()); - if (!id_column) - throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH}; - - const auto & ids = id_column->getData(); - - /// cache column pointers - const auto column_ptrs = ext::map( - ext::range(0, attributes.size()), [&block](size_t i) { return block.safeGetByPosition(i + 1).column.get(); }); - - for (const auto i : ext::range(0, ids.size())) - { - const auto id = ids[i]; - - const auto find_result = findCellIdx(id, now); - const auto & cell_idx = find_result.cell_idx; - - auto & cell = cells[cell_idx]; - - for (const auto attribute_idx : ext::range(0, attributes.size())) - { - const auto & attribute_column = *column_ptrs[attribute_idx]; - auto & attribute = attributes[attribute_idx]; - - setAttributeValue(attribute, cell_idx, attribute_column[i]); - } - - /// if cell id is zero and zero does not map to this cell, then the cell is unused - if (cell.id == 0 && cell_idx != zero_cell_idx) - element_count.fetch_add(1, std::memory_order_relaxed); - - cell.id = id; - if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) - { - std::uniform_int_distribution distribution{dict_lifetime.min_sec, dict_lifetime.max_sec}; - cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)}); - } - else - cell.setExpiresAt(std::chrono::time_point::max()); - - /// inform caller - on_cell_updated(id, cell_idx); - /// mark corresponding id as found - remaining_ids[id] = 1; - } - } - - stream->readSuffix(); - - error_count = 0; - last_exception = std::exception_ptr{}; - backoff_end_time = std::chrono::system_clock::time_point{}; - - ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed()); - } - catch (...) - { - ++error_count; - last_exception = std::current_exception(); - backoff_end_time = now + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count)); - - tryLogException(last_exception, log, "Could not update cache dictionary '" + getName() + - "', next update is scheduled at " + DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(backoff_end_time))); - } - } - - size_t not_found_num = 0, found_num = 0; - - /// Check which ids have not been found and require setting null_value - for (const auto & id_found_pair : remaining_ids) - { - if (id_found_pair.second) - { - ++found_num; - continue; - } - ++not_found_num; - - const auto id = id_found_pair.first; - - const auto find_result = findCellIdx(id, now); - const auto & cell_idx = find_result.cell_idx; - auto & cell = cells[cell_idx]; - - if (error_count) - { - if (find_result.outdated) - { - /// We have expired data for that `id` so we can continue using it. - bool was_default = cell.isDefault(); - cell.setExpiresAt(backoff_end_time); - if (was_default) - cell.setDefault(); - if (was_default) - on_id_not_found(id, cell_idx); - else - on_cell_updated(id, cell_idx); - continue; - } - /// We don't have expired data for that `id` so all we can do is to rethrow `last_exception`. - std::rethrow_exception(last_exception); - } - - /// Check if cell had not been occupied before and increment element counter if it hadn't - if (cell.id == 0 && cell_idx != zero_cell_idx) - element_count.fetch_add(1, std::memory_order_relaxed); - - cell.id = id; - - if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) - { - std::uniform_int_distribution distribution{dict_lifetime.min_sec, dict_lifetime.max_sec}; - cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)}); - } - else - cell.setExpiresAt(std::chrono::time_point::max()); - - /// Set null_value for each attribute - cell.setDefault(); - for (auto & attribute : attributes) - setDefaultAttributeValue(attribute, cell_idx); - - /// inform caller that the cell has not been found - on_id_not_found(id, cell_idx); - } - - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num); - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num); - ProfileEvents::increment(ProfileEvents::DictCacheRequests); - } - -} From 5f561af792eac397c5561c8c2b5087d4520c29f0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Thu, 2 Apr 2020 01:10:22 +0300 Subject: [PATCH 082/743] some fix for errors with shadowing name --- dbms/src/Dictionaries/DictionaryStructure.cpp | 4 ++-- dbms/src/Dictionaries/DictionaryStructure.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Dictionaries/DictionaryStructure.cpp b/dbms/src/Dictionaries/DictionaryStructure.cpp index 132b9effcb7..33196f627d8 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.cpp +++ b/dbms/src/Dictionaries/DictionaryStructure.cpp @@ -364,7 +364,7 @@ std::vector DictionaryStructure::getAttributes( void DictionaryStructure::getSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - Settings & settings) + Settings & dict_settings) { Poco::Util::AbstractConfiguration::Keys config_elems; config.keys(config_prefix, config_elems); @@ -379,7 +379,7 @@ void DictionaryStructure::getSettings( const auto prefix = config_prefix + '.' + config_elem; Poco::Util::AbstractConfiguration::Keys setting_keys; config.keys(prefix, setting_keys); - settings.loadSettingsFromConfig(prefix, config); + dict_settings.loadSettingsFromConfig(prefix, config); } } diff --git a/dbms/src/Dictionaries/DictionaryStructure.h b/dbms/src/Dictionaries/DictionaryStructure.h index 9a7428959a6..cd9d41f67ee 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.h +++ b/dbms/src/Dictionaries/DictionaryStructure.h @@ -124,7 +124,7 @@ private: void getSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - Settings & settings); + Settings & dict_settings); }; } From 3321f4490484f914ef0004975166e5c02587d40b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Apr 2020 01:41:29 +0300 Subject: [PATCH 083/743] add uuid to system.tables --- .../programs/server/ReplicasStatusHandler.cpp | 2 +- dbms/src/Databases/DatabaseAtomic.cpp | 21 ++++++++++++++++++ dbms/src/Databases/DatabaseAtomic.h | 3 +++ dbms/src/Databases/DatabaseDictionary.cpp | 19 ++++++++-------- dbms/src/Databases/DatabaseDictionary.h | 7 +++--- dbms/src/Databases/DatabaseFactory.cpp | 2 +- dbms/src/Databases/DatabaseLazy.cpp | 21 +++++++++--------- dbms/src/Databases/DatabaseLazy.h | 6 ++--- dbms/src/Databases/DatabaseMySQL.cpp | 4 ++-- dbms/src/Databases/DatabaseMySQL.h | 4 ++-- dbms/src/Databases/DatabaseOnDisk.cpp | 3 ++- dbms/src/Databases/DatabaseOnDisk.h | 1 + .../Databases/DatabaseWithDictionaries.cpp | 18 +++++++-------- dbms/src/Databases/DatabaseWithDictionaries.h | 6 ++--- dbms/src/Databases/DatabasesCommon.cpp | 2 +- dbms/src/Databases/DatabasesCommon.h | 2 +- dbms/src/Databases/IDatabase.h | 22 ++++++++++++++----- dbms/src/Interpreters/ActionLocksManager.cpp | 6 ++--- dbms/src/Interpreters/ActionLocksManager.h | 4 ---- dbms/src/Interpreters/AsynchronousMetrics.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 2 +- ...ExternalLoaderDatabaseConfigRepository.cpp | 2 +- .../src/Interpreters/InterpreterDropQuery.cpp | 4 ++-- .../Interpreters/InterpreterSystemQuery.cpp | 4 ++-- dbms/src/Storages/StorageMerge.cpp | 2 +- .../Storages/System/StorageSystemColumns.cpp | 2 +- .../Storages/System/StorageSystemGraphite.cpp | 8 +++---- .../System/StorageSystemMutations.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 2 +- .../Storages/System/StorageSystemReplicas.cpp | 2 +- .../System/StorageSystemReplicationQueue.cpp | 2 +- .../Storages/System/StorageSystemTables.cpp | 13 +++++++++-- .../src/TableFunctions/TableFunctionMerge.cpp | 6 ++--- 33 files changed, 124 insertions(+), 82 deletions(-) diff --git a/dbms/programs/server/ReplicasStatusHandler.cpp b/dbms/programs/server/ReplicasStatusHandler.cpp index e0943e2682b..2f2aa5953b6 100644 --- a/dbms/programs/server/ReplicasStatusHandler.cpp +++ b/dbms/programs/server/ReplicasStatusHandler.cpp @@ -44,7 +44,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) { auto & table = iterator->table(); StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); diff --git a/dbms/src/Databases/DatabaseAtomic.cpp b/dbms/src/Databases/DatabaseAtomic.cpp index a9da008e419..871eac48647 100644 --- a/dbms/src/Databases/DatabaseAtomic.cpp +++ b/dbms/src/Databases/DatabaseAtomic.cpp @@ -18,6 +18,15 @@ namespace ErrorCodes extern const int CANNOT_ASSIGN_ALTER; } +class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator +{ +public: + AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) + : DatabaseTablesSnapshotIterator(std::move(base)) {} + UUID uuid() const override { return table()->getStorageID().uuid; } +}; + + DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_) : DatabaseOrdinary(name_, metadata_path_, context_) { @@ -233,5 +242,17 @@ void DatabaseAtomic::cleenupDetachedTables() } } +DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const IDatabase::FilterByNameFunction & filter_by_table_name) +{ + auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(filter_by_table_name); + return std::make_unique(std::move(typeid_cast(*base_iter))); +} + +DatabaseTablesIteratorPtr DatabaseAtomic::getTablesWithDictionaryTablesIterator(const IDatabase::FilterByNameFunction & filter_by_dictionary_name) +{ + auto base_iter = DatabaseWithDictionaries::getTablesWithDictionaryTablesIterator(filter_by_dictionary_name); + return std::make_unique(std::move(typeid_cast(*base_iter))); +} + } diff --git a/dbms/src/Databases/DatabaseAtomic.h b/dbms/src/Databases/DatabaseAtomic.h index f02f4438f5d..ce2c2fd79ae 100644 --- a/dbms/src/Databases/DatabaseAtomic.h +++ b/dbms/src/Databases/DatabaseAtomic.h @@ -42,6 +42,9 @@ public: void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; void shutdown() override; + DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const FilterByNameFunction & filter_by_dictionary_name) override; + private: void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index 006eb1656a2..a1cf10f1c42 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -17,25 +17,26 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -DatabaseDictionary::DatabaseDictionary(const String & name_) - : IDatabase(name_), - log(&Logger::get("DatabaseDictionary(" + database_name + ")")) +DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & global_context_) + : IDatabase(name_) + , log(&Logger::get("DatabaseDictionary(" + database_name + ")")) + , global_context(global_context_.getGlobalContext()) { } -Tables DatabaseDictionary::listTables(const Context & context, const FilterByNameFunction & filter_by_name) +Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_name) { Tables tables; ExternalLoader::LoadResults load_results; if (filter_by_name) { /// If `filter_by_name` is set, we iterate through all dictionaries with such names. That's why we need to load all of them. - load_results = context.getExternalDictionariesLoader().tryLoad(filter_by_name); + load_results = global_context.getExternalDictionariesLoader().tryLoad(filter_by_name); } else { /// If `filter_by_name` isn't set, we iterate through only already loaded dictionaries. We don't try to load all dictionaries in this case. - load_results = context.getExternalDictionariesLoader().getCurrentLoadResults(); + load_results = global_context.getExternalDictionariesLoader().getCurrentLoadResults(); } for (const auto & load_result: load_results) @@ -47,7 +48,7 @@ Tables DatabaseDictionary::listTables(const Context & context, const FilterByNam auto dict_name = dict_ptr->getName(); const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - tables[dict_name] = StorageDictionary::create(StorageID(getDatabaseName(), dict_name), ColumnsDescription{columns}, context, true, dict_name); + tables[dict_name] = StorageDictionary::create(StorageID(getDatabaseName(), dict_name), ColumnsDescription{columns}, global_context, true, dict_name); } } return tables; @@ -75,9 +76,9 @@ StoragePtr DatabaseDictionary::tryGetTable( return {}; } -DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const FilterByNameFunction & filter_by_table_name) { - return std::make_unique(listTables(context, filter_by_table_name)); + return std::make_unique(listTables(filter_by_table_name)); } bool DatabaseDictionary::empty(const Context & context) const diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index b586cb1403f..d18d10ccb5e 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -22,7 +22,7 @@ namespace DB class DatabaseDictionary final : public IDatabase { public: - DatabaseDictionary(const String & name_); + DatabaseDictionary(const String & name_, const Context & global_context); String getEngineName() const override { @@ -37,7 +37,7 @@ public: const Context & context, const String & table_name) const override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; bool empty(const Context & context) const override; @@ -52,8 +52,9 @@ private: mutable std::mutex mutex; Poco::Logger * log; + const Context & global_context; - Tables listTables(const Context & context, const FilterByNameFunction & filter_by_name); + Tables listTables(const FilterByNameFunction & filter_by_name); }; } diff --git a/dbms/src/Databases/DatabaseFactory.cpp b/dbms/src/Databases/DatabaseFactory.cpp index 29cb24629d7..ebbf4710b04 100644 --- a/dbms/src/Databases/DatabaseFactory.cpp +++ b/dbms/src/Databases/DatabaseFactory.cpp @@ -70,7 +70,7 @@ DatabasePtr DatabaseFactory::getImpl( else if (engine_name == "Memory") return std::make_shared(database_name); else if (engine_name == "Dictionary") - return std::make_shared(database_name); + return std::make_shared(database_name, context); #if USE_MYSQL diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 683d315d012..1faa41c547e 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -115,7 +115,7 @@ bool DatabaseLazy::isTableExist( } StoragePtr DatabaseLazy::tryGetTable( - const Context & context, + const Context & /*context*/, const String & table_name) const { SCOPE_EXIT({ clearExpiredTables(); }); @@ -135,10 +135,10 @@ StoragePtr DatabaseLazy::tryGetTable( } } - return loadTable(context, table_name); + return loadTable(table_name); } -DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); Strings filtered_tables; @@ -148,7 +148,7 @@ DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context & contex filtered_tables.push_back(table_name); } std::sort(filtered_tables.begin(), filtered_tables.end()); - return std::make_unique(*this, context, std::move(filtered_tables)); + return std::make_unique(*this, std::move(filtered_tables)); } bool DatabaseLazy::empty(const Context & /* context */) const @@ -218,7 +218,7 @@ DatabaseLazy::~DatabaseLazy() } } -StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table_name) const +StoragePtr DatabaseLazy::loadTable(const String & table_name) const { SCOPE_EXIT({ clearExpiredTables(); }); @@ -229,9 +229,9 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table try { StoragePtr table; - Context context_copy(context); /// some tables can change context, but not LogTables + Context context_copy(global_context); /// some tables can change context, but not LogTables - auto ast = parseQueryFromMetadata(log, context, table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false); + auto ast = parseQueryFromMetadata(log, global_context, table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false); if (ast) { auto & ast_create = ast->as(); @@ -299,10 +299,9 @@ void DatabaseLazy::clearExpiredTables() const } -DatabaseLazyIterator::DatabaseLazyIterator(DatabaseLazy & database_, const Context & context_, Strings && table_names_) +DatabaseLazyIterator::DatabaseLazyIterator(DatabaseLazy & database_, Strings && table_names_) : database(database_) , table_names(std::move(table_names_)) - , context(context_) , iterator(table_names.begin()) , current_storage(nullptr) { @@ -312,7 +311,7 @@ void DatabaseLazyIterator::next() { current_storage.reset(); ++iterator; - while (isValid() && !database.isTableExist(context, *iterator)) + while (isValid() && !database.isTableExist(database.global_context, *iterator)) ++iterator; } @@ -329,7 +328,7 @@ const String & DatabaseLazyIterator::name() const const StoragePtr & DatabaseLazyIterator::table() const { if (!current_storage) - current_storage = database.tryGetTable(context, *iterator); + current_storage = database.tryGetTable(database.global_context, *iterator); return current_storage; } diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index 99bdfea11ae..153157ce17a 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -61,7 +61,7 @@ public: bool empty(const Context & context) const override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; @@ -104,7 +104,7 @@ private: mutable TablesCache tables_cache; mutable CacheExpirationQueue cache_expiration_queue; - StoragePtr loadTable(const Context & context, const String & table_name) const; + StoragePtr loadTable(const String & table_name) const; void clearExpiredTables() const; @@ -117,7 +117,6 @@ class DatabaseLazyIterator final : public IDatabaseTablesIterator public: DatabaseLazyIterator( DatabaseLazy & database_, - const Context & context_, Strings && table_names_); void next() override; @@ -128,7 +127,6 @@ public: private: const DatabaseLazy & database; const Strings table_names; - const Context context; Strings::const_iterator iterator; mutable StoragePtr current_storage; }; diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index dad42a6fd90..10d1189c031 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -62,7 +62,7 @@ DatabaseMySQL::DatabaseMySQL( const Context & global_context_, const String & database_name_, const String & metadata_path_, const ASTStorage * database_engine_define_, const String & database_name_in_mysql_, mysqlxx::Pool && pool) : IDatabase(database_name_) - , global_context(global_context_) + , global_context(global_context_.getGlobalContext()) , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) , database_name_in_mysql(database_name_in_mysql_) @@ -86,7 +86,7 @@ bool DatabaseMySQL::empty(const Context &) const return true; } -DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const FilterByNameFunction & filter_by_table_name) { Tables tables; std::lock_guard lock(mutex); diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index 5a459cb3095..6d7f1a29e55 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -30,7 +30,7 @@ public: bool empty(const Context & context) const override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; ASTPtr getCreateDatabaseQuery(const Context & /*context*/) const override; @@ -60,7 +60,7 @@ protected: ASTPtr getCreateTableQueryImpl(const Context & context, const String & name, bool throw_on_error) const override; private: - Context global_context; + const Context & global_context; String metadata_path; ASTPtr database_engine_define; String database_name_in_mysql; diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 2d13d6e13ab..a8a8e85d6ee 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -123,10 +123,11 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) return statement_stream.str(); } -DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context &) +DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context & context) : DatabaseWithOwnTablesBase(name, logger) , metadata_path(metadata_path_) , data_path("data/" + escapeForFileName(database_name) + "/") + , global_context(context.getGlobalContext()) { } diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index b7fa0b1b015..5c7e41343a4 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -91,6 +91,7 @@ protected: const String metadata_path; /*const*/ String data_path; + const Context & global_context; }; } diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index e849962aae3..cd1f69aef9b 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -173,17 +173,17 @@ StoragePtr DatabaseWithDictionaries::tryGetTable(const Context & context, const if (isDictionaryExist(context, table_name)) /// We don't need lock database here, because database doesn't store dictionary itself /// just metadata - return getDictionaryStorage(context, table_name); + return getDictionaryStorage(table_name); return {}; } DatabaseTablesIteratorPtr DatabaseWithDictionaries::getTablesWithDictionaryTablesIterator( - const Context & context, const FilterByNameFunction & filter_by_dictionary_name) + const FilterByNameFunction & filter_by_dictionary_name) { /// NOTE: it's not atomic - auto tables_it = getTablesIterator(context, filter_by_dictionary_name); - auto dictionaries_it = getDictionariesIterator(context, filter_by_dictionary_name); + auto tables_it = getTablesIterator(filter_by_dictionary_name); + auto dictionaries_it = getDictionariesIterator(filter_by_dictionary_name); Tables result; while (tables_it && tables_it->isValid()) @@ -195,7 +195,7 @@ DatabaseTablesIteratorPtr DatabaseWithDictionaries::getTablesWithDictionaryTable while (dictionaries_it && dictionaries_it->isValid()) { auto table_name = dictionaries_it->name(); - auto table_ptr = getDictionaryStorage(context, table_name); + auto table_ptr = getDictionaryStorage(table_name); if (table_ptr) result.emplace(table_name, table_ptr); dictionaries_it->next(); @@ -204,7 +204,7 @@ DatabaseTablesIteratorPtr DatabaseWithDictionaries::getTablesWithDictionaryTable return std::make_unique(result); } -DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_dictionary_name) +DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(const FilterByNameFunction & filter_by_dictionary_name) { std::lock_guard lock(mutex); if (!filter_by_dictionary_name) @@ -223,16 +223,16 @@ bool DatabaseWithDictionaries::isDictionaryExist(const Context & /*context*/, co return dictionaries.find(dictionary_name) != dictionaries.end(); } -StoragePtr DatabaseWithDictionaries::getDictionaryStorage(const Context & context, const String & table_name) const +StoragePtr DatabaseWithDictionaries::getDictionaryStorage(const String & table_name) const { auto dict_name = database_name + "." + table_name; - const auto & external_loader = context.getExternalDictionariesLoader(); + const auto & external_loader = global_context.getExternalDictionariesLoader(); auto dict_ptr = external_loader.tryGetDictionary(dict_name); if (dict_ptr) { const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - return StorageDictionary::create(StorageID(database_name, table_name), ColumnsDescription{columns}, context, true, dict_name); + return StorageDictionary::create(StorageID(database_name, table_name), ColumnsDescription{columns}, global_context, true, dict_name); } return nullptr; } diff --git a/dbms/src/Databases/DatabaseWithDictionaries.h b/dbms/src/Databases/DatabaseWithDictionaries.h index 407c564954b..4a3132a432a 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.h +++ b/dbms/src/Databases/DatabaseWithDictionaries.h @@ -20,9 +20,9 @@ public: StoragePtr tryGetTable(const Context & context, const String & table_name) const override; - DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name) override; + DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const FilterByNameFunction & filter_by_dictionary_name) override; - DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name) override; + DatabaseDictionariesIteratorPtr getDictionariesIterator(const FilterByNameFunction & filter_by_dictionary_name) override; bool isDictionaryExist(const Context & context, const String & dictionary_name) const override; @@ -37,7 +37,7 @@ protected: void attachToExternalDictionariesLoader(Context & context); void detachFromExternalDictionariesLoader(); - StoragePtr getDictionaryStorage(const Context & context, const String & table_name) const; + StoragePtr getDictionaryStorage(const String & table_name) const; ASTPtr getCreateDictionaryQueryImpl(const Context & context, const String & dictionary_name, diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 37c5b95dfb2..c1aea97e61c 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -42,7 +42,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable( return {}; } -DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); if (!filter_by_table_name) diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 7f699c55422..b07224fbc13 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -33,7 +33,7 @@ public: StoragePtr detachTable(const String & table_name) override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; void shutdown() override; diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 44ade7542bf..d9626fac934 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -42,15 +42,27 @@ public: virtual const StoragePtr & table() const = 0; virtual ~IDatabaseTablesIterator() = default; + + virtual UUID uuid() const { return UUIDHelpers::Nil; } }; /// Copies list of tables and iterates through such snapshot. -class DatabaseTablesSnapshotIterator final : public IDatabaseTablesIterator +class DatabaseTablesSnapshotIterator : public IDatabaseTablesIterator { private: Tables tables; Tables::iterator it; +protected: + DatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && other) + { + size_t idx = std::distance(other.tables.begin(), other.it); + std::swap(tables, other.tables); + other.it = other.tables.end(); + it = tables.begin(); + std::advance(it, idx); + } + public: DatabaseTablesSnapshotIterator(Tables & tables_) : tables(tables_), it(tables.begin()) {} @@ -133,18 +145,18 @@ public: /// Get an iterator that allows you to pass through all the tables. /// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above. - virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0; + virtual DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name = {}) = 0; /// Get an iterator to pass through all the dictionaries. - virtual DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & /*context*/, [[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {}) + virtual DatabaseDictionariesIteratorPtr getDictionariesIterator([[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {}) { return std::make_unique(); } /// Get an iterator to pass through all the tables and dictionary tables. - virtual DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name = {}) + virtual DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const FilterByNameFunction & filter_by_name = {}) { - return getTablesIterator(context, filter_by_name); + return getTablesIterator(filter_by_name); } /// Is the database empty. diff --git a/dbms/src/Interpreters/ActionLocksManager.cpp b/dbms/src/Interpreters/ActionLocksManager.cpp index 9ab220d0a84..fe5b3def658 100644 --- a/dbms/src/Interpreters/ActionLocksManager.cpp +++ b/dbms/src/Interpreters/ActionLocksManager.cpp @@ -20,17 +20,17 @@ namespace ActionLocks template -inline void forEachTable(Context & context, F && f) +inline void forEachTable(F && f) { for (auto & elem : DatabaseCatalog::instance().getDatabases()) - for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = elem.second->getTablesIterator(); iterator->isValid(); iterator->next()) f(iterator->table()); } void ActionLocksManager::add(StorageActionBlockType action_type) { - forEachTable(global_context, [&](const StoragePtr & table) { add(table, action_type); }); + forEachTable([&](const StoragePtr & table) { add(table, action_type); }); } void ActionLocksManager::add(const StorageID & table_id, StorageActionBlockType action_type) diff --git a/dbms/src/Interpreters/ActionLocksManager.h b/dbms/src/Interpreters/ActionLocksManager.h index 4d2f9bef256..039a95ce218 100644 --- a/dbms/src/Interpreters/ActionLocksManager.h +++ b/dbms/src/Interpreters/ActionLocksManager.h @@ -19,8 +19,6 @@ class Context; class ActionLocksManager { public: - explicit ActionLocksManager(Context & global_context_) : global_context(global_context_) {} - /// Adds new locks for each table void add(StorageActionBlockType action_type); /// Add new lock for a table if it has not been already added @@ -37,8 +35,6 @@ public: void cleanExpired(); private: - Context & global_context; - using StorageRawPtr = const IStorage *; using Locks = std::unordered_map; using StorageLocks = std::unordered_map; diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index 35a352b67b6..544b345c2bb 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -155,7 +155,7 @@ void AsynchronousMetrics::update() /// Lazy database can not contain MergeTree tables if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) { ++total_number_of_tables; auto & table = iterator->table(); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index bc769b780d2..ebf27e624da 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1957,7 +1957,7 @@ std::shared_ptr Context::getActionLocksManager() auto lock = getLock(); if (!shared->action_locks_manager) - shared->action_locks_manager = std::make_shared(getGlobalContext()); + shared->action_locks_manager = std::make_shared(); return shared->action_locks_manager; } diff --git a/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp b/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp index 10f99262da7..20d43b1b329 100644 --- a/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp +++ b/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp @@ -52,7 +52,7 @@ std::set ExternalLoaderDatabaseConfigRepository::getAllLoadablesDef { std::set result; const auto & dbname = database.getDatabaseName(); - auto itr = database.getDictionariesIterator(context); + auto itr = database.getDictionariesIterator(); while (itr && itr->isValid()) { result.insert(dbname + "." + itr->name()); diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index e6279de818c..da4e7f31d04 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -225,13 +225,13 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS ASTDropQuery query; query.kind = kind; query.database = database_name; - for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) { query.table = iterator->name(); executeToTable({query.database, query.table}, query); } - for (auto iterator = database->getDictionariesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getDictionariesIterator(); iterator->isValid(); iterator->next()) { String current_dictionary = iterator->name(); executeToDictionary(database_name, current_dictionary, kind, false, false, false); diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 87ed4a1f749..891786db84d 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -140,7 +140,7 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, auto access = context.getAccess(); for (auto & elem : DatabaseCatalog::instance().getDatabases()) { - for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = elem.second->getTablesIterator(); iterator->isValid(); iterator->next()) { if (!access->isGranted(log, getRequiredAccessType(action_type), elem.first, iterator->name())) continue; @@ -362,7 +362,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) for (auto & elem : DatabaseCatalog::instance().getDatabases()) { DatabasePtr & database = elem.second; - for (auto iterator = database->getTablesIterator(system_context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) { if (dynamic_cast(iterator->table().get())) replica_names.emplace_back(iterator->table()->getStorageID()); diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 657950c0bcb..d05f5137028 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -415,7 +415,7 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator() const checkStackSize(); auto database = DatabaseCatalog::instance().getDatabase(source_database); auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); }; - return database->getTablesIterator(global_context, table_name_match); + return database->getTablesIterator(table_name_match); } diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 9af8904ab26..b443cf7b6b5 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -302,7 +302,7 @@ Pipes StorageSystemColumns::read( const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getTablesWithDictionaryTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesWithDictionaryTablesIterator(); iterator->isValid(); iterator->next()) { const String & table_name = iterator->name(); storages.emplace(std::piecewise_construct, diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index a6163f89182..e8cca255c56 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -25,7 +25,7 @@ NamesAndTypesList StorageSystemGraphite::getNamesAndTypes() /* * Looking for (Replicated)*GraphiteMergeTree and get all configuration parameters for them */ -static StorageSystemGraphite::Configs getConfigs(const Context & context) +static StorageSystemGraphite::Configs getConfigs() { const Databases databases = DatabaseCatalog::instance().getDatabases(); StorageSystemGraphite::Configs graphite_configs; @@ -36,7 +36,7 @@ static StorageSystemGraphite::Configs getConfigs(const Context & context) if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) { auto & table = iterator->table(); @@ -71,9 +71,9 @@ static StorageSystemGraphite::Configs getConfigs(const Context & context) return graphite_configs; } -void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const +void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { - Configs graphite_configs = getConfigs(context); + Configs graphite_configs = getConfigs(); for (const auto & config : graphite_configs) { diff --git a/dbms/src/Storages/System/StorageSystemMutations.cpp b/dbms/src/Storages/System/StorageSystemMutations.cpp index e7d9cc38671..968c67bc14a 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.cpp +++ b/dbms/src/Storages/System/StorageSystemMutations.cpp @@ -50,7 +50,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index c212b30d268..46a33dcf0cf 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -110,7 +110,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) { String table_name = iterator->name(); StoragePtr storage = iterator->table(); diff --git a/dbms/src/Storages/System/StorageSystemReplicas.cpp b/dbms/src/Storages/System/StorageSystemReplicas.cpp index 251b45e44b6..734d0098d8d 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicas.cpp @@ -76,7 +76,7 @@ Pipes StorageSystemReplicas::read( if (db.second->getEngineName() == "Lazy") continue; const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp index 2c188cf3734..a6f0af8cc6c 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -60,7 +60,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 5d0aec921de..8f22df617cf 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB @@ -36,6 +37,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_) { {"database", std::make_shared()}, {"name", std::make_shared()}, + {"uuid", std::make_shared()}, {"engine", std::make_shared()}, {"is_temporary", std::make_shared()}, {"data_paths", std::make_shared(std::make_shared())}, @@ -74,7 +76,7 @@ static bool needLockStructure(const DatabasePtr & database, const Block & header if (database->getEngineName() != "Lazy") return true; - static const std::set columns_without_lock = { "database", "name", "metadata_modification_time" }; + static const std::set columns_without_lock = { "database", "name", "uuid", "metadata_modification_time" }; for (const auto & column : header.getColumnsWithTypeAndName()) { if (columns_without_lock.find(column.name) == columns_without_lock.end()) @@ -152,6 +154,10 @@ protected: if (columns_mask[src_index++]) res_columns[res_index++]->insert(table.first); + // uuid + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getStorageID().uuid); + // engine if (columns_mask[src_index++]) res_columns[res_index++]->insert(table.second->getName()); @@ -226,7 +232,7 @@ protected: const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); if (!tables_it || !tables_it->isValid()) - tables_it = database->getTablesWithDictionaryTablesIterator(context); + tables_it = database->getTablesWithDictionaryTablesIterator(); const bool need_lock_structure = needLockStructure(database, getPort().getHeader()); @@ -265,6 +271,9 @@ protected: if (columns_mask[src_index++]) res_columns[res_index++]->insert(table_name); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(tables_it->uuid()); + if (columns_mask[src_index++]) { assert(table != nullptr); diff --git a/dbms/src/TableFunctions/TableFunctionMerge.cpp b/dbms/src/TableFunctions/TableFunctionMerge.cpp index 1ced074761b..f0f5c7eff19 100644 --- a/dbms/src/TableFunctions/TableFunctionMerge.cpp +++ b/dbms/src/TableFunctions/TableFunctionMerge.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes } -static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_, const Context & context) +static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_) { OptimizedRegularExpression table_name_regexp(table_name_regexp_); auto table_name_match = [&](const String & table_name) { return table_name_regexp.match(table_name); }; @@ -32,7 +32,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str { auto database = DatabaseCatalog::instance().getDatabase(source_database); - auto iterator = database->getTablesIterator(context, table_name_match); + auto iterator = database->getTablesIterator(table_name_match); if (iterator->isValid()) any_table = iterator->table(); @@ -72,7 +72,7 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Co auto res = StorageMerge::create( StorageID(getDatabaseName(), table_name), - ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}, + ColumnsDescription{chooseColumns(source_database, table_name_regexp)}, source_database, table_name_regexp, context); From ab93987f7eda094104fe6a3751e099b9665f6e0b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Apr 2020 04:00:21 +0300 Subject: [PATCH 084/743] fix --- dbms/src/Interpreters/InterpreterSystemQuery.cpp | 5 +++-- .../0_stateless/01108_restart_replicas_rename_deadlock.sh | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index cc1f014c434..78a0d92a567 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -340,16 +340,17 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context); auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints); + auto data_path = database->getTableDataPath(create); table = StorageFactory::instance().get(create, - database->getTableDataPath(create), + data_path, system_context, system_context.getGlobalContext(), columns, constraints, false); - database->createTable(system_context, replica.table_name, table, create_ast); + database->attachTable(replica.table_name, table, data_path); table->startup(); return table; diff --git a/dbms/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh b/dbms/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh index aac5b637d2b..01c9ef0a277 100755 --- a/dbms/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh +++ b/dbms/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh @@ -67,6 +67,7 @@ $CLICKHOUSE_CLIENT -q "SELECT sum(n), count(n) FROM merge(currentDatabase(), '^r for i in `seq 4`; do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS replica_01108_$i" - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS replica_01108_${i}_tmp" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS replica_01108_$i NO DELAY" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS replica_01108_${i}_tmp NO DELAY" done +sleep 2 From 338c18ae3bb3d50695cd2b941286ea0e3aa647d7 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 2 Apr 2020 13:02:51 +0300 Subject: [PATCH 085/743] review fixes --- dbms/src/Interpreters/ActionsVisitor.cpp | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 30f3fc754b4..bb12664c8d5 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -200,14 +200,10 @@ static String getUniqueName(ActionsVisitor::Data & data, const String & prefix) auto & block = data.getSampleBlock(); auto result = prefix; - if (block.has(result)) + while (block.has(result)) { - do - { - result = prefix + "_" + toString(data.next_unique_suffix); - ++data.next_unique_suffix; - } - while (block.has(result)); + result = prefix + "_" + toString(data.next_unique_suffix); + ++data.next_unique_suffix; } return result; From 20fc52f02be78b774927302a0139c73667d50ff3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Apr 2020 17:33:30 +0300 Subject: [PATCH 086/743] Added GraphiteRollupSortedTransform. --- .../Merges/GraphiteRollupSortedTransform.cpp | 385 ++++++++++++++++++ .../Merges/GraphiteRollupSortedTransform.h | 275 +++++++++++++ 2 files changed, 660 insertions(+) create mode 100644 dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp create mode 100644 dbms/src/Processors/Merges/GraphiteRollupSortedTransform.h diff --git a/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp new file mode 100644 index 00000000000..64654ecc73f --- /dev/null +++ b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp @@ -0,0 +1,385 @@ +#include + +namespace DB +{ + +static GraphiteRollupSortedTransform::ColumnsDefinition defineColumns( + const Block & header, const Graphite::Params & params) +{ + GraphiteRollupSortedTransform::ColumnsDefinition def; + + def.path_column_num = header.getPositionByName(params.path_column_name); + def.time_column_num = header.getPositionByName(params.time_column_name); + def.value_column_num = header.getPositionByName(params.value_column_name); + def.version_column_num = header.getPositionByName(params.version_column_name); + + size_t num_columns = header.columns(); + for (size_t i = 0; i < num_columns; ++i) + if (i != def.time_column_num && i != def.value_column_num && i != def.version_column_num) + def.unmodified_column_numbers.push_back(i); + + return def; +} + +GraphiteRollupSortedTransform::GraphiteRollupSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description_, size_t max_block_size, + Graphite::Params params_, time_t time_of_merge_) + : IMergingTransform(num_inputs, header, header, true) + , merged_data(header.cloneEmptyColumns(), false, max_block_size) + , description(std::move(description_)) + , source_chunks(num_inputs) + , cursors(num_inputs) + , params(std::move(params_)), time_of_merge(time_of_merge_) + , chunk_allocator(num_inputs + max_row_refs) +{ + size_t max_size_of_aggregate_state = 0; + size_t max_alignment_of_aggregate_state = 1; + + for (const auto & pattern : params.patterns) + { + if (pattern.function) + { + max_size_of_aggregate_state = std::max(max_size_of_aggregate_state, pattern.function->sizeOfData()); + max_alignment_of_aggregate_state = std::max(max_alignment_of_aggregate_state, pattern.function->alignOfData()); + } + } + + merged_data.allocMemForAggregates(max_size_of_aggregate_state, max_alignment_of_aggregate_state); + columns_definition = defineColumns(header, params); +} + +Graphite::RollupRule GraphiteRollupSortedTransform::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 GraphiteRollupSortedTransform::selectPrecision(const Graphite::Retentions & retentions, time_t time) const +{ + static_assert(is_signed_v, "time_t must be signed type"); + + for (const auto & retention : retentions) + { + if (time_of_merge - time >= static_cast(retention.age)) + return retention.precision; + } + + /// No rounding. + return 1; +} + +/** Round the unix timestamp to seconds precision. + * In this case, the date should not change. The date is calculated using the local time zone. + * + * If the rounding value is less than an hour, + * then, assuming that time zones that differ from UTC by a non-integer number of hours are not supported, + * just simply round the unix timestamp down to a multiple of 3600. + * And if the rounding value is greater, + * then we will round down the number of seconds from the beginning of the day in the local time zone. + * + * Rounding to more than a day is not supported. + */ +static time_t roundTimeToPrecision(const DateLUTImpl & date_lut, time_t time, UInt32 precision) +{ + if (precision <= 3600) + { + return time / precision * precision; + } + else + { + time_t date = date_lut.toDate(time); + time_t remainder = time - date; + return date + remainder / precision * precision; + } +} + +void GraphiteRollupSortedTransform::initializeInputs() +{ + queue = SortingHeap(cursors); + is_queue_initialized = true; +} + +void GraphiteRollupSortedTransform::consume(Chunk chunk, size_t input_number) +{ + updateCursor(std::move(chunk), input_number); + + if (is_queue_initialized) + queue.push(cursors[input_number]); +} + +void GraphiteRollupSortedTransform::updateCursor(Chunk chunk, size_t source_num) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); + + auto & source_chunk = source_chunks[source_num]; + + if (source_chunk) + { + source_chunk = chunk_allocator.alloc(std::move(chunk)); + cursors[source_num].reset(source_chunk->getColumns(), {}); + } + else + { + if (cursors[source_num].has_collation) + throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); + + source_chunk = chunk_allocator.alloc(std::move(chunk)); + cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); + } + + source_chunk->all_columns = cursors[source_num].all_columns; + source_chunk->sort_columns = cursors[source_num].sort_columns; +} + +void GraphiteRollupSortedTransform::work() +{ + merge(); + prepareOutputChunk(merged_data); +} + +void GraphiteRollupSortedTransform::merge() +{ + const DateLUTImpl & date_lut = DateLUT::instance(); + + /// Take rows in needed order and put them into `merged_data` until we get `max_block_size` rows. + /// + /// Variables starting with current_* refer to the rows previously popped from the queue that will + /// contribute towards current output row. + /// Variables starting with next_* refer to the row at the top of the queue. + + while (queue.isValid()) + { + SortCursor current = queue.current(); + + StringRef next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->pos); + bool new_path = is_first || next_path != current_group_path; + + is_first = false; + + time_t next_row_time = current->all_columns[columns_definition.time_column_num]->getUInt(current->pos); + /// Is new key before rounding. + bool is_new_key = new_path || next_row_time != current_time; + + if (is_new_key) + { + /// Accumulate the row that has maximum version in the previous group of rows with the same key: + if (merged_data.wasGroupStarted()) + accumulateRow(current_subgroup_newest_row); + + Graphite::RollupRule next_rule = merged_data.currentRule(); + if (new_path) + next_rule = selectPatternForPath(next_path); + + const Graphite::RetentionPattern * retention_pattern = std::get<0>(next_rule); + time_t next_time_rounded; + if (retention_pattern) + { + UInt32 precision = selectPrecision(retention_pattern->retentions, next_row_time); + next_time_rounded = roundTimeToPrecision(date_lut, next_row_time, precision); + } + else + { + /// If no pattern has matched - take the value as-is. + next_time_rounded = next_row_time; + } + + /// Key will be new after rounding. It means new result row. + bool will_be_new_key = new_path || next_time_rounded != current_time_rounded; + + if (will_be_new_key) + { + if (merged_data.wasGroupStarted()) + { + finishCurrentGroup(); + + /// We have enough rows - return, but don't advance the loop. At the beginning of the + /// next call to merge() the same next_cursor will be processed once more and + /// the next output row will be created from it. + if (merged_data.hasEnoughRows()) + return; + } + + /// At this point previous row has been fully processed, so we can advance the loop + /// (substitute current_* values for next_*, advance the cursor). + + startNextGroup(current, next_rule); + + current_time_rounded = next_time_rounded; + } + + current_time = next_row_time; + } + + /// Within all rows with same key, we should leave only one row with maximum version; + /// and for rows with same maximum version - only last row. + if (is_new_key + || current->all_columns[columns_definition.version_column_num]->compareAt( + current->pos, current_subgroup_newest_row.row_num, + *(*current_subgroup_newest_row.all_columns)[columns_definition.version_column_num], + /* nan_direction_hint = */ 1) >= 0) + { + current_subgroup_newest_row.set(current, source_chunks[current.impl->order]); + + /// Small hack: group and subgroups have the same path, so we can set current_group_path here instead of startNextGroup + /// But since we keep in memory current_subgroup_newest_row's block, we could use StringRef for current_group_path and don't + /// make deep copy of the path. + current_group_path = next_path; + } + + if (!current->isLast()) + { + queue.next(); + } + else + { + /// We get the next block from the appropriate source, if there is one. + queue.removeTop(); + requestDataForInput(current.impl->order); + return; + } + } + + /// Write result row for the last group. + if (merged_data.wasGroupStarted()) + { + accumulateRow(current_subgroup_newest_row); + finishCurrentGroup(); + } + + is_finished = true; +} + +void GraphiteRollupSortedTransform::startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule) +{ + merged_data.startNextGroup(cursor->all_columns, cursor->pos, next_rule, columns_definition); +} + +void GraphiteRollupSortedTransform::finishCurrentGroup() +{ + merged_data.insertRow(current_time_rounded, current_subgroup_newest_row, columns_definition); +} + +void GraphiteRollupSortedTransform::accumulateRow(RowRef & row) +{ + merged_data.accumulateRow(row, columns_definition); +} + +void GraphiteRollupSortedTransform::GraphiteRollupMergedData::startNextGroup( + const ColumnRawPtrs & raw_columns, size_t row, + Graphite::RollupRule next_rule, ColumnsDefinition & def) +{ + const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(next_rule); + + /// Copy unmodified column values (including path column). + for (size_t j : def.unmodified_column_numbers) + columns[j]->insertFrom(*raw_columns[j], row); + + if (aggregation_pattern) + { + aggregation_pattern->function->create(place_for_aggregate_state.data()); + aggregate_state_created = true; + } + + current_rule = next_rule; + was_group_started = true; +} + +void GraphiteRollupSortedTransform::GraphiteRollupMergedData::insertRow( + time_t time, RowRef & row, ColumnsDefinition & def) +{ + /// Insert calculated values of the columns `time`, `value`, `version`. + columns[def.time_column_num]->insert(time); + auto & row_ref_version_column = (*row.all_columns)[def.version_column_num]; + columns[def.version_column_num]->insertFrom(*row_ref_version_column, row.row_num); + + auto & value_column = columns[def.value_column_num]; + const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule); + if (aggregate_state_created) + { + aggregation_pattern->function->insertResultInto(place_for_aggregate_state.data(), *value_column); + aggregation_pattern->function->destroy(place_for_aggregate_state.data()); + aggregate_state_created = false; + } + else + value_column->insertFrom(*(*row.all_columns)[def.value_column_num], row.row_num); + + ++total_merged_rows; + ++merged_rows; + /// TODO: sum_blocks_granularity += block_size; + + was_group_started = false; +} + +void GraphiteRollupSortedTransform::GraphiteRollupMergedData::accumulateRow(RowRef & row, ColumnsDefinition & def) +{ + const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule); + if (aggregate_state_created) + { + auto & column = (*row.all_columns)[def.value_column_num]; + aggregation_pattern->function->add(place_for_aggregate_state.data(), &column, row.row_num, nullptr); + } +} + +GraphiteRollupSortedTransform::GraphiteRollupMergedData::~GraphiteRollupMergedData() +{ + if (aggregate_state_created) + std::get<1>(current_rule)->function->destroy(place_for_aggregate_state.data()); +} + +} diff --git a/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.h b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.h new file mode 100644 index 00000000000..dd0ba6d9493 --- /dev/null +++ b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -0,0 +1,275 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +/** Intended for implementation of "rollup" - aggregation (rounding) of older data + * for a table with Graphite data (Graphite is the system for time series monitoring). + * + * Table with graphite data has at least the following columns (accurate to the name): + * Path, Time, Value, Version + * + * Path - name of metric (sensor); + * Time - time of measurement; + * Value - value of measurement; + * Version - a number, that for equal pairs of Path and Time, need to leave only record with maximum version. + * + * Each row in a table correspond to one value of one sensor. + * + * Pattern should contain function, retention scheme, or both of them. The order of patterns does mean as well: + * * Aggregation OR retention patterns should be first + * * Then aggregation AND retention full patterns have to be placed + * * default pattern without regexp must be the last + * + * Rollup rules are specified in the following way: + * + * pattern + * regexp + * function + * pattern + * regexp + * age -> precision + * age -> precision + * ... + * pattern + * regexp + * function + * age -> precision + * age -> precision + * ... + * pattern + * ... + * default + * function + * age -> precision + * ... + * + * regexp - pattern for sensor name + * default - if no pattern has matched + * + * age - minimal data age (in seconds), to start rounding with specified precision. + * precision - rounding precision (in seconds) + * + * function - name of aggregate function to be applied for values, that time was rounded to same. + * + * Example: + * + * + * + * \.max$ + * max + * + * + * click_cost + * any + * + * 0 + * 5 + * + * + * 86400 + * 60 + * + * + * + * max + * + * 0 + * 60 + * + * + * 3600 + * 300 + * + * + * 86400 + * 3600 + * + * + * + */ +namespace Graphite +{ + struct Retention + { + UInt32 age; + UInt32 precision; + }; + + using Retentions = std::vector; + + struct Pattern + { + std::shared_ptr regexp; + std::string regexp_str; + AggregateFunctionPtr function; + Retentions retentions; /// Must be ordered by 'age' descending. + enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically + }; + + using Patterns = std::vector; + using RetentionPattern = Pattern; + using AggregationPattern = Pattern; + + struct Params + { + String config_name; + String path_column_name; + String time_column_name; + String value_column_name; + String version_column_name; + Graphite::Patterns patterns; + }; + + using RollupRule = std::pair; +} + +/** Merges several sorted ports into one. + * + * For each group of consecutive identical values of the `path` column, + * and the same `time` values, rounded to some precision + * (where rounding accuracy depends on the template set for `path` + * and the amount of time elapsed from `time` to the specified time), + * keeps one line, + * performing the rounding of time, + * merge `value` values using the specified aggregate functions, + * as well as keeping the maximum value of the `version` column. + */ +class GraphiteRollupSortedTransform : public IMergingTransform +{ +public: + GraphiteRollupSortedTransform( + size_t num_inputs, const Block & header, + SortDescription description_, size_t max_block_size, + Graphite::Params params_, time_t time_of_merge_); + + String getName() const override { return "GraphiteRollupSortedTransform"; } + void work() override; + + struct ColumnsDefinition + { + size_t path_column_num; + size_t time_column_num; + size_t value_column_num; + size_t version_column_num; + + /// All columns other than 'time', 'value', 'version'. They are unmodified during rollup. + ColumnNumbers unmodified_column_numbers; + }; + + using RowRef = detail::RowRefWithOwnedChunk; + + /// Specialization for SummingSortedTransform. + class GraphiteRollupMergedData : public MergedData + { + public: + using MergedData::MergedData; + ~GraphiteRollupMergedData(); + + void startNextGroup(const ColumnRawPtrs & raw_columns, size_t row, + Graphite::RollupRule next_rule, ColumnsDefinition & def); + void insertRow(time_t time, RowRef & row, ColumnsDefinition & def); + void accumulateRow(RowRef & row, ColumnsDefinition & def); + bool wasGroupStarted() const { return was_group_started; } + + const Graphite::RollupRule & currentRule() const { return current_rule; } + void allocMemForAggregates(size_t size, size_t alignment) { place_for_aggregate_state.reset(size, alignment); } + + private: + Graphite::RollupRule current_rule = {nullptr, nullptr}; + AlignedBuffer place_for_aggregate_state; + bool aggregate_state_created = false; /// Invariant: if true then current_rule is not NULL. + bool was_group_started = false; + }; + +protected: + void initializeInputs() override; + void consume(Chunk chunk, size_t input_number) override; + +private: + Logger * log = &Logger::get("GraphiteRollupSortedBlockInputStream"); + + GraphiteRollupMergedData merged_data; + SortDescription description; + + /// Chunks currently being merged. + using SourceChunks = std::vector; + SourceChunks source_chunks; + SortCursorImpls cursors; + + SortingHeap queue; + bool is_queue_initialized = false; + + const Graphite::Params params; + ColumnsDefinition columns_definition; + + time_t time_of_merge; + + /// No data has been read. + bool is_first = true; + + /* | path | time | rounded_time | version | value | unmodified | + * ----------------------------------------------------------------------------------- + * | A | 11 | 10 | 1 | 1 | a | | + * | A | 11 | 10 | 3 | 2 | b |> subgroup(A, 11) | + * | A | 11 | 10 | 2 | 3 | c | |> group(A, 10) + * ----------------------------------------------------------------------------------|> + * | A | 12 | 10 | 0 | 4 | d | |> Outputs (A, 10, avg(2, 5), a) + * | A | 12 | 10 | 1 | 5 | e |> subgroup(A, 12) | + * ----------------------------------------------------------------------------------- + * | A | 21 | 20 | 1 | 6 | f | + * | B | 11 | 10 | 1 | 7 | g | + * ... + */ + + /// Path name of current bucket + StringRef current_group_path; + + static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row. + /// Last row with maximum version for current primary key (time bucket). + RowRef current_subgroup_newest_row; + + detail::SharedChunkAllocator chunk_allocator; + + /// Time of last read row + 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. + void startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule); + + /// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows. + void finishCurrentGroup(); + + /// Update the state of the aggregate function with the new `value`. + void accumulateRow(RowRef & row); + + void merge(); + void updateCursor(Chunk chunk, size_t source_num); +}; + +} From 2daef385af8862a29c02181ae8ca01d700660408 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Apr 2020 19:28:50 +0300 Subject: [PATCH 087/743] Use SortedTransforms in merge. --- .../Merges/AggregatingSortedTransform.h | 2 +- .../Merges/GraphiteRollupSortedTransform.cpp | 2 +- .../Merges/GraphiteRollupSortedTransform.h | 2 +- .../Merges/ReplacingSortedTransform.cpp | 2 +- .../Merges/ReplacingSortedTransform.h | 2 +- .../Merges/SummingSortedTransform.cpp | 2 +- .../Merges/SummingSortedTransform.h | 2 +- .../Merges/VersionedCollapsingTransform.cpp | 2 +- .../Merges/VersionedCollapsingTransform.h | 10 +-- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 81 +++++++++++-------- ...ream.cpp => MergeTreeSequentialSource.cpp} | 72 ++++++----------- ...utStream.h => MergeTreeSequentialSource.h} | 27 +++---- 13 files changed, 95 insertions(+), 113 deletions(-) rename dbms/src/Storages/MergeTree/{MergeTreeSequentialBlockInputStream.cpp => MergeTreeSequentialSource.cpp} (65%) rename dbms/src/Storages/MergeTree/{MergeTreeSequentialBlockInputStream.h => MergeTreeSequentialSource.h} (73%) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.h b/dbms/src/Processors/Merges/AggregatingSortedTransform.h index 613ac0baa58..5ebc41d6ccf 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.h +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.h @@ -19,7 +19,7 @@ class AggregatingSortedTransform : public IMergingTransform { public: AggregatingSortedTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size); struct SimpleAggregateDescription; diff --git a/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp index 64654ecc73f..acec6587f2c 100644 --- a/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp +++ b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp @@ -22,7 +22,7 @@ static GraphiteRollupSortedTransform::ColumnsDefinition defineColumns( } GraphiteRollupSortedTransform::GraphiteRollupSortedTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size, Graphite::Params params_, time_t time_of_merge_) : IMergingTransform(num_inputs, header, header, true) diff --git a/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.h b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.h index dd0ba6d9493..4dd394198ad 100644 --- a/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -151,7 +151,7 @@ class GraphiteRollupSortedTransform : public IMergingTransform { public: GraphiteRollupSortedTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size, Graphite::Params params_, time_t time_of_merge_); diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp index e39b33a5a46..d45e6f04aa6 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.cpp @@ -10,7 +10,7 @@ namespace ErrorCodes } ReplacingSortedTransform::ReplacingSortedTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, const String & version_column, size_t max_block_size, WriteBuffer * out_row_sources_buf_, diff --git a/dbms/src/Processors/Merges/ReplacingSortedTransform.h b/dbms/src/Processors/Merges/ReplacingSortedTransform.h index d28bd239cfe..ff447ee8da6 100644 --- a/dbms/src/Processors/Merges/ReplacingSortedTransform.h +++ b/dbms/src/Processors/Merges/ReplacingSortedTransform.h @@ -17,7 +17,7 @@ class ReplacingSortedTransform final : public IMergingTransform { public: ReplacingSortedTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, const String & version_column, size_t max_block_size, WriteBuffer * out_row_sources_buf_ = nullptr, diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index 99008025232..b9359d2543f 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -366,7 +366,7 @@ namespace } SummingSortedTransform::SummingSortedTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. const Names & column_names_to_sum, diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.h b/dbms/src/Processors/Merges/SummingSortedTransform.h index e7915cd3c8c..390f1113985 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.h +++ b/dbms/src/Processors/Merges/SummingSortedTransform.h @@ -19,7 +19,7 @@ class SummingSortedTransform final : public IMergingTransform public: SummingSortedTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. const Names & column_names_to_sum, diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp index 8b8b2bfa063..1e5043bb5b9 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp @@ -8,7 +8,7 @@ namespace DB static const size_t MAX_ROWS_IN_MULTIVERSION_QUEUE = 8192; VersionedCollapsingTransform::VersionedCollapsingTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, const String & sign_column_, size_t max_block_size, WriteBuffer * out_row_sources_buf_, diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h index 0dbdf8e2a40..04a18142072 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.h @@ -20,11 +20,11 @@ class VersionedCollapsingTransform final : public IMergingTransform public: /// Don't need version column. It's in primary key. VersionedCollapsingTransform( - size_t num_inputs, const Block & header, - SortDescription description_, const String & sign_column_, - size_t max_block_size, - WriteBuffer * out_row_sources_buf_ = nullptr, - bool use_average_block_sizes = false); + const Block & header, size_t num_inputs, + SortDescription description_, const String & sign_column_, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_ = nullptr, + bool use_average_block_sizes = false); String getName() const override { return "VersionedCollapsingTransform"; } void work() override; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 2279618c9a0..129ecaea27e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -30,7 +30,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 0c8c39b074c..b5356fbce88 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1,6 +1,6 @@ #include "MergeTreeDataMergerMutator.h" -#include +#include #include #include #include @@ -12,16 +12,19 @@ #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 @@ -660,7 +663,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. */ - BlockInputStreams src_streams; + Pipes pipes; UInt64 watch_prev_elapsed = 0; /// We count total amount of bytes in parts @@ -687,18 +690,24 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor for (const auto & part : parts) { - auto input = std::make_unique( + auto input = std::make_unique( data, part, merging_column_names, read_with_direct_io, true); input->setProgressCallback( MergeProgressCallback(merge_entry, watch_prev_elapsed, horizontal_stage_progress)); - BlockInputStreamPtr stream = std::move(input); - if (data.hasPrimaryKey() || data.hasSkipIndices()) - stream = std::make_shared( - std::make_shared(stream, data.sorting_key_and_skip_indices_expr)); + Pipe pipe(std::move(input)); - src_streams.emplace_back(stream); + if (data.hasPrimaryKey() || data.hasSkipIndices()) + { + auto expr = std::make_shared(pipe.getHeader(), data.sorting_key_and_skip_indices_expr); + pipe.addSimpleTransform(std::move(expr)); + + auto materializing = std::make_shared(pipe.getHeader()); + pipe.addSimpleTransform(std::move(materializing)); + } + + pipes.emplace_back(std::move(pipe)); } Names sort_columns = data.sorting_key_columns; @@ -706,14 +715,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); - Block header = src_streams.at(0)->getHeader(); + Block header = pipes.at(0).getHeader(); for (size_t i = 0; i < sort_columns_size; ++i) sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); /// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number. /// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part, /// that is going in insertion order. - std::shared_ptr merged_stream; + ProcessorPtr merged_transform; /// If merge is vertical we cannot calculate it bool blocks_are_granules_size = (merge_alg == MergeAlgorithm::Vertical); @@ -722,45 +731,48 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor switch (data.merging_params.mode) { case MergeTreeData::MergingParams::Ordinary: - merged_stream = std::make_unique( - src_streams, sort_description, merge_block_size, 0, rows_sources_write_buf.get(), true, blocks_are_granules_size); + merged_transform = std::make_unique( + header, pipes.size(), sort_description, merge_block_size, 0, rows_sources_write_buf.get(), true, blocks_are_granules_size); break; case MergeTreeData::MergingParams::Collapsing: - merged_stream = std::make_unique( - src_streams, sort_description, data.merging_params.sign_column, + merged_transform = std::make_unique( + header, pipes.size(), sort_description, data.merging_params.sign_column, merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size); break; case MergeTreeData::MergingParams::Summing: - merged_stream = std::make_unique( - src_streams, sort_description, data.merging_params.columns_to_sum, merge_block_size); + merged_transform = std::make_unique( + header, pipes.size(), sort_description, data.merging_params.columns_to_sum, merge_block_size); break; case MergeTreeData::MergingParams::Aggregating: - merged_stream = std::make_unique( - src_streams, sort_description, merge_block_size); + merged_transform = std::make_unique( + header, pipes.size(), sort_description, merge_block_size); break; case MergeTreeData::MergingParams::Replacing: - merged_stream = std::make_unique( - src_streams, sort_description, data.merging_params.version_column, + merged_transform = std::make_unique( + header, pipes.size(), sort_description, data.merging_params.version_column, merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size); break; case MergeTreeData::MergingParams::Graphite: - merged_stream = std::make_unique( - src_streams, sort_description, merge_block_size, + merged_transform = std::make_unique( + header, pipes.size(), sort_description, merge_block_size, data.merging_params.graphite_params, time_of_merge); break; case MergeTreeData::MergingParams::VersionedCollapsing: - merged_stream = std::make_unique( - src_streams, sort_description, data.merging_params.sign_column, + merged_transform = std::make_unique( + header, pipes.size(), sort_description, data.merging_params.sign_column, merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size); break; } + Pipe merged_pipe(std::move(pipes), std::move(merged_transform)); + BlockInputStreamPtr merged_stream = std::make_shared(std::move(merged_pipe)); + if (deduplicate) merged_stream = std::make_shared(merged_stream, SizeLimits(), 0 /*limit_hint*/, Names()); @@ -857,13 +869,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor MergeStageProgress column_progress(progress_before, column_sizes->columnWeight(column_name)); for (size_t part_num = 0; part_num < parts.size(); ++part_num) { - auto column_part_stream = std::make_shared( + auto column_part_source = std::make_shared( data, parts[part_num], column_names, read_with_direct_io, true); - column_part_stream->setProgressCallback( + column_part_source->setProgressCallback( MergeProgressCallback(merge_entry, watch_prev_elapsed, column_progress)); - column_part_streams[part_num] = std::move(column_part_stream); + column_part_streams[part_num] = std::make_shared( + Pipe(std::move(column_part_source))); } rows_sources_read_buf.seek(0, 0); diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialSource.cpp similarity index 65% rename from dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp rename to dbms/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 9e0737810df..b2ac3d995f9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB @@ -8,16 +8,17 @@ namespace ErrorCodes extern const int MEMORY_LIMIT_EXCEEDED; } -MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( +MergeTreeSequentialSource::MergeTreeSequentialSource( const MergeTreeData & storage_, - const MergeTreeData::DataPartPtr & data_part_, + MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, bool read_with_direct_io_, bool take_column_types_from_storage, bool quiet) - : storage(storage_) - , data_part(data_part_) - , columns_to_read(columns_to_read_) + : SourceWithProgress(storage_.getSampleBlockForColumns(columns_to_read)) + , storage(storage_) + , data_part(std::move(data_part_)) + , columns_to_read(std::move(columns_to_read_)) , read_with_direct_io(read_with_direct_io_) , mark_cache(storage.global_context.getMarkCache()) { @@ -33,8 +34,6 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( addTotalRowsApprox(data_part->rows_count); - header = storage.getSampleBlockForColumns(columns_to_read); - /// Add columns because we don't want to read empty blocks injectRequiredColumns(storage, data_part, columns_to_read); NamesAndTypesList columns_for_reader; @@ -62,33 +61,11 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( /* uncompressed_cache = */ nullptr, mark_cache.get(), reader_settings); } - -void MergeTreeSequentialBlockInputStream::fixHeader(Block & header_block) const -{ - /// Types may be different during ALTER (when this stream is used to perform an ALTER). - for (const auto & name_type : data_part->getColumns()) - { - if (header_block.has(name_type.name)) - { - auto & elem = header_block.getByName(name_type.name); - if (!elem.type->equals(*name_type.type)) - { - elem.type = name_type.type; - elem.column = elem.type->createColumn(); - } - } - } -} - -Block MergeTreeSequentialBlockInputStream::getHeader() const -{ - return header; -} - -Block MergeTreeSequentialBlockInputStream::readImpl() +Chunk MergeTreeSequentialSource::generate() try { - Block res; + auto & header = getPort().getHeader(); + if (!isCancelled() && current_row < data_part->rows_count) { size_t rows_to_read = data_part->index_granularity.getMarkRows(current_mark); @@ -96,15 +73,15 @@ try auto & sample = reader->getColumns(); Columns columns(sample.size()); - size_t rows_readed = reader->readRows(current_mark, continue_reading, rows_to_read, columns); + size_t rows_read = reader->readRows(current_mark, continue_reading, rows_to_read, columns); - if (rows_readed) + if (rows_read) { - current_row += rows_readed; - current_mark += (rows_to_read == rows_readed); + current_row += rows_read; + current_mark += (rows_to_read == rows_read); bool should_evaluate_missing_defaults = false; - reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_readed); + reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read); if (should_evaluate_missing_defaults) { @@ -113,20 +90,21 @@ try reader->performRequiredConversions(columns); - res = header.cloneEmpty(); - /// Reorder columns and fill result block. size_t num_columns = sample.size(); + Columns res_columns; + res_columns.reserve(num_columns); + auto it = sample.begin(); for (size_t i = 0; i < num_columns; ++i) { - if (res.has(it->name)) - res.getByName(it->name).column = std::move(columns[i]); + if (header.has(it->name)) + res_columns.emplace_back(std::move(columns[i])); ++it; } - res.checkNumberOfRows(); + return Chunk(std::move(res_columns), rows_read); } } else @@ -134,7 +112,7 @@ try finish(); } - return res; + return {}; } catch (...) { @@ -144,8 +122,7 @@ catch (...) throw; } - -void MergeTreeSequentialBlockInputStream::finish() +void MergeTreeSequentialSource::finish() { /** Close the files (before destroying the object). * When many sources are created, but simultaneously reading only a few of them, @@ -155,7 +132,6 @@ void MergeTreeSequentialBlockInputStream::finish() data_part.reset(); } - -MergeTreeSequentialBlockInputStream::~MergeTreeSequentialBlockInputStream() = default; +MergeTreeSequentialSource::~MergeTreeSequentialSource() = default; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeSequentialSource.h similarity index 73% rename from dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h rename to dbms/src/Storages/MergeTree/MergeTreeSequentialSource.h index fd57a39fd7a..dac559913aa 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include #include @@ -9,40 +9,33 @@ namespace DB { /// Lightweight (in terms of logic) stream for reading single part from MergeTree -class MergeTreeSequentialBlockInputStream : public IBlockInputStream +class MergeTreeSequentialSource : public SourceWithProgress { public: - MergeTreeSequentialBlockInputStream( + MergeTreeSequentialSource( const MergeTreeData & storage_, - const MergeTreeData::DataPartPtr & data_part_, + MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, bool read_with_direct_io_, bool take_column_types_from_storage, bool quiet = false ); - ~MergeTreeSequentialBlockInputStream() override; + ~MergeTreeSequentialSource() override; - String getName() const override { return "MergeTreeSequentialBlockInputStream"; } - - Block getHeader() const override; - - /// Closes readers and unlock part locks - void finish(); + String getName() const override { return "MergeTreeSequentialSource"; } size_t getCurrentMark() const { return current_mark; } size_t getCurrentRow() const { return current_row; } protected: - Block readImpl() override; + Chunk generate() override; private: const MergeTreeData & storage; - Block header; - /// Data part will not be removed if the pointer owns it MergeTreeData::DataPartPtr data_part; @@ -52,7 +45,7 @@ private: /// Should read using direct IO bool read_with_direct_io; - Logger * log = &Logger::get("MergeTreeSequentialBlockInputStream"); + Logger * log = &Logger::get("MergeTreeSequentialSource"); std::shared_ptr mark_cache; using MergeTreeReaderPtr = std::unique_ptr; @@ -65,8 +58,8 @@ private: size_t current_row = 0; private: - void fixHeader(Block & header_block) const; - + /// Closes readers and unlock part locks + void finish(); }; } From 4510b25df2e982492306fd6909c583fe9e86d3b2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Apr 2020 19:39:35 +0300 Subject: [PATCH 088/743] Try fix build. --- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index fbc42de5517..0a9869adfd9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include From 87a456839b392637a0ee9e1c5a97f24201732851 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Apr 2020 19:40:50 +0300 Subject: [PATCH 089/743] Try fix build. --- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 0a9869adfd9..38055a6b67b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include From dd40c7dc726d12adaf2036947a71f4c770f503e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Apr 2020 20:02:08 +0300 Subject: [PATCH 090/743] Try fix build. --- dbms/src/Processors/Merges/AggregatingSortedTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 1c94bca4e3a..2aebd029f0a 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -66,7 +66,7 @@ namespace } AggregatingSortedTransform::AggregatingSortedTransform( - size_t num_inputs, const Block & header, + const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size) : IMergingTransform(num_inputs, header, header, true) , columns_definition(defineColumns(header, description_)) From d16a435531ed740d08275c149d68e3a34d744bb1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Apr 2020 21:33:23 +0300 Subject: [PATCH 091/743] Try fix tests. --- dbms/src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index b2ac3d995f9..19bc6ddf790 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -15,7 +15,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( bool read_with_direct_io_, bool take_column_types_from_storage, bool quiet) - : SourceWithProgress(storage_.getSampleBlockForColumns(columns_to_read)) + : SourceWithProgress(storage_.getSampleBlockForColumns(columns_to_read_)) , storage(storage_) , data_part(std::move(data_part_)) , columns_to_read(std::move(columns_to_read_)) From e50ebd5ee12a5bc226ac08aef0822feb1f0ec27a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Fri, 3 Apr 2020 01:35:22 +0300 Subject: [PATCH 092/743] added support of custom settings to FileDictionarySource --- dbms/Dictionaries/DictionarySourceFactory.cpp | 7 ++-- dbms/Dictionaries/DictionaryStructure.cpp | 32 +------------------ dbms/Dictionaries/FileDictionarySource.cpp | 15 +++++++-- dbms/Dictionaries/FileDictionarySource.h | 8 ++--- 4 files changed, 22 insertions(+), 40 deletions(-) diff --git a/dbms/Dictionaries/DictionarySourceFactory.cpp b/dbms/Dictionaries/DictionarySourceFactory.cpp index fa3b3017ad2..8431e065dd4 100644 --- a/dbms/Dictionaries/DictionarySourceFactory.cpp +++ b/dbms/Dictionaries/DictionarySourceFactory.cpp @@ -84,10 +84,11 @@ DictionarySourcePtr DictionarySourceFactory::create( { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); - if (keys.size() != 1) - throw Exception{name + ": element dictionary.source should have exactly one child element", + + if (keys.empty() || keys.size() > 2) + throw Exception{name + ": element dictionary.source should have one or two child elements", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; - + const auto & source_type = keys.front(); const auto found = registered_sources.find(source_type); diff --git a/dbms/Dictionaries/DictionaryStructure.cpp b/dbms/Dictionaries/DictionaryStructure.cpp index 33196f627d8..fe4b1474e84 100644 --- a/dbms/Dictionaries/DictionaryStructure.cpp +++ b/dbms/Dictionaries/DictionaryStructure.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include @@ -196,9 +195,6 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration attributes = getAttributes(config, config_prefix); - settings = Settings(); - getSettings(config, config_prefix, settings); - if (attributes.empty()) throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS}; } @@ -360,30 +356,4 @@ std::vector DictionaryStructure::getAttributes( return res_attributes; } - -void DictionaryStructure::getSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - Settings & dict_settings) -{ - Poco::Util::AbstractConfiguration::Keys config_elems; - config.keys(config_prefix, config_elems); - - - for (const auto & config_elem : config_elems) - { - if (startsWith(config_elem, "settings")) - { - /* i won't do break after this if in case there can be multiple settings sections */ - - const auto prefix = config_prefix + '.' + config_elem; - Poco::Util::AbstractConfiguration::Keys setting_keys; - config.keys(prefix, setting_keys); - dict_settings.loadSettingsFromConfig(prefix, config); - - } - } - -} - -} +} \ No newline at end of file diff --git a/dbms/Dictionaries/FileDictionarySource.cpp b/dbms/Dictionaries/FileDictionarySource.cpp index 5e7a784183c..7871b4a2a20 100644 --- a/dbms/Dictionaries/FileDictionarySource.cpp +++ b/dbms/Dictionaries/FileDictionarySource.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes FileDictionarySource::FileDictionarySource( const std::string & filepath_, const std::string & format_, - Block & sample_block_, const Context & context_, bool check_config) + Block & sample_block_, Context & context_, bool check_config) : filepath{filepath_} , format{format_} , sample_block{sample_block_} @@ -83,7 +83,18 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const auto filepath = config.getString(config_prefix + ".file.path"); const auto format = config.getString(config_prefix + ".file.format"); - return std::make_unique(filepath, format, sample_block, context, check_config); + Context context_local_copy(context); + if (config.has(config_prefix + ".settings")) + { + const auto prefix = config_prefix + ".settings"; + Settings settings; + + settings.loadSettingsFromConfig(prefix, config); + // const_cast(context).setSettings(settings); + context_local_copy.setSettings(settings); + } + + return std::make_unique(filepath, format, sample_block, context_local_copy, check_config); }; factory.registerSource("file", create_table_source); diff --git a/dbms/Dictionaries/FileDictionarySource.h b/dbms/Dictionaries/FileDictionarySource.h index 3d00c026e07..e22906633db 100644 --- a/dbms/Dictionaries/FileDictionarySource.h +++ b/dbms/Dictionaries/FileDictionarySource.h @@ -3,7 +3,7 @@ #include #include "IDictionarySource.h" #include - +#include namespace DB { @@ -11,14 +11,14 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } -class Context; +// class Context; /// Allows loading dictionaries from a file with given format, does not support "random access" class FileDictionarySource final : public IDictionarySource { public: FileDictionarySource(const std::string & filepath_, const std::string & format_, - Block & sample_block_, const Context & context_, bool check_config); + Block & sample_block_, Context & context_, bool check_config); FileDictionarySource(const FileDictionarySource & other); @@ -62,7 +62,7 @@ private: const std::string filepath; const std::string format; Block sample_block; - const Context & context; + const Context context; Poco::Timestamp last_modification; }; From 9f7d1f992025618e22692d38ca06b0824e222f01 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Apr 2020 12:30:20 +0300 Subject: [PATCH 093/743] Fix style. --- dbms/src/Processors/Merges/AggregatingSortedTransform.cpp | 5 +++++ dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp | 5 +++++ dbms/src/Processors/Merges/SummingSortedTransform.cpp | 2 -- dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp | 5 +++++ 4 files changed, 15 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 2aebd029f0a..3de4e7773f0 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { AggregatingSortedTransform::ColumnsDefinition defineColumns( diff --git a/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp index acec6587f2c..5ada52790a3 100644 --- a/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp +++ b/dbms/src/Processors/Merges/GraphiteRollupSortedTransform.cpp @@ -3,6 +3,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static GraphiteRollupSortedTransform::ColumnsDefinition defineColumns( const Block & header, const Graphite::Params & params) { diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index b9359d2543f..af97999a4b9 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -623,6 +623,4 @@ void SummingSortedTransform::merge() is_finished = true; } - - } diff --git a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp index 1e5043bb5b9..c11cf46a6ca 100644 --- a/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp +++ b/dbms/src/Processors/Merges/VersionedCollapsingTransform.cpp @@ -5,6 +5,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static const size_t MAX_ROWS_IN_MULTIVERSION_QUEUE = 8192; VersionedCollapsingTransform::VersionedCollapsingTransform( From a3534046d1fd2dcc01f60d0f51b48f7dc1bbbdc4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Apr 2020 14:12:44 +0300 Subject: [PATCH 094/743] Try fix summing. --- dbms/src/Processors/Merges/SummingSortedTransform.h | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.h b/dbms/src/Processors/Merges/SummingSortedTransform.h index 390f1113985..2e07ae12115 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.h +++ b/dbms/src/Processors/Merges/SummingSortedTransform.h @@ -54,8 +54,12 @@ public: void insertRow(const Row & row, const ColumnNumbers & column_numbers) { - for (auto column_number :column_numbers) - columns[column_number]->insert(row[column_number]); + size_t next_column = columns.size() - column_numbers.size(); + for (auto column_number : column_numbers) + { + columns[next_column]->insert(row[column_number]); + ++next_column; + } ++total_merged_rows; ++merged_rows; From 704a94f022f50f5e6aedea9ed9581cd411df150b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Apr 2020 14:57:24 +0300 Subject: [PATCH 095/743] Try fix summing. --- .../Merges/AggregatingSortedTransform.cpp | 21 ++++++++---------- .../Merges/SummingSortedTransform.cpp | 22 ++++++++----------- 2 files changed, 18 insertions(+), 25 deletions(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 3de4e7773f0..239d8b95cdf 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -168,19 +168,13 @@ void AggregatingSortedTransform::merge() bool has_previous_group = !last_key.empty(); SortCursor current = queue.current(); + detail::RowRef current_key; + current_key.set(current); - { - detail::RowRef current_key; - current_key.set(current); - - if (!has_previous_group) /// The first key encountered. - key_differs = true; - else - key_differs = !last_key.hasEqualSortColumnsWith(current_key); - - last_key = current_key; - last_chunk_sort_columns.clear(); - } + if (!has_previous_group) /// The first key encountered. + key_differs = true; + else + key_differs = !last_key.hasEqualSortColumnsWith(current_key); if (key_differs) { @@ -189,6 +183,7 @@ void AggregatingSortedTransform::merge() { /// Write the simple aggregation result for the previous group. insertSimpleAggregationResult(); + last_key.reset(); return; } @@ -216,6 +211,8 @@ void AggregatingSortedTransform::merge() if (!current->isLast()) { + last_key = current_key; + last_chunk_sort_columns.clear(); queue.next(); } else diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index af97999a4b9..efdd7f85472 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -540,22 +540,16 @@ void SummingSortedTransform::merge() bool has_previous_group = !last_key.empty(); SortCursor current = queue.current(); + detail::RowRef current_key; + current_key.set(current); + if (!has_previous_group) /// The first key encountered. { - detail::RowRef current_key; - current_key.set(current); - - if (!has_previous_group) /// The first key encountered. - { - key_differs = true; - current_row_is_zero = true; - } - else - key_differs = !last_key.hasEqualSortColumnsWith(current_key); - - last_key = current_key; - last_chunk_sort_columns.clear(); + key_differs = true; + current_row_is_zero = true; } + else + key_differs = !last_key.hasEqualSortColumnsWith(current_key); if (key_differs) { @@ -605,6 +599,8 @@ void SummingSortedTransform::merge() if (!current->isLast()) { + last_key = current_key; + last_chunk_sort_columns.clear(); queue.next(); } else From 0fe5528842d9946b2b2c200f7a6127d5140cde74 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Apr 2020 15:15:08 +0300 Subject: [PATCH 096/743] Try fix summing. --- .../Merges/AggregatingSortedTransform.cpp | 21 ++++++++++-------- .../Merges/SummingSortedTransform.cpp | 22 +++++++++++-------- 2 files changed, 25 insertions(+), 18 deletions(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 239d8b95cdf..3de4e7773f0 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -168,13 +168,19 @@ void AggregatingSortedTransform::merge() bool has_previous_group = !last_key.empty(); SortCursor current = queue.current(); - detail::RowRef current_key; - current_key.set(current); - if (!has_previous_group) /// The first key encountered. - key_differs = true; - else - key_differs = !last_key.hasEqualSortColumnsWith(current_key); + { + detail::RowRef current_key; + current_key.set(current); + + if (!has_previous_group) /// The first key encountered. + key_differs = true; + else + key_differs = !last_key.hasEqualSortColumnsWith(current_key); + + last_key = current_key; + last_chunk_sort_columns.clear(); + } if (key_differs) { @@ -183,7 +189,6 @@ void AggregatingSortedTransform::merge() { /// Write the simple aggregation result for the previous group. insertSimpleAggregationResult(); - last_key.reset(); return; } @@ -211,8 +216,6 @@ void AggregatingSortedTransform::merge() if (!current->isLast()) { - last_key = current_key; - last_chunk_sort_columns.clear(); queue.next(); } else diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index efdd7f85472..af97999a4b9 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -540,16 +540,22 @@ void SummingSortedTransform::merge() bool has_previous_group = !last_key.empty(); SortCursor current = queue.current(); - detail::RowRef current_key; - current_key.set(current); - if (!has_previous_group) /// The first key encountered. { - key_differs = true; - current_row_is_zero = true; + detail::RowRef current_key; + current_key.set(current); + + if (!has_previous_group) /// The first key encountered. + { + key_differs = true; + current_row_is_zero = true; + } + else + key_differs = !last_key.hasEqualSortColumnsWith(current_key); + + last_key = current_key; + last_chunk_sort_columns.clear(); } - else - key_differs = !last_key.hasEqualSortColumnsWith(current_key); if (key_differs) { @@ -599,8 +605,6 @@ void SummingSortedTransform::merge() if (!current->isLast()) { - last_key = current_key; - last_chunk_sort_columns.clear(); queue.next(); } else From 47eccf233289eb7f1a4c00904733f8716acc65b0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Apr 2020 15:32:12 +0300 Subject: [PATCH 097/743] Try fix summing. --- dbms/src/Processors/Merges/RowRef.h | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/dbms/src/Processors/Merges/RowRef.h b/dbms/src/Processors/Merges/RowRef.h index ac4be79f560..48ce92dbdbd 100644 --- a/dbms/src/Processors/Merges/RowRef.h +++ b/dbms/src/Processors/Merges/RowRef.h @@ -105,7 +105,8 @@ inline void intrusive_ptr_release(SharedChunk * ptr) /// This class represents a row in a chunk. struct RowRef { - ColumnRawPtrs * sort_columns = nullptr; /// Point to sort_columns from SortCursor or last_chunk_sort_columns. + const IColumn ** sort_columns = nullptr; /// Point to sort_columns from SortCursor or last_chunk_sort_columns. + size_t num_columns = 0; UInt64 row_num = 0; bool empty() const { return sort_columns == nullptr; } @@ -113,19 +114,19 @@ struct RowRef void set(SortCursor & cursor) { - sort_columns = &cursor.impl->sort_columns; + sort_columns = cursor.impl->sort_columns.data(); + num_columns = cursor.impl->sort_columns.size(); row_num = cursor.impl->pos; } - static bool checkEquals(const ColumnRawPtrs * left, size_t left_row, const ColumnRawPtrs * right, size_t right_row) + static bool checkEquals(size_t size, const IColumn ** lhs, size_t lhs_row, const IColumn ** rhs, size_t rhs_row) { - auto size = left->size(); for (size_t col_number = 0; col_number < size; ++col_number) { - auto & cur_column = (*left)[col_number]; - auto & other_column = (*right)[col_number]; + auto & cur_column = lhs[col_number]; + auto & other_column = rhs[col_number]; - if (0 != cur_column->compareAt(left_row, right_row, *other_column, 1)) + if (0 != cur_column->compareAt(lhs_row, rhs_row, *other_column, 1)) return false; } @@ -134,7 +135,7 @@ struct RowRef bool hasEqualSortColumnsWith(const RowRef & other) { - return checkEquals(sort_columns, row_num, other.sort_columns, other.row_num); + return checkEquals(num_columns, sort_columns, row_num, other.sort_columns, other.row_num); } }; @@ -178,7 +179,8 @@ struct RowRefWithOwnedChunk bool hasEqualSortColumnsWith(const RowRefWithOwnedChunk & other) { - return RowRef::checkEquals(sort_columns, row_num, other.sort_columns, other.row_num); + return RowRef::checkEquals(sort_columns->size(), sort_columns->data(), row_num, + other.sort_columns->data(), other.row_num); } }; From c53b902d6c34001539cb43359f6ce119fe0aa3d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Apr 2020 17:09:01 +0300 Subject: [PATCH 098/743] Try fix summing. --- dbms/src/Processors/Merges/AggregatingSortedTransform.cpp | 2 +- dbms/src/Processors/Merges/SummingSortedTransform.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 3de4e7773f0..b3a1be9f253 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -117,7 +117,7 @@ void AggregatingSortedTransform::updateCursor(Chunk chunk, size_t source_num) { /// Extend lifetime of last chunk. last_chunk = std::move(source_chunk); - last_chunk_sort_columns = std::move(cursors[source_num].all_columns); + last_chunk_sort_columns = std::move(cursors[source_num].sort_columns); source_chunk = std::move(chunk); cursors[source_num].reset(source_chunk.getColumns(), {}); diff --git a/dbms/src/Processors/Merges/SummingSortedTransform.cpp b/dbms/src/Processors/Merges/SummingSortedTransform.cpp index af97999a4b9..f34a3b479d7 100644 --- a/dbms/src/Processors/Merges/SummingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/SummingSortedTransform.cpp @@ -411,7 +411,7 @@ void SummingSortedTransform::updateCursor(Chunk chunk, size_t source_num) { /// Extend lifetime of last chunk. last_chunk = std::move(source_chunk); - last_chunk_sort_columns = std::move(cursors[source_num].all_columns); + last_chunk_sort_columns = std::move(cursors[source_num].sort_columns); source_chunk = std::move(chunk); cursors[source_num].reset(source_chunk.getColumns(), {}); From 4c3ab3d28cbbdc101b6dfe1ed0a3db97b8e228d2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 3 Apr 2020 18:05:49 +0300 Subject: [PATCH 099/743] resolve table uuid before query execution --- dbms/Interpreters/AddDefaultDatabaseVisitor.h | 15 ++++++++++++--- dbms/Interpreters/InterpreterAlterQuery.cpp | 8 +++++++- dbms/Interpreters/InterpreterFactory.cpp | 12 ++++++++++++ 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/dbms/Interpreters/AddDefaultDatabaseVisitor.h b/dbms/Interpreters/AddDefaultDatabaseVisitor.h index 8ca22cb94a9..c9d4d6dfbd4 100644 --- a/dbms/Interpreters/AddDefaultDatabaseVisitor.h +++ b/dbms/Interpreters/AddDefaultDatabaseVisitor.h @@ -23,10 +23,13 @@ namespace DB class AddDefaultDatabaseVisitor { public: - AddDefaultDatabaseVisitor(const String & database_name_, std::ostream * ostr_ = nullptr) + using IDResolver = std::function; + + AddDefaultDatabaseVisitor(const String & database_name_, std::ostream * ostr_ = nullptr, const IDResolver & resolver_ = {}) : database_name(database_name_), visit_depth(0), - ostr(ostr_) + ostr(ostr_), + resolver(resolver_) {} void visitDDL(ASTPtr & ast) const @@ -62,6 +65,7 @@ private: const String database_name; mutable size_t visit_depth; std::ostream * ostr; + IDResolver resolver; void visit(ASTSelectWithUnionQuery & select, ASTPtr &) const { @@ -101,7 +105,12 @@ private: void visit(const ASTIdentifier & identifier, ASTPtr & ast) const { if (!identifier.compound()) - ast = createTableIdentifier(database_name, identifier.name); + { + if (resolver) + ast = resolver(identifier); + else + ast = createTableIdentifier(database_name, identifier.name); + } } void visit(ASTSubquery & subquery, ASTPtr &) const diff --git a/dbms/Interpreters/InterpreterAlterQuery.cpp b/dbms/Interpreters/InterpreterAlterQuery.cpp index 315527765ef..20df0391747 100644 --- a/dbms/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/Interpreters/InterpreterAlterQuery.cpp @@ -46,7 +46,13 @@ BlockIO InterpreterAlterQuery::execute() /// Add default database to table identifiers that we can encounter in e.g. default expressions, /// mutation expression, etc. - AddDefaultDatabaseVisitor visitor(table_id.getDatabaseName()); + auto resolver = [&](const ASTIdentifier & table_name) + { + assert(table_name.uuid == UUIDHelpers::Nil); + auto resolved_id = context.resolveStorageID(table_name, Context::ResolveOrdinary); + return createTableIdentifier(DatabaseCatalog::instance().getTable(resolved_id)->getStorageID()); + }; + AddDefaultDatabaseVisitor visitor(table_id.getDatabaseName(), nullptr, resolver); ASTPtr command_list_ptr = alter.command_list->ptr(); visitor.visit(command_list_ptr); diff --git a/dbms/Interpreters/InterpreterFactory.cpp b/dbms/Interpreters/InterpreterFactory.cpp index 0c34d6ed79f..58403d4686b 100644 --- a/dbms/Interpreters/InterpreterFactory.cpp +++ b/dbms/Interpreters/InterpreterFactory.cpp @@ -66,6 +66,7 @@ #include #include +#include namespace ProfileEvents @@ -88,6 +89,13 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { ProfileEvents::increment(ProfileEvents::Query); + auto table_resolver = [&](const ASTIdentifier & table_name) + { + assert(table_name.uuid == UUIDHelpers::Nil); + auto resolved_id = context.resolveStorageID(table_name); + return createTableIdentifier(DatabaseCatalog::instance().getTable(resolved_id)->getStorageID()); + }; + if (query->as()) { /// This is internal part of ASTSelectWithUnionQuery. @@ -97,11 +105,15 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & else if (query->as()) { ProfileEvents::increment(ProfileEvents::SelectQuery); + AddDefaultDatabaseVisitor visitor("", nullptr, table_resolver); + visitor.visit(query); return std::make_unique(query, context, SelectQueryOptions(stage)); } else if (query->as()) { ProfileEvents::increment(ProfileEvents::InsertQuery); + AddDefaultDatabaseVisitor visitor("", nullptr, table_resolver); + visitor.visit(query); bool allow_materialized = static_cast(context.getSettingsRef().insert_allow_materialized_columns); return std::make_unique(query, context, allow_materialized); } From e0cc0d7b73bd7f877b5180ba3cdaa3e39c0d772d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 3 Apr 2020 23:55:29 +0300 Subject: [PATCH 100/743] Revert "resolve table uuid before query execution" This reverts commit 4c3ab3d28cbbdc101b6dfe1ed0a3db97b8e228d2. --- dbms/Interpreters/AddDefaultDatabaseVisitor.h | 15 +++------------ dbms/Interpreters/InterpreterAlterQuery.cpp | 8 +------- dbms/Interpreters/InterpreterFactory.cpp | 12 ------------ 3 files changed, 4 insertions(+), 31 deletions(-) diff --git a/dbms/Interpreters/AddDefaultDatabaseVisitor.h b/dbms/Interpreters/AddDefaultDatabaseVisitor.h index c9d4d6dfbd4..8ca22cb94a9 100644 --- a/dbms/Interpreters/AddDefaultDatabaseVisitor.h +++ b/dbms/Interpreters/AddDefaultDatabaseVisitor.h @@ -23,13 +23,10 @@ namespace DB class AddDefaultDatabaseVisitor { public: - using IDResolver = std::function; - - AddDefaultDatabaseVisitor(const String & database_name_, std::ostream * ostr_ = nullptr, const IDResolver & resolver_ = {}) + AddDefaultDatabaseVisitor(const String & database_name_, std::ostream * ostr_ = nullptr) : database_name(database_name_), visit_depth(0), - ostr(ostr_), - resolver(resolver_) + ostr(ostr_) {} void visitDDL(ASTPtr & ast) const @@ -65,7 +62,6 @@ private: const String database_name; mutable size_t visit_depth; std::ostream * ostr; - IDResolver resolver; void visit(ASTSelectWithUnionQuery & select, ASTPtr &) const { @@ -105,12 +101,7 @@ private: void visit(const ASTIdentifier & identifier, ASTPtr & ast) const { if (!identifier.compound()) - { - if (resolver) - ast = resolver(identifier); - else - ast = createTableIdentifier(database_name, identifier.name); - } + ast = createTableIdentifier(database_name, identifier.name); } void visit(ASTSubquery & subquery, ASTPtr &) const diff --git a/dbms/Interpreters/InterpreterAlterQuery.cpp b/dbms/Interpreters/InterpreterAlterQuery.cpp index 20df0391747..315527765ef 100644 --- a/dbms/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/Interpreters/InterpreterAlterQuery.cpp @@ -46,13 +46,7 @@ BlockIO InterpreterAlterQuery::execute() /// Add default database to table identifiers that we can encounter in e.g. default expressions, /// mutation expression, etc. - auto resolver = [&](const ASTIdentifier & table_name) - { - assert(table_name.uuid == UUIDHelpers::Nil); - auto resolved_id = context.resolveStorageID(table_name, Context::ResolveOrdinary); - return createTableIdentifier(DatabaseCatalog::instance().getTable(resolved_id)->getStorageID()); - }; - AddDefaultDatabaseVisitor visitor(table_id.getDatabaseName(), nullptr, resolver); + AddDefaultDatabaseVisitor visitor(table_id.getDatabaseName()); ASTPtr command_list_ptr = alter.command_list->ptr(); visitor.visit(command_list_ptr); diff --git a/dbms/Interpreters/InterpreterFactory.cpp b/dbms/Interpreters/InterpreterFactory.cpp index 58403d4686b..0c34d6ed79f 100644 --- a/dbms/Interpreters/InterpreterFactory.cpp +++ b/dbms/Interpreters/InterpreterFactory.cpp @@ -66,7 +66,6 @@ #include #include -#include namespace ProfileEvents @@ -89,13 +88,6 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { ProfileEvents::increment(ProfileEvents::Query); - auto table_resolver = [&](const ASTIdentifier & table_name) - { - assert(table_name.uuid == UUIDHelpers::Nil); - auto resolved_id = context.resolveStorageID(table_name); - return createTableIdentifier(DatabaseCatalog::instance().getTable(resolved_id)->getStorageID()); - }; - if (query->as()) { /// This is internal part of ASTSelectWithUnionQuery. @@ -105,15 +97,11 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & else if (query->as()) { ProfileEvents::increment(ProfileEvents::SelectQuery); - AddDefaultDatabaseVisitor visitor("", nullptr, table_resolver); - visitor.visit(query); return std::make_unique(query, context, SelectQueryOptions(stage)); } else if (query->as()) { ProfileEvents::increment(ProfileEvents::InsertQuery); - AddDefaultDatabaseVisitor visitor("", nullptr, table_resolver); - visitor.visit(query); bool allow_materialized = static_cast(context.getSettingsRef().insert_allow_materialized_columns); return std::make_unique(query, context, allow_materialized); } From 64a4640e0ec6590b969435906a1cdce6d62417a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Sat, 4 Apr 2020 00:32:06 +0300 Subject: [PATCH 101/743] added support of custom settings to all sources that needed having them --- .../ClickHouseDictionarySource.cpp | 7 ++++-- dbms/Dictionaries/DictionarySourceHelpers.cpp | 22 ++++++++++++++++++- dbms/Dictionaries/DictionarySourceHelpers.h | 12 +++++++++- .../ExecutableDictionarySource.cpp | 6 +++-- .../Dictionaries/ExecutableDictionarySource.h | 4 ++-- dbms/Dictionaries/FileDictionarySource.cpp | 14 +++--------- dbms/Dictionaries/FileDictionarySource.h | 2 +- dbms/Dictionaries/HTTPDictionarySource.cpp | 4 +++- dbms/Dictionaries/HTTPDictionarySource.h | 3 ++- dbms/Dictionaries/XDBCDictionarySource.cpp | 8 +++++++ 10 files changed, 60 insertions(+), 22 deletions(-) diff --git a/dbms/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/Dictionaries/ClickHouseDictionarySource.cpp index 0894a655724..45895ae93b2 100644 --- a/dbms/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/Dictionaries/ClickHouseDictionarySource.cpp @@ -12,7 +12,7 @@ #include "readInvalidateQuery.h" #include "writeParenthesisedString.h" #include "DictionaryFactory.h" - +#include "DictionarySourceHelpers.h" namespace DB { @@ -216,7 +216,10 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const Context & context, bool /* check_config */) -> DictionarySourcePtr { - return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context); + Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + /// Note that processors are not supported yet (see constructor), + /// hence it is not possible to override experimental_use_processors setting + return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context_local_copy); }; factory.registerSource("clickhouse", create_table_source); } diff --git a/dbms/Dictionaries/DictionarySourceHelpers.cpp b/dbms/Dictionaries/DictionarySourceHelpers.cpp index d01c7560832..7dfa67b5167 100644 --- a/dbms/Dictionaries/DictionarySourceHelpers.cpp +++ b/dbms/Dictionaries/DictionarySourceHelpers.cpp @@ -6,7 +6,9 @@ #include #include #include "DictionaryStructure.h" - +#include +#include +#include namespace DB { @@ -50,4 +52,22 @@ void formatKeys( out->flush(); } +Context copyContextAndApplySettings( + const std::string & config_prefix, + const Context & context, + const Poco::Util::AbstractConfiguration & config) +{ + Context local_context(context); + if (config.has(config_prefix + ".settings")) + { + const auto prefix = config_prefix + ".settings"; + Settings settings; + + settings.loadSettingsFromConfig(prefix, config); + local_context.setSettings(settings); + } + + return local_context; +} + } diff --git a/dbms/Dictionaries/DictionarySourceHelpers.h b/dbms/Dictionaries/DictionarySourceHelpers.h index 99d5df5bda7..7dc5f319432 100644 --- a/dbms/Dictionaries/DictionarySourceHelpers.h +++ b/dbms/Dictionaries/DictionarySourceHelpers.h @@ -3,7 +3,8 @@ #include #include #include - +#include +#include namespace DB { @@ -11,6 +12,9 @@ class IBlockOutputStream; using BlockOutputStreamPtr = std::shared_ptr; struct DictionaryStructure; +class Context; + + /// Write keys to block output stream. @@ -24,4 +28,10 @@ void formatKeys( const Columns & key_columns, const std::vector & requested_rows); +/// Used for applying settings to copied context in some register[...]Source functions +Context copyContextAndApplySettings( + const std::string & config_prefix, + const Context & context, + const Poco::Util::AbstractConfiguration & config); + } diff --git a/dbms/Dictionaries/ExecutableDictionarySource.cpp b/dbms/Dictionaries/ExecutableDictionarySource.cpp index d9903f759c0..6f4cd747b87 100644 --- a/dbms/Dictionaries/ExecutableDictionarySource.cpp +++ b/dbms/Dictionaries/ExecutableDictionarySource.cpp @@ -15,7 +15,7 @@ #include "DictionarySourceHelpers.h" #include "DictionaryStructure.h" #include "registerDictionaries.h" - +#include "DictionarySourceHelpers.h" namespace DB { @@ -232,9 +232,11 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) if (check_config) throw Exception("Dictionaries with Executable dictionary source is not allowed", ErrorCodes::DICTIONARY_ACCESS_DENIED); + Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + return std::make_unique( dict_struct, config, config_prefix + ".executable", - sample_block, context); + sample_block, context_local_copy); }; factory.registerSource("executable", create_table_source); } diff --git a/dbms/Dictionaries/ExecutableDictionarySource.h b/dbms/Dictionaries/ExecutableDictionarySource.h index 879248663dc..f28d71ca5e3 100644 --- a/dbms/Dictionaries/ExecutableDictionarySource.h +++ b/dbms/Dictionaries/ExecutableDictionarySource.h @@ -3,7 +3,7 @@ #include "DictionaryStructure.h" #include "IDictionarySource.h" #include - +#include namespace Poco { class Logger; } @@ -56,7 +56,7 @@ private: const std::string update_field; const std::string format; Block sample_block; - const Context & context; + Context context; }; } diff --git a/dbms/Dictionaries/FileDictionarySource.cpp b/dbms/Dictionaries/FileDictionarySource.cpp index 7871b4a2a20..0f20ab8edc4 100644 --- a/dbms/Dictionaries/FileDictionarySource.cpp +++ b/dbms/Dictionaries/FileDictionarySource.cpp @@ -7,6 +7,7 @@ #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" #include "registerDictionaries.h" +#include "DictionarySourceHelpers.h" namespace DB { @@ -21,7 +22,7 @@ namespace ErrorCodes FileDictionarySource::FileDictionarySource( const std::string & filepath_, const std::string & format_, - Block & sample_block_, Context & context_, bool check_config) + Block & sample_block_, const Context & context_, bool check_config) : filepath{filepath_} , format{format_} , sample_block{sample_block_} @@ -83,16 +84,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const auto filepath = config.getString(config_prefix + ".file.path"); const auto format = config.getString(config_prefix + ".file.format"); - Context context_local_copy(context); - if (config.has(config_prefix + ".settings")) - { - const auto prefix = config_prefix + ".settings"; - Settings settings; - - settings.loadSettingsFromConfig(prefix, config); - // const_cast(context).setSettings(settings); - context_local_copy.setSettings(settings); - } + Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); return std::make_unique(filepath, format, sample_block, context_local_copy, check_config); }; diff --git a/dbms/Dictionaries/FileDictionarySource.h b/dbms/Dictionaries/FileDictionarySource.h index e22906633db..5155e3d81e1 100644 --- a/dbms/Dictionaries/FileDictionarySource.h +++ b/dbms/Dictionaries/FileDictionarySource.h @@ -18,7 +18,7 @@ class FileDictionarySource final : public IDictionarySource { public: FileDictionarySource(const std::string & filepath_, const std::string & format_, - Block & sample_block_, Context & context_, bool check_config); + Block & sample_block_, const Context & context_, bool check_config); FileDictionarySource(const FileDictionarySource & other); diff --git a/dbms/Dictionaries/HTTPDictionarySource.cpp b/dbms/Dictionaries/HTTPDictionarySource.cpp index 87d5381f30f..61f16797ce0 100644 --- a/dbms/Dictionaries/HTTPDictionarySource.cpp +++ b/dbms/Dictionaries/HTTPDictionarySource.cpp @@ -202,9 +202,11 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) if (dict_struct.has_expressions) throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; + Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + return std::make_unique( dict_struct, config, config_prefix + ".http", - sample_block, context, check_config); + sample_block, context_local_copy, check_config); }; factory.registerSource("http", create_table_source); } diff --git a/dbms/Dictionaries/HTTPDictionarySource.h b/dbms/Dictionaries/HTTPDictionarySource.h index d04a6f1a789..e7920132e83 100644 --- a/dbms/Dictionaries/HTTPDictionarySource.h +++ b/dbms/Dictionaries/HTTPDictionarySource.h @@ -7,6 +7,7 @@ #include #include "DictionaryStructure.h" #include "IDictionarySource.h" +#include namespace Poco { @@ -64,7 +65,7 @@ private: std::string update_field; const std::string format; Block sample_block; - const Context & context; + Context context; ConnectionTimeouts timeouts; }; diff --git a/dbms/Dictionaries/XDBCDictionarySource.cpp b/dbms/Dictionaries/XDBCDictionarySource.cpp index 8647c8ccac5..149764dda03 100644 --- a/dbms/Dictionaries/XDBCDictionarySource.cpp +++ b/dbms/Dictionaries/XDBCDictionarySource.cpp @@ -19,6 +19,9 @@ #include #include "registerDictionaries.h" +/// For custom settings support +// #include "DictionarySourceHelpers.h" + #if USE_POCO_SQLODBC || USE_POCO_DATAODBC # include #endif @@ -269,6 +272,11 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) bool /* check_config */) -> DictionarySourcePtr { throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.", ErrorCodes::SUPPORT_IS_DISABLED}; + + /// Here is the support for custom settings. Good luck with fixing Nullable's segfault :) + /// NOTE: you should store the field context locally (const Context & -> Context) to make it work properly! + // Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + // BridgeHelperPtr bridge = std::make_shared>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string")); // return std::make_unique(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge); }; From c080e6c969b65c9caba1b535518f46b4c37c2ba8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 4 Apr 2020 02:37:58 +0300 Subject: [PATCH 102/743] resolve table uuid another way --- dbms/Interpreters/Context.cpp | 19 ++++++++++++++----- dbms/Interpreters/InterpreterDropQuery.cpp | 8 +++++--- .../00508_materialized_view_to.sql | 5 +---- 3 files changed, 20 insertions(+), 12 deletions(-) diff --git a/dbms/Interpreters/Context.cpp b/dbms/Interpreters/Context.cpp index 3986986ae41..87c3f0da8e2 100644 --- a/dbms/Interpreters/Context.cpp +++ b/dbms/Interpreters/Context.cpp @@ -2049,12 +2049,15 @@ StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where if (storage_id.uuid != UUIDHelpers::Nil) return storage_id; - auto lock = getLock(); + StorageID resolved = StorageID::createEmpty(); std::optional exc; - auto resolved = resolveStorageIDImpl(std::move(storage_id), where, &exc); + { + auto lock = getLock(); + resolved = resolveStorageIDImpl(std::move(storage_id), where, &exc); + } if (exc) throw Exception(*exc); - return resolved; + return DatabaseCatalog::instance().getTable(resolved)->getStorageID(); } StorageID Context::tryResolveStorageID(StorageID storage_id, StorageNamespace where) const @@ -2062,8 +2065,14 @@ StorageID Context::tryResolveStorageID(StorageID storage_id, StorageNamespace wh if (storage_id.uuid != UUIDHelpers::Nil) return storage_id; - auto lock = getLock(); - return resolveStorageIDImpl(std::move(storage_id), where, nullptr); + StorageID resolved = StorageID::createEmpty(); + { + auto lock = getLock(); + resolved = resolveStorageIDImpl(std::move(storage_id), where, nullptr); + } + if (auto table = DatabaseCatalog::instance().tryGetTable(resolved)) + return table->getStorageID(); + return StorageID::createEmpty(); } StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace where, std::optional * exception) const diff --git a/dbms/Interpreters/InterpreterDropQuery.cpp b/dbms/Interpreters/InterpreterDropQuery.cpp index da4e7f31d04..d46c8a0d759 100644 --- a/dbms/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/Interpreters/InterpreterDropQuery.cpp @@ -73,18 +73,20 @@ BlockIO InterpreterDropQuery::executeToTable( ErrorCodes::UNKNOWN_TABLE); } - auto table_id = context.resolveStorageID(table_id_, Context::ResolveOrdinary); + auto table_id = query.if_exists ? context.tryResolveStorageID(table_id_, Context::ResolveOrdinary) + : context.resolveStorageID(table_id_, Context::ResolveOrdinary); + if (!table_id) + return {}; auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); - auto [database, table] = tryGetDatabaseAndTable(table_id.database_name, table_id.table_name, query.if_exists); + auto [database, table] = DatabaseCatalog::instance().getDatabaseAndTable(table_id); if (database && table) { if (query_ptr->as().is_view && !table->isView()) throw Exception("Table " + table_id.getNameForLogs() + " is not a View", ErrorCodes::LOGICAL_ERROR); - table_id = table->getStorageID(); if (query.kind == ASTDropQuery::Kind::Detach) { context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id); diff --git a/tests/queries/0_stateless/00508_materialized_view_to.sql b/tests/queries/0_stateless/00508_materialized_view_to.sql index 7010df090eb..d330faf4fb7 100644 --- a/tests/queries/0_stateless/00508_materialized_view_to.sql +++ b/tests/queries/0_stateless/00508_materialized_view_to.sql @@ -1,11 +1,8 @@ +DROP DATABASE IF EXISTS test_00508; CREATE DATABASE test_00508; USE test_00508; -DROP TABLE IF EXISTS src; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS mv_00508; - CREATE TABLE src (x UInt8) ENGINE = Null; CREATE TABLE dst (x UInt8) ENGINE = Memory; From 9fac9a7d38c080909288147623ddd77e2e88a621 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 4 Apr 2020 18:18:35 +0300 Subject: [PATCH 103/743] Try fix aggregating. --- .../Merges/AggregatingSortedTransform.cpp | 21 ++++++++++--------- .../Merges/AggregatingSortedTransform.h | 7 +++++-- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index b3a1be9f253..c490d6a7762 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -184,26 +184,24 @@ void AggregatingSortedTransform::merge() if (key_differs) { + /// Write the simple aggregation result for the previous group. + if (merged_data.mergedRows() > 0) + insertSimpleAggregationResult(); + + merged_data.insertRow(); + /// if there are enough rows accumulated and the last one is calculated completely if (merged_data.hasEnoughRows()) - { - /// Write the simple aggregation result for the previous group. - insertSimpleAggregationResult(); return; - } /// We will write the data for the group. We copy the values of ordinary columns. - merged_data.insertRow(current->all_columns, current->pos, - columns_definition.column_numbers_not_to_aggregate); + merged_data.initializeRow(current->all_columns, current->pos, + columns_definition.column_numbers_not_to_aggregate); /// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function. for (auto & column_to_aggregate : columns_definition.columns_to_aggregate) column_to_aggregate.column->insertDefault(); - /// Write the simple aggregation result for the previous group. - if (merged_data.mergedRows() > 0) - insertSimpleAggregationResult(); - /// Reset simple aggregation states for next row for (auto & desc : columns_definition.columns_to_simple_aggregate) desc.createState(); @@ -229,7 +227,10 @@ void AggregatingSortedTransform::merge() /// Write the simple aggregation result for the previous group. if (merged_data.mergedRows() > 0) + { insertSimpleAggregationResult(); + merged_data.insertRow(); + } last_chunk_sort_columns.clear(); is_finished = true; diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.h b/dbms/src/Processors/Merges/AggregatingSortedTransform.h index 5ebc41d6ccf..7f32ed4eade 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.h +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.h @@ -59,11 +59,14 @@ private: public: using MergedData::MergedData; - void insertRow(const ColumnRawPtrs & raw_columns, size_t row, const ColumnNumbers & column_numbers) + void initializeRow(const ColumnRawPtrs & raw_columns, size_t row, const ColumnNumbers & column_numbers) { - for (auto column_number :column_numbers) + for (auto column_number : column_numbers) columns[column_number]->insertFrom(*raw_columns[column_number], row); + } + void insertRow() + { ++total_merged_rows; ++merged_rows; /// TODO: sum_blocks_granularity += block_size; From 36604e93ce1cd71c16316dde1bf6b259aae088f0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 4 Apr 2020 18:28:11 +0300 Subject: [PATCH 104/743] Try fix aggregating. --- dbms/src/Processors/Merges/AggregatingSortedTransform.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index c490d6a7762..82859559fb3 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -186,9 +186,10 @@ void AggregatingSortedTransform::merge() { /// Write the simple aggregation result for the previous group. if (merged_data.mergedRows() > 0) + { insertSimpleAggregationResult(); - - merged_data.insertRow(); + merged_data.insertRow(); + } /// if there are enough rows accumulated and the last one is calculated completely if (merged_data.hasEnoughRows()) From bcebad1d60898978c43715633b53a0e90a0bc57c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 4 Apr 2020 18:37:31 +0300 Subject: [PATCH 105/743] Try fix aggregating. --- dbms/src/Processors/Merges/AggregatingSortedTransform.cpp | 4 ++-- dbms/src/Processors/Merges/AggregatingSortedTransform.h | 7 +++++++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 82859559fb3..67b1289c4cc 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -185,7 +185,7 @@ void AggregatingSortedTransform::merge() if (key_differs) { /// Write the simple aggregation result for the previous group. - if (merged_data.mergedRows() > 0) + if (merged_data.isGroupStarted()) { insertSimpleAggregationResult(); merged_data.insertRow(); @@ -227,7 +227,7 @@ void AggregatingSortedTransform::merge() } /// Write the simple aggregation result for the previous group. - if (merged_data.mergedRows() > 0) + if (merged_data.isGroupStarted()) { insertSimpleAggregationResult(); merged_data.insertRow(); diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.h b/dbms/src/Processors/Merges/AggregatingSortedTransform.h index 7f32ed4eade..bb950a68257 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.h +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.h @@ -63,10 +63,15 @@ private: { for (auto column_number : column_numbers) columns[column_number]->insertFrom(*raw_columns[column_number], row); + + is_group_started = true; } + bool isGroupStarted() const { return is_group_started; } + void insertRow() { + is_group_started = false; ++total_merged_rows; ++merged_rows; /// TODO: sum_blocks_granularity += block_size; @@ -81,6 +86,8 @@ private: for (auto & desc : def.columns_to_aggregate) desc.column = typeid_cast(columns[desc.column_number].get()); } + private: + bool is_group_started = false; }; ColumnsDefinition columns_definition; From 12a12eb3b85ae5f0485276631948ee20cad27616 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 4 Apr 2020 18:43:55 +0300 Subject: [PATCH 106/743] Try fix aggregating. --- dbms/src/Processors/Merges/AggregatingSortedTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 67b1289c4cc..16140ff744f 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -189,6 +189,7 @@ void AggregatingSortedTransform::merge() { insertSimpleAggregationResult(); merged_data.insertRow(); + last_key.reset(); } /// if there are enough rows accumulated and the last one is calculated completely From 5bbfa40910509374af5a99d050da6d48263092cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 4 Apr 2020 18:46:44 +0300 Subject: [PATCH 107/743] Try fix aggregating. --- dbms/src/Processors/Merges/AggregatingSortedTransform.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp index 16140ff744f..68f60aed8a2 100644 --- a/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/dbms/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -189,12 +189,14 @@ void AggregatingSortedTransform::merge() { insertSimpleAggregationResult(); merged_data.insertRow(); - last_key.reset(); } /// if there are enough rows accumulated and the last one is calculated completely if (merged_data.hasEnoughRows()) + { + last_key.reset(); return; + } /// We will write the data for the group. We copy the values of ordinary columns. merged_data.initializeRow(current->all_columns, current->pos, From 51ea2cf61098b02eaa4ac778fdb265581734bc0e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Apr 2020 23:14:56 +0300 Subject: [PATCH 108/743] some tests --- dbms/src/Interpreters/InterserverIOHandler.h | 10 +- .../test_quorum_inserts/__init__.py | 0 .../configs/conf.d/clusters.xml | 37 ++++ .../configs/conf.d/ddl.xml | 5 + .../test_quorum_inserts/configs/users.xml | 27 +++ .../integration/test_quorum_inserts/test.py | 173 ++++++++++++++++++ 6 files changed, 251 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/integration/test_quorum_inserts/__init__.py create mode 100644 dbms/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml create mode 100644 dbms/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml create mode 100644 dbms/tests/integration/test_quorum_inserts/configs/users.xml create mode 100644 dbms/tests/integration/test_quorum_inserts/test.py diff --git a/dbms/src/Interpreters/InterserverIOHandler.h b/dbms/src/Interpreters/InterserverIOHandler.h index 4651c8cb978..ca3a92df2e6 100644 --- a/dbms/src/Interpreters/InterserverIOHandler.h +++ b/dbms/src/Interpreters/InterserverIOHandler.h @@ -13,6 +13,8 @@ #include #include #include +#include +#include namespace Poco { namespace Net { class HTTPServerResponse; } } @@ -51,14 +53,18 @@ public: void addEndpoint(const String & name, InterserverIOEndpointPtr endpoint) { std::lock_guard lock(mutex); + LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), "anime addEndpoint() " << name); + LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), StackTrace().toString()); bool inserted = endpoint_map.try_emplace(name, std::move(endpoint)).second; if (!inserted) throw Exception("Duplicate interserver IO endpoint: " + name, ErrorCodes::DUPLICATE_INTERSERVER_IO_ENDPOINT); } - bool removeEndpointIfExists(const String & name) + bool removeEndpointIfExists(const String & name)>> { std::lock_guard lock(mutex); + LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), "anime removeEndpointIfExists() " << name); + LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), StackTrace().toString()); return endpoint_map.erase(name); } @@ -66,6 +72,8 @@ public: try { std::lock_guard lock(mutex); + LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), "anime getEndpoint() " << name); + LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), StackTrace().toString()); return endpoint_map.at(name); } catch (...) diff --git a/dbms/tests/integration/test_quorum_inserts/__init__.py b/dbms/tests/integration/test_quorum_inserts/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml b/dbms/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml new file mode 100644 index 00000000000..adf6ad80247 --- /dev/null +++ b/dbms/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml @@ -0,0 +1,37 @@ + + + + + + true + + s0_0_0 + 9000 + + + s0_0_1 + 9000 + + + + + + + true + + s0_0_0 + 9000 + + + s0_0_1 + 9000 + + + s0_0_2 + 9000 + + + + + + diff --git a/dbms/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml b/dbms/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml new file mode 100644 index 00000000000..abad0dee450 --- /dev/null +++ b/dbms/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml @@ -0,0 +1,5 @@ + + + /clickhouse/task_queue/ddl + + \ No newline at end of file diff --git a/dbms/tests/integration/test_quorum_inserts/configs/users.xml b/dbms/tests/integration/test_quorum_inserts/configs/users.xml new file mode 100644 index 00000000000..c5114c10cde --- /dev/null +++ b/dbms/tests/integration/test_quorum_inserts/configs/users.xml @@ -0,0 +1,27 @@ + + + + + 1 + 2 + 1 + 5000 + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/dbms/tests/integration/test_quorum_inserts/test.py b/dbms/tests/integration/test_quorum_inserts/test.py new file mode 100644 index 00000000000..1f6c1b9b852 --- /dev/null +++ b/dbms/tests/integration/test_quorum_inserts/test.py @@ -0,0 +1,173 @@ +import os +import sys +import time + +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + clusters_schema = { + "0" : {"0" : ["0", "1", "2"]} + } + + for cluster_name, shards in clusters_schema.iteritems(): + for shard_name, replicas in shards.iteritems(): + for replica_name in replicas: + name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) + cluster.add_instance(name, + config_dir="configs", + macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, + with_zookeeper=True) + + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +def test_drop_replica_and_achieve_quorum(started_cluster): + zero = cluster.instances['s0_0_0'] + first = cluster.instances['s0_0_1'] + second = cluster.instances['s0_0_2'] + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_three_replicas") + + create_query = "CREATE TABLE bug.test_drop_replica_and_achieve_quorum " \ + "(a Int8, d Date) " \ + "Engine = ReplicatedMergeTree('/clickhouse/tables/test_drop_replica_and_achieve_quorum', '{}') " \ + "PARTITION BY d ORDER BY a" + + print("Create Replicated table with two replicas") + zero.query(create_query.format(0)) + first.query(create_query.format(1)) + + print("Stop fetches on one replica. Since that, it will be isolated.") + first.query("SYSTEM STOP FETCHES bug.test_drop_replica_and_achieve_quorum") + + print("Insert to other replica. This query will fail.") + quorum_timeout = zero.query_and_get_error("INSERT INTO bug.test_drop_replica_and_achieve_quorum(a,d) VALUES (1, '2011-01-01')") + assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." + + assert "1\t2011-01-01\n" == zero.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum", + settings={'select_sequential_consistency' : 0}) + + print("Add third replica") + second.query(create_query.format(2)) + + zero.query("SYSTEM RESTART REPLICA bug.test_drop_replica_and_achieve_quorum") + + print("START FETCHES first replica") + first.query("SYSTEM START FETCHES bug.test_drop_replica_and_achieve_quorum") + + time.sleep(5) + + print(zero.query("SELECT * from system.replicas format Vertical")) + + + print("---------") + print(zero.query("SELECT * from system.replication_queue format Vertical")) + print("---------") + + + print(first.query("SELECT * from system.replicas format Vertical")) + print("---------") + print(first.query("SELECT * from system.replication_queue format Vertical")) + print("---------") + print(second.query("SELECT * from system.replicas format Vertical")) + print("---------") + print(first.query("SELECT * from system.replication_queue format Vertical")) + + + print("SYNC first replica") + first.query("SYSTEM SYNC REPLICA bug.test_drop_replica_and_achieve_quorum") + + print("SYNC second replica") + second.query("SYSTEM SYNC REPLICA bug.test_drop_replica_and_achieve_quorum") + + print("Quorum for previous insert achieved.") + assert "1\t2011-01-01\n" == second.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum", + settings={'select_sequential_consistency' : 1}) + + print("Now we can insert some other data.") + zero.query("INSERT INTO bug.test_drop_replica_and_achieve_quorum(a,d) VALUES (2, '2012-02-02')") + + assert "1\t2011-01-01\n2 2012-02-02" == zero.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum") + assert "1\t2011-01-01\n2 2012-02-02" == second.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum") + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + + +def test_insert_quorum_with_drop_partition(started_cluster): + zero = cluster.instances['s0_0_0'] + first = cluster.instances['s0_0_1'] + second = cluster.instances['s0_0_2'] + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_three_replicas") + + zero.query("CREATE TABLE bug.quorum_insert_with_drop_partition ON CLUSTER one_shard_three_replicas " + "(a Int8, d Date) " + "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " + "PARTITION BY d ORDER BY a ") + + print("Stop fetches for bug.quorum_insert_with_drop_partition at first replica.") + first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_drop_partition") + + print("Insert with quorum. (zero and second)") + zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_drop_partition(a,d) VALUES(1, '2011-01-01')") + + print("Drop partition.") + zero.query_and_get_error("ALTER TABLE bug.quorum_insert_with_drop_partition DROP PARTITION '2011-01-01'") + + print("Insert to deleted partition") + zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_drop_partition(a,d) VALUES(2, '2011-01-01')") + + print("Sync other replica from quorum.") + second.query("SYSTEM SYNC REPLICA bug.quorum_insert_with_drop_partition") + + print("Select from updated partition.") + assert "2 2011-01-01\n" == zero.query("SELECT * FROM bug.quorum_insert_with_drop_partition") + assert "2 2011-01-01\n" == second.query("SELECT * FROM bug.quorum_insert_with_drop_partition") + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + + +def test_insert_quorum_with_ttl(started_cluster): + zero = cluster.instances['s0_0_0'] + first = cluster.instances['s0_0_1'] + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_two_replicas") + zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_two_replicas") + + zero.query("CREATE TABLE bug.quorum_insert_with_ttl ON CLUSTER one_shard_two_replicas " + "(a Int8, d Date) " + "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " + "PARTITION BY d ORDER BY a " + "TTL d + INTERVAL 5 second " + "SETTINGS merge_with_ttl_timeout=2 ") + + print("Stop fetches for bug.quorum_insert_with_ttl at first replica.") + first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_ttl") + + print("Insert should fail since it can not reach the quorum.") + quorum_timeout = zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_ttl(a,d) VALUES(6, now())") + assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." + + print("Wait 10 seconds and the data should be dropped by TTL.") + time.sleep(10) + count = zero.query("SELECT count() FROM bug.quorum_insert_with_ttl WHERE a=6") + assert count == "0\n", "Data have to be dropped by TTL" + + print("Resume fetches for bug.quorum_test_with_ttl at first replica.") + first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_ttl") + time.sleep(5) + + print("Inserts should resume.") + zero.query("INSERT INTO bug.quorum_insert_with_ttl(a) VALUES(6)") From 0c0eff36d132ce73cea771ffb4bfba8a0ca43c56 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Apr 2020 23:34:04 +0300 Subject: [PATCH 109/743] move --- .../test_quorum_inserts/__init__.py | 0 .../configs/conf.d/clusters.xml | 37 ++++ .../configs/conf.d/ddl.xml | 5 + .../test_quorum_inserts/configs/users.xml | 27 +++ tests/integration/test_quorum_inserts/test.py | 173 ++++++++++++++++++ 5 files changed, 242 insertions(+) create mode 100644 tests/integration/test_quorum_inserts/__init__.py create mode 100644 tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml create mode 100644 tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml create mode 100644 tests/integration/test_quorum_inserts/configs/users.xml create mode 100644 tests/integration/test_quorum_inserts/test.py diff --git a/tests/integration/test_quorum_inserts/__init__.py b/tests/integration/test_quorum_inserts/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml b/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml new file mode 100644 index 00000000000..adf6ad80247 --- /dev/null +++ b/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml @@ -0,0 +1,37 @@ + + + + + + true + + s0_0_0 + 9000 + + + s0_0_1 + 9000 + + + + + + + true + + s0_0_0 + 9000 + + + s0_0_1 + 9000 + + + s0_0_2 + 9000 + + + + + + diff --git a/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml b/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml new file mode 100644 index 00000000000..abad0dee450 --- /dev/null +++ b/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml @@ -0,0 +1,5 @@ + + + /clickhouse/task_queue/ddl + + \ No newline at end of file diff --git a/tests/integration/test_quorum_inserts/configs/users.xml b/tests/integration/test_quorum_inserts/configs/users.xml new file mode 100644 index 00000000000..c5114c10cde --- /dev/null +++ b/tests/integration/test_quorum_inserts/configs/users.xml @@ -0,0 +1,27 @@ + + + + + 1 + 2 + 1 + 5000 + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py new file mode 100644 index 00000000000..1f6c1b9b852 --- /dev/null +++ b/tests/integration/test_quorum_inserts/test.py @@ -0,0 +1,173 @@ +import os +import sys +import time + +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + clusters_schema = { + "0" : {"0" : ["0", "1", "2"]} + } + + for cluster_name, shards in clusters_schema.iteritems(): + for shard_name, replicas in shards.iteritems(): + for replica_name in replicas: + name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) + cluster.add_instance(name, + config_dir="configs", + macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, + with_zookeeper=True) + + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +def test_drop_replica_and_achieve_quorum(started_cluster): + zero = cluster.instances['s0_0_0'] + first = cluster.instances['s0_0_1'] + second = cluster.instances['s0_0_2'] + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_three_replicas") + + create_query = "CREATE TABLE bug.test_drop_replica_and_achieve_quorum " \ + "(a Int8, d Date) " \ + "Engine = ReplicatedMergeTree('/clickhouse/tables/test_drop_replica_and_achieve_quorum', '{}') " \ + "PARTITION BY d ORDER BY a" + + print("Create Replicated table with two replicas") + zero.query(create_query.format(0)) + first.query(create_query.format(1)) + + print("Stop fetches on one replica. Since that, it will be isolated.") + first.query("SYSTEM STOP FETCHES bug.test_drop_replica_and_achieve_quorum") + + print("Insert to other replica. This query will fail.") + quorum_timeout = zero.query_and_get_error("INSERT INTO bug.test_drop_replica_and_achieve_quorum(a,d) VALUES (1, '2011-01-01')") + assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." + + assert "1\t2011-01-01\n" == zero.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum", + settings={'select_sequential_consistency' : 0}) + + print("Add third replica") + second.query(create_query.format(2)) + + zero.query("SYSTEM RESTART REPLICA bug.test_drop_replica_and_achieve_quorum") + + print("START FETCHES first replica") + first.query("SYSTEM START FETCHES bug.test_drop_replica_and_achieve_quorum") + + time.sleep(5) + + print(zero.query("SELECT * from system.replicas format Vertical")) + + + print("---------") + print(zero.query("SELECT * from system.replication_queue format Vertical")) + print("---------") + + + print(first.query("SELECT * from system.replicas format Vertical")) + print("---------") + print(first.query("SELECT * from system.replication_queue format Vertical")) + print("---------") + print(second.query("SELECT * from system.replicas format Vertical")) + print("---------") + print(first.query("SELECT * from system.replication_queue format Vertical")) + + + print("SYNC first replica") + first.query("SYSTEM SYNC REPLICA bug.test_drop_replica_and_achieve_quorum") + + print("SYNC second replica") + second.query("SYSTEM SYNC REPLICA bug.test_drop_replica_and_achieve_quorum") + + print("Quorum for previous insert achieved.") + assert "1\t2011-01-01\n" == second.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum", + settings={'select_sequential_consistency' : 1}) + + print("Now we can insert some other data.") + zero.query("INSERT INTO bug.test_drop_replica_and_achieve_quorum(a,d) VALUES (2, '2012-02-02')") + + assert "1\t2011-01-01\n2 2012-02-02" == zero.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum") + assert "1\t2011-01-01\n2 2012-02-02" == second.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum") + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + + +def test_insert_quorum_with_drop_partition(started_cluster): + zero = cluster.instances['s0_0_0'] + first = cluster.instances['s0_0_1'] + second = cluster.instances['s0_0_2'] + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_three_replicas") + + zero.query("CREATE TABLE bug.quorum_insert_with_drop_partition ON CLUSTER one_shard_three_replicas " + "(a Int8, d Date) " + "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " + "PARTITION BY d ORDER BY a ") + + print("Stop fetches for bug.quorum_insert_with_drop_partition at first replica.") + first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_drop_partition") + + print("Insert with quorum. (zero and second)") + zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_drop_partition(a,d) VALUES(1, '2011-01-01')") + + print("Drop partition.") + zero.query_and_get_error("ALTER TABLE bug.quorum_insert_with_drop_partition DROP PARTITION '2011-01-01'") + + print("Insert to deleted partition") + zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_drop_partition(a,d) VALUES(2, '2011-01-01')") + + print("Sync other replica from quorum.") + second.query("SYSTEM SYNC REPLICA bug.quorum_insert_with_drop_partition") + + print("Select from updated partition.") + assert "2 2011-01-01\n" == zero.query("SELECT * FROM bug.quorum_insert_with_drop_partition") + assert "2 2011-01-01\n" == second.query("SELECT * FROM bug.quorum_insert_with_drop_partition") + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + + +def test_insert_quorum_with_ttl(started_cluster): + zero = cluster.instances['s0_0_0'] + first = cluster.instances['s0_0_1'] + + zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_two_replicas") + zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_two_replicas") + + zero.query("CREATE TABLE bug.quorum_insert_with_ttl ON CLUSTER one_shard_two_replicas " + "(a Int8, d Date) " + "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " + "PARTITION BY d ORDER BY a " + "TTL d + INTERVAL 5 second " + "SETTINGS merge_with_ttl_timeout=2 ") + + print("Stop fetches for bug.quorum_insert_with_ttl at first replica.") + first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_ttl") + + print("Insert should fail since it can not reach the quorum.") + quorum_timeout = zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_ttl(a,d) VALUES(6, now())") + assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." + + print("Wait 10 seconds and the data should be dropped by TTL.") + time.sleep(10) + count = zero.query("SELECT count() FROM bug.quorum_insert_with_ttl WHERE a=6") + assert count == "0\n", "Data have to be dropped by TTL" + + print("Resume fetches for bug.quorum_test_with_ttl at first replica.") + first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_ttl") + time.sleep(5) + + print("Inserts should resume.") + zero.query("INSERT INTO bug.quorum_insert_with_ttl(a) VALUES(6)") From fef7140ea924fbf75e46e1a745ca612aec527113 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Apr 2020 23:35:20 +0300 Subject: [PATCH 110/743] delete old --- .../test_quorum_inserts/__init__.py | 0 .../configs/conf.d/clusters.xml | 37 ---- .../configs/conf.d/ddl.xml | 5 - .../test_quorum_inserts/configs/users.xml | 27 --- .../integration/test_quorum_inserts/test.py | 173 ------------------ 5 files changed, 242 deletions(-) delete mode 100644 dbms/tests/integration/test_quorum_inserts/__init__.py delete mode 100644 dbms/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml delete mode 100644 dbms/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml delete mode 100644 dbms/tests/integration/test_quorum_inserts/configs/users.xml delete mode 100644 dbms/tests/integration/test_quorum_inserts/test.py diff --git a/dbms/tests/integration/test_quorum_inserts/__init__.py b/dbms/tests/integration/test_quorum_inserts/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml b/dbms/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml deleted file mode 100644 index adf6ad80247..00000000000 --- a/dbms/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml +++ /dev/null @@ -1,37 +0,0 @@ - - - - - - true - - s0_0_0 - 9000 - - - s0_0_1 - 9000 - - - - - - - true - - s0_0_0 - 9000 - - - s0_0_1 - 9000 - - - s0_0_2 - 9000 - - - - - - diff --git a/dbms/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml b/dbms/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml deleted file mode 100644 index abad0dee450..00000000000 --- a/dbms/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - /clickhouse/task_queue/ddl - - \ No newline at end of file diff --git a/dbms/tests/integration/test_quorum_inserts/configs/users.xml b/dbms/tests/integration/test_quorum_inserts/configs/users.xml deleted file mode 100644 index c5114c10cde..00000000000 --- a/dbms/tests/integration/test_quorum_inserts/configs/users.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - - 1 - 2 - 1 - 5000 - - - - - - - - ::/0 - - default - default - - - - - - - - diff --git a/dbms/tests/integration/test_quorum_inserts/test.py b/dbms/tests/integration/test_quorum_inserts/test.py deleted file mode 100644 index 1f6c1b9b852..00000000000 --- a/dbms/tests/integration/test_quorum_inserts/test.py +++ /dev/null @@ -1,173 +0,0 @@ -import os -import sys -import time - -import pytest - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - clusters_schema = { - "0" : {"0" : ["0", "1", "2"]} - } - - for cluster_name, shards in clusters_schema.iteritems(): - for shard_name, replicas in shards.iteritems(): - for replica_name in replicas: - name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) - cluster.add_instance(name, - config_dir="configs", - macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, - with_zookeeper=True) - - cluster.start() - yield cluster - - finally: - cluster.shutdown() - -def test_drop_replica_and_achieve_quorum(started_cluster): - zero = cluster.instances['s0_0_0'] - first = cluster.instances['s0_0_1'] - second = cluster.instances['s0_0_2'] - - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") - zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_three_replicas") - - create_query = "CREATE TABLE bug.test_drop_replica_and_achieve_quorum " \ - "(a Int8, d Date) " \ - "Engine = ReplicatedMergeTree('/clickhouse/tables/test_drop_replica_and_achieve_quorum', '{}') " \ - "PARTITION BY d ORDER BY a" - - print("Create Replicated table with two replicas") - zero.query(create_query.format(0)) - first.query(create_query.format(1)) - - print("Stop fetches on one replica. Since that, it will be isolated.") - first.query("SYSTEM STOP FETCHES bug.test_drop_replica_and_achieve_quorum") - - print("Insert to other replica. This query will fail.") - quorum_timeout = zero.query_and_get_error("INSERT INTO bug.test_drop_replica_and_achieve_quorum(a,d) VALUES (1, '2011-01-01')") - assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." - - assert "1\t2011-01-01\n" == zero.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum", - settings={'select_sequential_consistency' : 0}) - - print("Add third replica") - second.query(create_query.format(2)) - - zero.query("SYSTEM RESTART REPLICA bug.test_drop_replica_and_achieve_quorum") - - print("START FETCHES first replica") - first.query("SYSTEM START FETCHES bug.test_drop_replica_and_achieve_quorum") - - time.sleep(5) - - print(zero.query("SELECT * from system.replicas format Vertical")) - - - print("---------") - print(zero.query("SELECT * from system.replication_queue format Vertical")) - print("---------") - - - print(first.query("SELECT * from system.replicas format Vertical")) - print("---------") - print(first.query("SELECT * from system.replication_queue format Vertical")) - print("---------") - print(second.query("SELECT * from system.replicas format Vertical")) - print("---------") - print(first.query("SELECT * from system.replication_queue format Vertical")) - - - print("SYNC first replica") - first.query("SYSTEM SYNC REPLICA bug.test_drop_replica_and_achieve_quorum") - - print("SYNC second replica") - second.query("SYSTEM SYNC REPLICA bug.test_drop_replica_and_achieve_quorum") - - print("Quorum for previous insert achieved.") - assert "1\t2011-01-01\n" == second.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum", - settings={'select_sequential_consistency' : 1}) - - print("Now we can insert some other data.") - zero.query("INSERT INTO bug.test_drop_replica_and_achieve_quorum(a,d) VALUES (2, '2012-02-02')") - - assert "1\t2011-01-01\n2 2012-02-02" == zero.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum") - assert "1\t2011-01-01\n2 2012-02-02" == second.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum") - - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") - - -def test_insert_quorum_with_drop_partition(started_cluster): - zero = cluster.instances['s0_0_0'] - first = cluster.instances['s0_0_1'] - second = cluster.instances['s0_0_2'] - - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") - zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_three_replicas") - - zero.query("CREATE TABLE bug.quorum_insert_with_drop_partition ON CLUSTER one_shard_three_replicas " - "(a Int8, d Date) " - "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " - "PARTITION BY d ORDER BY a ") - - print("Stop fetches for bug.quorum_insert_with_drop_partition at first replica.") - first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_drop_partition") - - print("Insert with quorum. (zero and second)") - zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_drop_partition(a,d) VALUES(1, '2011-01-01')") - - print("Drop partition.") - zero.query_and_get_error("ALTER TABLE bug.quorum_insert_with_drop_partition DROP PARTITION '2011-01-01'") - - print("Insert to deleted partition") - zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_drop_partition(a,d) VALUES(2, '2011-01-01')") - - print("Sync other replica from quorum.") - second.query("SYSTEM SYNC REPLICA bug.quorum_insert_with_drop_partition") - - print("Select from updated partition.") - assert "2 2011-01-01\n" == zero.query("SELECT * FROM bug.quorum_insert_with_drop_partition") - assert "2 2011-01-01\n" == second.query("SELECT * FROM bug.quorum_insert_with_drop_partition") - - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") - - -def test_insert_quorum_with_ttl(started_cluster): - zero = cluster.instances['s0_0_0'] - first = cluster.instances['s0_0_1'] - - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_two_replicas") - zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_two_replicas") - - zero.query("CREATE TABLE bug.quorum_insert_with_ttl ON CLUSTER one_shard_two_replicas " - "(a Int8, d Date) " - "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " - "PARTITION BY d ORDER BY a " - "TTL d + INTERVAL 5 second " - "SETTINGS merge_with_ttl_timeout=2 ") - - print("Stop fetches for bug.quorum_insert_with_ttl at first replica.") - first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_ttl") - - print("Insert should fail since it can not reach the quorum.") - quorum_timeout = zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_ttl(a,d) VALUES(6, now())") - assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." - - print("Wait 10 seconds and the data should be dropped by TTL.") - time.sleep(10) - count = zero.query("SELECT count() FROM bug.quorum_insert_with_ttl WHERE a=6") - assert count == "0\n", "Data have to be dropped by TTL" - - print("Resume fetches for bug.quorum_test_with_ttl at first replica.") - first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_ttl") - time.sleep(5) - - print("Inserts should resume.") - zero.query("INSERT INTO bug.quorum_insert_with_ttl(a) VALUES(6)") From bcc77fc96aae29d47df04ecac67bfea649234b88 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 6 Apr 2020 23:42:20 +0300 Subject: [PATCH 111/743] Update perf-comparison-tweaks-config.xml --- .../config/config.d/perf-comparison-tweaks-config.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml index a6d59fc3b4c..090d8ebe581 100644 --- a/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml @@ -1,4 +1,7 @@ + + + :: From a2101c9b1ced5fc0aa2e5e8804541ee5267455f6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 7 Apr 2020 02:22:44 +0300 Subject: [PATCH 112/743] better code --- src/Common/ErrorCodes.cpp | 2 +- src/Databases/DatabaseAtomic.cpp | 31 ++++++------ src/Databases/DatabaseAtomic.h | 14 ++---- src/Databases/DatabaseLazy.cpp | 4 +- src/Databases/DatabaseOnDisk.cpp | 38 ++++----------- src/Databases/DatabaseOnDisk.h | 10 +--- src/Databases/DatabaseOrdinary.cpp | 36 +++++++++----- src/Databases/DatabaseOrdinary.h | 2 + src/Databases/DatabaseWithDictionaries.h | 4 +- src/Interpreters/Context.cpp | 1 - src/Interpreters/DatabaseCatalog.cpp | 21 ++++---- src/Interpreters/DatabaseCatalog.h | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 21 ++------ src/Interpreters/InterpreterCreateQuery.h | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 24 +--------- src/Interpreters/InterpreterDropQuery.h | 2 - .../InterpreterShowCreateQuery.cpp | 6 --- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Interpreters/StorageID.cpp | 4 +- src/Parsers/ExpressionElementParsers.cpp | 48 +++++-------------- src/Parsers/ExpressionElementParsers.h | 12 ++--- src/Parsers/ParserCreateQuery.cpp | 35 ++++++++------ src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 5 +- src/Storages/LiveView/StorageLiveView.cpp | 1 - src/Storages/StorageFile.h | 1 + src/Storages/StorageMaterializedView.cpp | 1 + .../System/StorageSystemDatabases.cpp | 1 - src/Storages/System/attachSystemTables.cpp | 2 +- tests/config/access_management.xml | 0 30 files changed, 126 insertions(+), 208 deletions(-) create mode 100644 tests/config/access_management.xml diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index c1f812e3981..2fcd54dfa3e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -491,7 +491,7 @@ namespace ErrorCodes extern const int CANNOT_ASSIGN_ALTER = 517; extern const int CANNOT_COMMIT_OFFSET = 518; extern const int NO_REMOTE_SHARD_AVAILABLE = 519; - extern const int ATOMIC_RENAME_FAIL = 519; + extern const int ATOMIC_RENAME_FAIL = 520; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 871eac48647..f57fa16073c 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -28,10 +28,8 @@ public: DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_) - : DatabaseOrdinary(name_, metadata_path_, context_) + : DatabaseOrdinary(name_, metadata_path_, "store/", "DatabaseAtomic (" + name_ + ")", context_) { - data_path = "store/"; - log = &Logger::get("DatabaseAtomic (" + name_ + ")"); } String DatabaseAtomic::getTableDataPath(const String & table_name) const @@ -46,7 +44,7 @@ String DatabaseAtomic::getTableDataPath(const String & table_name) const String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const { - auto tmp = data_path + getPathForUUID(query.uuid); + auto tmp = data_path + DatabaseCatalog::getPathForUUID(query.uuid); assert(tmp != data_path && !tmp.empty()); return tmp; @@ -60,7 +58,9 @@ void DatabaseAtomic::drop(const Context &) void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) { assert(relative_table_path != data_path && !relative_table_path.empty()); + DetachedTables not_in_use; std::lock_guard lock(mutex); + not_in_use = cleenupDetachedTables(); assertDetachedTableNotInUse(table->getStorageID().uuid); DatabaseWithDictionaries::attachTableUnlocked(name, table, relative_table_path); table_name_to_path.emplace(std::make_pair(name, relative_table_path)); @@ -68,11 +68,12 @@ void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, StoragePtr DatabaseAtomic::detachTable(const String & name) { + DetachedTables not_in_use; std::lock_guard lock(mutex); auto table = DatabaseWithDictionaries::detachTableUnlocked(name); table_name_to_path.erase(name); detached_tables.emplace(table->getStorageID().uuid, table); - cleenupDetachedTables(); + not_in_use = cleenupDetachedTables(); return table; } @@ -179,23 +180,15 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n attach(*this, table_name, other_table_data_path, other_table); } -void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) -{ - DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag); -} - -void DatabaseAtomic::shutdown() -{ - DatabaseWithDictionaries::shutdown(); -} - void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path) { + DetachedTables not_in_use; auto table_data_path = getTableDataPath(query); try { std::lock_guard lock{mutex}; + not_in_use = cleenupDetachedTables(); assertDetachedTableNotInUse(query.uuid); renameNoReplace(table_metadata_tmp_path, table_metadata_path); attachTableUnlocked(query.table, table, table_data_path); /// Should never throw @@ -224,22 +217,26 @@ void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid) { - cleenupDetachedTables(); if (detached_tables.count(uuid)) throw Exception("Cannot attach table with UUID " + toString(uuid) + ", because it was detached but still used by come query. Retry later.", ErrorCodes::TABLE_ALREADY_EXISTS); } -void DatabaseAtomic::cleenupDetachedTables() +DatabaseAtomic::DetachedTables DatabaseAtomic::cleenupDetachedTables() { + DetachedTables not_in_use; auto it = detached_tables.begin(); while (it != detached_tables.end()) { if (it->second.unique()) + { + not_in_use.emplace(it->first, it->second); it = detached_tables.erase(it); + } else ++it; } + return not_in_use; } DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const IDatabase::FilterByNameFunction & filter_by_table_name) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index ce2c2fd79ae..e68e924d008 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -31,17 +31,8 @@ public: String getTableDataPath(const String & table_name) const override; String getTableDataPath(const ASTCreateQuery & query) const override; - inline static String getPathForUUID(const UUID & uuid) - { - const size_t uuid_prefix_len = 3; - return toString(uuid).substr(0, uuid_prefix_len) + '/' + toString(uuid) + '/'; - } - void drop(const Context & /*context*/) override; - void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; - void shutdown() override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const FilterByNameFunction & filter_by_dictionary_name) override; @@ -51,12 +42,13 @@ private: const String & table_metadata_tmp_path, const String & table_metadata_path) override; void assertDetachedTableNotInUse(const UUID & uuid); - void cleenupDetachedTables(); + typedef std::map DetachedTables; + DetachedTables cleenupDetachedTables(); //TODO store path in DatabaseWithOwnTables::tables std::map table_name_to_path; - std::map detached_tables; + DetachedTables detached_tables; }; } diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 1faa41c547e..e575233d942 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_) - : DatabaseOnDisk(name_, metadata_path_, "DatabaseLazy (" + name_ + ")", context_) + : DatabaseOnDisk(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseLazy (" + name_ + ")", context_) , expiration_time(expiration_time_) { } @@ -38,8 +38,6 @@ void DatabaseLazy::loadStoredObjects( Context & context, bool /* has_force_restore_data_flag */) { - Poco::File(context.getPath() + getDataPath()).createDirectories(); - Poco::File(getMetadataPath()).createDirectories(); iterateMetadataFiles(context, [this](const String & file_name) { const std::string table_name = file_name.substr(0, file_name.size() - 4); diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index a8a8e85d6ee..539f2e95a7e 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -57,7 +57,7 @@ std::pair createTableFromAST( const auto & table_function = ast_create_query.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table); - storage->resetStorageIDForTableFunction({ast_create_query.database, ast_create_query.table, ast_create_query.uuid}); + storage->resetStorageID({ast_create_query.database, ast_create_query.table, ast_create_query.uuid}); return {ast_create_query.table, storage}; } /// We do not directly use `InterpreterCreateQuery::execute`, because @@ -123,12 +123,14 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) return statement_stream.str(); } -DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context & context) +DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context) : DatabaseWithOwnTablesBase(name, logger) , metadata_path(metadata_path_) - , data_path("data/" + escapeForFileName(database_name) + "/") + , data_path(data_path_) , global_context(context.getGlobalContext()) { + Poco::File(context.getPath() + data_path).createDirectories(); + Poco::File(metadata_path).createDirectories(); } @@ -284,7 +286,7 @@ void DatabaseOnDisk::renameTable( /// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID table->rename(to_database.getTableDataPath(create), to_database.getDatabaseName(), to_table_name, table_lock); - table->resetStorageIDForTableFunction({create.database, create.table, create.uuid}); /// reset UUID + table->resetStorageID({create.database, create.table, create.uuid}); /// reset UUID } catch (const Exception &) { @@ -366,11 +368,11 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n return static_cast(0); } -void DatabaseOnDisk::iterateMetadataFiles(const Context & context, - const DatabaseOnDisk::IteratingFunction & process_metadata_file) const +void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const { - IteratingFunction process_tmp_drop_metadata_file = [&](const String & file_name) + auto process_tmp_drop_metadata_file = [&](const String & file_name) { + assert(getEngineName() != "Atomic"); static const char * tmp_drop_ext = ".sql.tmp_drop"; const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists()) @@ -386,14 +388,6 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, } }; - IteratingFunction do_nothing = [](const String &){}; - //FIXME refactor this trash - iterateMetadataFiles(context, process_metadata_file, dynamic_cast(this) ? do_nothing : process_tmp_drop_metadata_file); -} - -void DatabaseOnDisk::iterateMetadataFiles(const Context & /*context*/, const IteratingFunction & process_metadata_file, - const IteratingFunction & process_tmp_drop_metadata_file) const -{ Poco::DirectoryIterator dir_end; for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) { @@ -496,18 +490,4 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const Context & context, const return ast; } -void DatabaseOnDisk::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path) -{ - try - { - /// rename atomically replaces the old file with the new one. - Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path); - } - catch (...) - { - Poco::File(table_metadata_tmp_path).remove(); - throw; - } -} - } diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 5c7e41343a4..09f222818b0 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -30,7 +30,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query); class DatabaseOnDisk : public DatabaseWithOwnTablesBase { public: - DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context & context); + DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context); void createTable( const Context & context, @@ -72,8 +72,6 @@ protected: using IteratingFunction = std::function; void iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const; - void iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file, - const IteratingFunction & process_tmp_drop_metadata_file) const; ASTPtr getCreateTableQueryImpl( const Context & context, @@ -82,15 +80,11 @@ protected: ASTPtr getCreateQueryFromMetadata(const Context & context, const String & metadata_path, bool throw_on_error) const; - virtual void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path); virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path); - //bool detachTableAndRemoveMetadata(const String & table_name); - //void replaceMetadata(const ASTPtr & create, ); - const String metadata_path; - /*const*/ String data_path; + const String data_path; const Context & global_context; }; diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index ff98870efa4..26c79e4b261 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -29,18 +29,11 @@ #include #include #include -#include "DatabaseAtomic.h" namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_ASSIGN_ALTER; -} - - static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256; static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5; static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; @@ -104,17 +97,19 @@ namespace DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context_) - : DatabaseWithDictionaries(name_, metadata_path_,"DatabaseOrdinary (" + name_ + ")", context_) + : DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_) { } +DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context_) + : DatabaseWithDictionaries(name_, metadata_path_, data_path_, logger, context_) +{ +} void DatabaseOrdinary::loadStoredObjects( Context & context, bool has_force_restore_data_flag) { - Poco::File(context.getPath() + getDataPath()).createDirectories(); - Poco::File(getMetadataPath()).createDirectories(); /** Tables load faster if they are loaded in sorted (by name) order. * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, * which does not correspond to order tables creation and does not correspond to order of their location on disk. @@ -123,7 +118,8 @@ void DatabaseOrdinary::loadStoredObjects( FileNames file_names; size_t total_dictionaries = 0; - iterateMetadataFiles(context, [&context, &file_names, &total_dictionaries, this](const String & file_name) + + auto process_metadata = [&context, &file_names, &total_dictionaries, this](const String & file_name) { String full_path = getMetadataPath() + file_name; try @@ -142,7 +138,9 @@ void DatabaseOrdinary::loadStoredObjects( throw; } - }); + }; + + iterateMetadataFiles(context, process_metadata); size_t total_tables = file_names.size() - total_dictionaries; @@ -285,4 +283,18 @@ void DatabaseOrdinary::alterTable( commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path); } +void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path) +{ + try + { + /// rename atomically replaces the old file with the new one. + Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path); + } + catch (...) + { + Poco::File(table_metadata_tmp_path).remove(); + throw; + } +} + } diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 58f0a7314e9..a9e53edfe28 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -15,6 +15,7 @@ class DatabaseOrdinary : public DatabaseWithDictionaries { public: DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context); + DatabaseOrdinary(const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context_); String getEngineName() const override { return "Ordinary"; } @@ -28,6 +29,7 @@ public: const StorageInMemoryMetadata & metadata) override; protected: + virtual void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path); void startupTables(ThreadPool & thread_pool); }; diff --git a/src/Databases/DatabaseWithDictionaries.h b/src/Databases/DatabaseWithDictionaries.h index 4a3132a432a..1816f925778 100644 --- a/src/Databases/DatabaseWithDictionaries.h +++ b/src/Databases/DatabaseWithDictionaries.h @@ -31,8 +31,8 @@ public: ~DatabaseWithDictionaries() override; protected: - DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & logger, const Context & context) - : DatabaseOnDisk(name, metadata_path_, logger, context) {} + DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context) + : DatabaseOnDisk(name, metadata_path_, data_path_, logger, context) {} void attachToExternalDictionariesLoader(Context & context); void detachFromExternalDictionariesLoader(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 87c3f0da8e2..9a677278a2a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -490,7 +490,6 @@ String Context::resolveDatabase(const String & database_name) const String Context::getPath() const { - //FIXME there is no in clickhouse-local, so context->getPath() + relative_data_path may not work correctly (it may create StorageFile for example) auto lock = getLock(); return shared->path; } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 128558e76f1..bbe84baa466 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -525,7 +525,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr if (create) { - String data_path = "store/" + DatabaseAtomic::getPathForUUID(table_id.uuid); + String data_path = "store/" + getPathForUUID(table_id.uuid); create->database = table_id.database_name; create->table = table_id.table_name; try @@ -560,18 +560,13 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr void DatabaseCatalog::dropTableDataTask() { - //LOG_INFO(log, String("Wake up ") + __PRETTY_FUNCTION__); TableMarkedAsDropped table; try { std::lock_guard lock(tables_marked_droped_mutex); - LOG_INFO(log, "There are " + std::to_string(tables_marked_droped.size()) + " tables to drop"); time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - auto it = std::find_if(tables_marked_droped.begin(), tables_marked_droped.end(), [current_time/*, this*/](const auto & elem) + auto it = std::find_if(tables_marked_droped.begin(), tables_marked_droped.end(), [current_time](const auto & elem) { - //LOG_INFO(log, "Check table " + elem.table_id.getNameForLogs() + ": " + - // "refcount = " + std::to_string(elem.table.use_count()) + ", " + - // "time elapsed = " + std::to_string(current_time - elem.drop_time)); bool not_in_use = !elem.table || elem.table.unique(); bool old_enough = elem.drop_time + drop_delay_s < current_time; return not_in_use && old_enough; @@ -613,24 +608,30 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) const { if (table.table) { - LOG_INFO(log, "Trying to drop table " + table.table_id.getNameForLogs()); table.table->drop(); table.table->is_dropped = true; } /// Even if table is not loaded, try remove its data from disk. /// TODO remove data from all volumes - String data_path = global_context->getPath() + "store/" + DatabaseAtomic::getPathForUUID(table.table_id.uuid); + String data_path = global_context->getPath() + "store/" + getPathForUUID(table.table_id.uuid); Poco::File table_data_dir{data_path}; if (table_data_dir.exists()) { - LOG_INFO(log, "Removing data directory " << data_path << " of table " << table.table_id.getNameForLogs()); + LOG_INFO(log, "Removing data directory " << data_path << " of dropped table " << table.table_id.getNameForLogs()); table_data_dir.remove(true); } + LOG_INFO(log, "Removing metadata " << table.metadata_path << " of dropped table " << table.table_id.getNameForLogs()); Poco::File(table.metadata_path).remove(); } +String DatabaseCatalog::getPathForUUID(const UUID & uuid) +{ + const size_t uuid_prefix_len = 3; + return toString(uuid).substr(0, uuid_prefix_len) + '/' + toString(uuid) + '/'; +} + DDLGuard::DDLGuard(Map & map_, std::unique_lock guards_lock_, const String & elem) : map(map_), guards_lock(std::move(guards_lock_)) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 8d800faaebc..cce83970876 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -154,6 +154,8 @@ public: /// For moving table between databases void updateUUIDMapping(const UUID & uuid, DatabasePtr database, StoragePtr table); + static String getPathForUUID(const UUID & uuid); + DatabaseAndTable tryGetByUUID(const UUID & uuid) const; String getPathForDroppedMetadata(const StorageID & table_id) const; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d81fff61f83..8d31e8daa87 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -51,8 +51,6 @@ #include -#include -#include namespace DB { @@ -96,7 +94,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) { /// For new-style databases engine is explicitly specified in .sql /// When attaching old-style database during server startup, we must always use Ordinary engine - //FIXME is it possible, that database engine is not specified in metadata file? if (create.attach) throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE); bool old_style_database = context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; @@ -150,7 +147,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) bool renamed = false; try { - //FIXME is it possible to attach db only after it was loaded? (no, loadStoredObjects adds view dependencies) + /// TODO Attach db only after it was loaded. Now it's not possible because of view dependencies DatabaseCatalog::instance().attachDatabase(database_name, database); added = true; @@ -576,7 +573,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) return fillTableIfNeeded(create); } -bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, +bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties) { std::unique_ptr guard; @@ -591,7 +588,7 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, database = DatabaseCatalog::instance().getDatabase(create.database); if (database->getEngineName() == "Atomic") { - //TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/ + /// TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/ if (create.attach && create.uuid == UUIDHelpers::Nil) throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY); if (!create.attach && create.uuid == UUIDHelpers::Nil) @@ -603,9 +600,6 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, throw Exception("Table UUID specified, but engine of database " + create.database + " is not Atomic", ErrorCodes::INCORRECT_QUERY); } - if (!create.attach && create.uuid == UUIDHelpers::Nil && database->getEngineName() == "Atomic") - create.uuid = UUIDHelpers::generateV4(); - /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. */ @@ -653,7 +647,7 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create, const auto & table_function = create.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table); - res->resetStorageIDForTableFunction({create.database, create.table, create.uuid}); + res->resetStorageID({create.database, create.table, create.uuid}); } else { @@ -722,13 +716,6 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create) "Dictionary " + database_name + "." + dictionary_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); } - if (create.attach) - { - auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateDictionaryQuery(context, dictionary_name); - create = query->as(); - create.attach = true; - } - if (create.attach) database->attachDictionary(dictionary_name, context); else diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index ac2b1dae425..c52c12b2e4b 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -68,7 +68,7 @@ private: AccessRightsElements getRequiredAccess() const; /// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false. - bool doCreateTable(/*const*/ ASTCreateQuery & create, const TableProperties & properties); + bool doCreateTable(ASTCreateQuery & create, const TableProperties & properties); /// Inserts data in created table if it's CREATE ... SELECT BlockIO fillTableIfNeeded(const ASTCreateQuery & create); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index d46c8a0d759..d20150b7996 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -11,7 +11,6 @@ #include #include #include -#include namespace DB @@ -73,6 +72,7 @@ BlockIO InterpreterDropQuery::executeToTable( ErrorCodes::UNKNOWN_TABLE); } + /// If table was already dropped by anyone, an exception will be thrown auto table_id = query.if_exists ? context.tryResolveStorageID(table_id_, Context::ResolveOrdinary) : context.resolveStorageID(table_id_, Context::ResolveOrdinary); if (!table_id) @@ -93,9 +93,7 @@ BlockIO InterpreterDropQuery::executeToTable( table->shutdown(); TableStructureWriteLockHolder table_lock; if (database->getEngineName() != "Atomic") - { table_lock = table->lockExclusively(context.getCurrentQueryId()); - } /// Drop table from memory, don't touch data and metadata database->detachTable(table_id.table_name); } @@ -104,7 +102,6 @@ BlockIO InterpreterDropQuery::executeToTable( context.checkAccess(table->isView() ? AccessType::TRUNCATE_VIEW : AccessType::TRUNCATE_TABLE, table_id); table->checkTableCanBeDropped(); - /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); /// Drop table data, don't touch metadata table->truncate(query_ptr, context, table_lock); @@ -186,7 +183,6 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id); if (kind == ASTDropQuery::Kind::Truncate) { - /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); /// Drop table data, don't touch metadata table->truncate(query_ptr, context, table_lock); @@ -195,7 +191,6 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, { context_handle.removeExternalTable(table_name); table->shutdown(); - /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId()); /// Delete table data table->drop(); @@ -248,23 +243,6 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS } -DatabaseAndTable InterpreterDropQuery::tryGetDatabaseAndTable(const String & database_name, const String & table_name, bool if_exists) -{ - DatabasePtr database = tryGetDatabase(database_name, if_exists); - - if (database) - { - StoragePtr table = database->tryGetTable(context, table_name); - if (!table && !if_exists) - throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist", - ErrorCodes::UNKNOWN_TABLE); - - return {std::move(database), std::move(table)}; - } - return {}; -} - - AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() const { AccessRightsElements required_access; diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index f1acc3b9a33..80bd6c6531a 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -34,8 +34,6 @@ private: BlockIO executeToDictionary(const String & database_name, const String & dictionary_name, ASTDropQuery::Kind kind, bool if_exists, bool is_temporary, bool no_ddl_lock); - DatabaseAndTable tryGetDatabaseAndTable(const String & database_name, const String & table_name, bool if_exists); - BlockIO executeToTemporaryTable(const String & table_name, ASTDropQuery::Kind kind); }; } diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index cc73b7bac5c..8bee0b88fe8 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -15,8 +15,6 @@ #include -#include - namespace DB { @@ -74,10 +72,6 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (!create_query && show_query && show_query->temporary) throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY); - //FIXME temporary print create query without UUID for tests (remove it) - auto & create = create_query->as(); - create.uuid = UUIDHelpers::Nil; - std::stringstream stream; formatAST(*create_query, stream, false, true); String res = stream.str(); diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 351a0cee634..c0ad744657f 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -367,7 +367,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) { if (dynamic_cast(iterator->table().get())) - replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()}); + replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name(), iterator->uuid()}); } } diff --git a/src/Interpreters/StorageID.cpp b/src/Interpreters/StorageID.cpp index ef4e4257ce7..bad4bb4837c 100644 --- a/src/Interpreters/StorageID.cpp +++ b/src/Interpreters/StorageID.cpp @@ -33,7 +33,9 @@ StorageID::StorageID(const ASTIdentifier & table_identifier_node) StorageID::StorageID(const ASTPtr & node) { - if (auto identifier = dynamic_cast(node.get())) + if (!node) + *this = StorageID(); + else if (auto identifier = dynamic_cast(node.get())) *this = StorageID(*identifier); else if (auto simple_query = dynamic_cast(node.get())) *this = StorageID(*simple_query); diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 2fb0d2aea35..29d1536b0da 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -191,49 +191,27 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex name += parts.back(); } + ParserKeyword s_uuid("UUID"); + UUID uuid = UUIDHelpers::Nil; + + if (allow_uuid && s_uuid.ignore(pos, expected)) + { + ParserStringLiteral uuid_p; + ASTPtr ast_uuid; + if (!uuid_p.parse(pos, ast_uuid, expected)) + return false; + uuid = parseFromString(ast_uuid->as()->value.get()); + } + if (parts.size() == 1) parts.clear(); node = std::make_shared(name, std::move(parts)); + node->as()->uuid = uuid; return true; } -bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected) -{ - ParserKeyword s_uuid("UUID"); - ParserIdentifier name_p; - ParserStringLiteral uuid_p; - ParserToken s_dot(TokenType::Dot); - - ASTPtr database; - ASTPtr table; - ASTPtr uuid; - - 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 (s_uuid.ignore(pos, expected)) - { - if (!uuid_p.parse(pos, uuid, expected)) - return false; - } - - tryGetIdentifierNameInto(database, res.database_name); - tryGetIdentifierNameInto(table, res.table_name); - //FIXME - res.uuid = uuid ? parseFromString(uuid->as()->value.get()) : UUIDHelpers::Nil; - return true; -} - - bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 34680012808..190d9bc883a 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -55,21 +55,19 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -/** An identifier, possibly containing a dot, for example, x_yz123 or `something special` or Hits.EventTime +/** An identifier, possibly containing a dot, for example, x_yz123 or `something special` or Hits.EventTime, + * possibly with UUID clause like `db name`.`table name` UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx' */ class ParserCompoundIdentifier : public IParserBase { +public: + ParserCompoundIdentifier(bool allow_uuid_ = false) : allow_uuid(allow_uuid_) {} protected: const char * getName() const override { return "compound identifier"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + bool allow_uuid; }; -struct StorageID; -/// Table name, possibly with database name and UUID as string literal -/// [db_name.]table_name [UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx'] -//TODO replace with class -bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected); - /// Just * class ParserAsterisk : public IParserBase { diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 659b755562c..a5e077e7de8 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -338,6 +338,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_table("TABLE"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserCompoundIdentifier table_name_p(true); ParserKeyword s_on("ON"); ParserKeyword s_as("AS"); ParserToken s_dot(TokenType::Dot); @@ -350,7 +351,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserFunction table_function_p; ParserNameList names_p; - StorageID table_id = StorageID::createEmpty(); + ASTPtr table; ASTPtr columns_list; ASTPtr storage; ASTPtr as_database; @@ -381,7 +382,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (s_if_not_exists.ignore(pos, expected)) if_not_exists = true; - if (!parseStorageID(pos, table_id, expected)) + if (!table_name_p.parse(pos, table, expected)) return false; if (s_on.ignore(pos, expected)) @@ -390,6 +391,8 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; } + StorageID table_id{table}; + // Shortcut for ATTACH a previously detached table if (attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon)) { @@ -479,6 +482,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserKeyword s_create("CREATE"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserCompoundIdentifier table_name_p(true); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_live("LIVE"); @@ -490,8 +494,8 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; - StorageID table_id = StorageID::createEmpty(); - StorageID to_table_id = StorageID::createEmpty(); + ASTPtr table; + ASTPtr to_table; ASTPtr columns_list; ASTPtr storage; ASTPtr as_database; @@ -520,7 +524,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (s_if_not_exists.ignore(pos, expected)) if_not_exists = true; - if (!parseStorageID(pos, table_id, expected)) + if (!table_name_p.parse(pos, table, expected)) return false; if (ParserKeyword{"WITH TIMEOUT"}.ignore(pos, expected)) @@ -538,7 +542,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e // TO [db.]table if (ParserKeyword{"TO"}.ignore(pos, expected)) { - if (!parseStorageID(pos, to_table_id, expected)) + if (!table_name_p.parse(pos, to_table, expected)) return false; } @@ -567,12 +571,13 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->if_not_exists = if_not_exists; query->is_live_view = true; + StorageID table_id{table}; query->database = table_id.database_name; query->table = table_id.table_name; query->uuid = table_id.uuid; query->cluster = cluster_str; - query->to_table_id = to_table_id; + query->to_table_id = StorageID(to_table); query->set(query->columns_list, columns_list); @@ -647,7 +652,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_create("CREATE"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserKeyword s_uuid("UUID"); + ParserCompoundIdentifier table_name_p(true); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_materialized("MATERIALIZED"); @@ -661,10 +666,9 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; ParserNameList names_p; - ParserStringLiteral uuid_p; - StorageID table_id = StorageID::createEmpty(); - StorageID to_table_id = StorageID::createEmpty(); + ASTPtr table; + ASTPtr to_table; ASTPtr columns_list; ASTPtr storage; ASTPtr as_database; @@ -706,7 +710,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!replace_view && s_if_not_exists.ignore(pos, expected)) if_not_exists = true; - if (!parseStorageID(pos, table_id, expected)) + if (!table_name_p.parse(pos, table, expected)) return false; if (ParserKeyword{"ON"}.ignore(pos, expected)) @@ -718,7 +722,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec // TO [db.]table if (ParserKeyword{"TO"}.ignore(pos, expected)) { - if (!parseStorageID(pos, to_table_id, expected)) + if (!table_name_p.parse(pos, to_table, expected)) return false; } @@ -732,7 +736,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } - if (is_materialized_view && to_table_id.empty()) + if (is_materialized_view && !to_table) { /// Internal ENGINE for MATERIALIZED VIEW must be specified. if (!storage_p.parse(pos, storage, expected)) @@ -760,12 +764,13 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->is_populate = is_populate; query->replace_view = replace_view; + StorageID table_id{table}; query->database = table_id.database_name; query->table = table_id.table_name; query->uuid = table_id.uuid; query->cluster = cluster_str; - query->to_table_id = to_table_id; + query->to_table_id = StorageID(to_table); query->set(query->columns_list, columns_list); query->set(query->storage, storage); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 1d1cc6c9ecf..02a64897c23 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -423,7 +423,7 @@ void IStorage::renameInMemory(const String & new_database_name, const String & n storage_id.table_name = new_table_name; } -void IStorage::resetStorageIDForTableFunction(const StorageID & actual_table_id) +void IStorage::resetStorageID(const StorageID & actual_table_id) { std::lock_guard lock(id_mutex); storage_id = actual_table_id; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index d9cf2878022..80e11c1c62a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -95,9 +95,10 @@ public: /// The name of the table. StorageID getStorageID() const; - /// This method is required to set actual storage id for storage created by table function. Do not use it + /// This method is required to set actual storage id for storage created by table function. + /// Also it's required for moving table between Ordinary and Atomic databases. Do not use it for other purposes. /// TODO refactor table names in table functions - void resetStorageIDForTableFunction(const StorageID & actual_table_id); + void resetStorageID(const StorageID & actual_table_id); /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 1f9a40d2893..c0a46e1b853 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -33,7 +33,6 @@ limitations under the License. */ #include #include -#include #include #include #include diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 055ecc918f6..339c6b76eb9 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -67,6 +67,7 @@ protected: private: explicit StorageFile(CommonArguments args); + std::string format_name; int table_fd = -1; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index fe8df4b6048..f35688f2b93 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -330,6 +330,7 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name) { auto old_table_id = getStorageID(); + ///FIXME case when moving between DBs if (has_inner_table && tryGetTargetTable() && !old_table_id.hasUUID()) { auto new_target_table_name = generateInnerTableName({new_database_name, new_table_name}); diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index 728ea5b682a..5a35e079a5b 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -10,7 +10,6 @@ namespace DB NamesAndTypesList StorageSystemDatabases::getNamesAndTypes() { - //FIXME data_path return { {"name", std::make_shared()}, {"engine", std::make_shared()}, diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 5eed0126ff6..78783217f31 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -49,7 +49,7 @@ namespace DB { -//FIXME cannot store system tables in DatabaseAtomic +///TODO allow store system tables in DatabaseAtomic void attachSystemTablesLocal(IDatabase & system_database) { system_database.attachTable("one", StorageSystemOne::create("one")); diff --git a/tests/config/access_management.xml b/tests/config/access_management.xml new file mode 100644 index 00000000000..e69de29bb2d From 3bf67b1380688be5fb48bdc004e1eede9e8fd7bd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 7 Apr 2020 15:02:07 +0300 Subject: [PATCH 113/743] typo --- src/Interpreters/InterserverIOHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterserverIOHandler.h b/src/Interpreters/InterserverIOHandler.h index ca3a92df2e6..0e85f30c886 100644 --- a/src/Interpreters/InterserverIOHandler.h +++ b/src/Interpreters/InterserverIOHandler.h @@ -60,7 +60,7 @@ public: throw Exception("Duplicate interserver IO endpoint: " + name, ErrorCodes::DUPLICATE_INTERSERVER_IO_ENDPOINT); } - bool removeEndpointIfExists(const String & name)>> + bool removeEndpointIfExists(const String & name) { std::lock_guard lock(mutex); LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), "anime removeEndpointIfExists() " << name); From 4c48b7dd8001108be31cba41b8248480bf46ebae Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 7 Apr 2020 17:05:51 +0300 Subject: [PATCH 114/743] better rename --- src/Databases/DatabaseAtomic.cpp | 8 ++++---- src/Databases/DatabaseOnDisk.cpp | 5 ++--- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterShowCreateQuery.cpp | 5 +++++ src/Interpreters/StorageID.cpp | 4 +--- src/Parsers/ASTIdentifier.cpp | 14 ++++++++++++++ src/Parsers/ASTIdentifier.h | 2 ++ src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Parsers/ExpressionElementParsers.h | 5 +++-- src/Parsers/ParserCreateQuery.cpp | 12 +++++++----- src/Storages/IStorage.cpp | 11 ++--------- src/Storages/IStorage.h | 13 ++++--------- src/Storages/MergeTree/MergeTreeData.cpp | 6 ++---- src/Storages/MergeTree/MergeTreeData.h | 3 +-- src/Storages/StorageDistributed.cpp | 5 ++--- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageFile.cpp | 4 ++-- src/Storages/StorageFile.h | 2 +- src/Storages/StorageLog.cpp | 4 ++-- src/Storages/StorageLog.h | 6 +----- src/Storages/StorageMaterializedView.cpp | 13 ++++++++----- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 6 ++---- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageSet.cpp | 5 ++--- src/Storages/StorageSet.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 ++-- src/Storages/StorageStripeLog.h | 6 +----- src/Storages/StorageTinyLog.cpp | 4 ++-- src/Storages/StorageTinyLog.h | 6 +----- 31 files changed, 80 insertions(+), 87 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index f57fa16073c..8b6af677f5e 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -121,10 +121,10 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n auto detach = [](DatabaseAtomic & db, const String & table_name_) { - auto table_data_path_ = db.table_name_to_path.find(table_name_)->second; + auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second; db.tables.erase(table_name_); db.table_name_to_path.erase(table_name_); - return table_data_path_; + return table_data_path_saved; }; auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_) @@ -164,9 +164,9 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n if (exchange) other_table_data_path = detach(other_db, to_table_name); - table->renameInMemory(other_db.getDatabaseName(), to_table_name); + table->renameInMemory({other_db.getDatabaseName(), to_table_name, table->getStorageID().uuid}); if (exchange) - other_table->renameInMemory(getDatabaseName(), table_name); + other_table->renameInMemory({getDatabaseName(), table_name, other_table->getStorageID().uuid}); if (!inside_database) { diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 539f2e95a7e..a85adcf5887 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -57,7 +57,7 @@ std::pair createTableFromAST( const auto & table_function = ast_create_query.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table); - storage->resetStorageID({ast_create_query.database, ast_create_query.table, ast_create_query.uuid}); + storage->renameInMemory(ast_create_query); return {ast_create_query.table, storage}; } /// We do not directly use `InterpreterCreateQuery::execute`, because @@ -285,8 +285,7 @@ void DatabaseOnDisk::renameTable( create.uuid = UUIDHelpers::Nil; /// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID - table->rename(to_database.getTableDataPath(create), to_database.getDatabaseName(), to_table_name, table_lock); - table->resetStorageID({create.database, create.table, create.uuid}); /// reset UUID + table->rename(to_database.getTableDataPath(create), StorageID(create)); } catch (const Exception &) { diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index eaee356264d..0d399f1f43a 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -638,7 +638,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper) if (!query_with_table->table.empty()) { /// It's not CREATE DATABASE - auto table_id = context.resolveStorageID(*query_with_table, Context::ResolveOrdinary); + auto table_id = context.tryResolveStorageID(*query_with_table, Context::ResolveOrdinary); storage = DatabaseCatalog::instance().tryGetTable(table_id); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 8d31e8daa87..d66216519f5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -647,7 +647,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, const auto & table_function = create.as_table_function->as(); const auto & factory = TableFunctionFactory::instance(); res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table); - res->resetStorageID({create.database, create.table, create.uuid}); + res->renameInMemory({create.database, create.table, create.uuid}); } else { diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 8bee0b88fe8..a1682650c9e 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { @@ -72,6 +73,10 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (!create_query && show_query && show_query->temporary) throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY); + //FIXME temporary print create query without UUID for tests (remove it) + auto & create = create_query->as(); + create.uuid = UUIDHelpers::Nil; + std::stringstream stream; formatAST(*create_query, stream, false, true); String res = stream.str(); diff --git a/src/Interpreters/StorageID.cpp b/src/Interpreters/StorageID.cpp index bad4bb4837c..ef4e4257ce7 100644 --- a/src/Interpreters/StorageID.cpp +++ b/src/Interpreters/StorageID.cpp @@ -33,9 +33,7 @@ StorageID::StorageID(const ASTIdentifier & table_identifier_node) StorageID::StorageID(const ASTPtr & node) { - if (!node) - *this = StorageID(); - else if (auto identifier = dynamic_cast(node.get())) + if (auto identifier = dynamic_cast(node.get())) *this = StorageID(*identifier); else if (auto simple_query = dynamic_cast(node.get())) *this = StorageID(*simple_query); diff --git a/src/Parsers/ASTIdentifier.cpp b/src/Parsers/ASTIdentifier.cpp index c9d1ca328df..22c650ede92 100644 --- a/src/Parsers/ASTIdentifier.cpp +++ b/src/Parsers/ASTIdentifier.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int UNEXPECTED_AST_STRUCTURE; + extern const int SYNTAX_ERROR; } @@ -163,4 +164,17 @@ void setIdentifierSpecial(ASTPtr & ast) id->semantic->special = true; } +StorageID getTableIdentifier(const ASTPtr & ast) +{ + if (!ast) + throw Exception("AST node is nullptr", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + const auto & identifier = dynamic_cast(*ast); + if (identifier.name_parts.size() > 2) + throw Exception("Logical error: more than two components in table expression", ErrorCodes::SYNTAX_ERROR); + + if (identifier.name_parts.size() == 2) + return { identifier.name_parts[0], identifier.name_parts[1], identifier.uuid }; + return { "", identifier.name, identifier.uuid }; +} + } diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 9e28a1461ca..5ceb4f529a4 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -66,6 +66,7 @@ private: friend struct IdentifierSemantic; friend ASTPtr createTableIdentifier(const StorageID & table_id); friend void setIdentifierSpecial(ASTPtr & ast); + friend StorageID getTableIdentifier(const ASTPtr & ast); }; @@ -78,6 +79,7 @@ void setIdentifierSpecial(ASTPtr & ast); String getIdentifierName(const IAST * ast); std::optional tryGetIdentifierName(const IAST * ast); bool tryGetIdentifierNameInto(const IAST * ast, String & name); +StorageID getTableIdentifier(const ASTPtr & ast); inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); } inline std::optional tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 29d1536b0da..ad8c811872d 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -194,7 +194,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ParserKeyword s_uuid("UUID"); UUID uuid = UUIDHelpers::Nil; - if (allow_uuid && s_uuid.ignore(pos, expected)) + if (table_name_with_optional_uuid && parts.size() <= 2 && s_uuid.ignore(pos, expected)) { ParserStringLiteral uuid_p; ASTPtr ast_uuid; diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 190d9bc883a..ab6aa22a140 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -61,11 +61,12 @@ protected: class ParserCompoundIdentifier : public IParserBase { public: - ParserCompoundIdentifier(bool allow_uuid_ = false) : allow_uuid(allow_uuid_) {} + ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false) + : table_name_with_optional_uuid(table_name_with_optional_uuid_) {} protected: const char * getName() const override { return "compound identifier"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - bool allow_uuid; + bool table_name_with_optional_uuid; }; /// Just * diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index a5e077e7de8..6d8e81edbea 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -391,7 +391,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; } - StorageID table_id{table}; + StorageID table_id = getTableIdentifier(table); // Shortcut for ATTACH a previously detached table if (attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon)) @@ -571,13 +571,14 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->if_not_exists = if_not_exists; query->is_live_view = true; - StorageID table_id{table}; + StorageID table_id = getTableIdentifier(table); query->database = table_id.database_name; query->table = table_id.table_name; query->uuid = table_id.uuid; query->cluster = cluster_str; - query->to_table_id = StorageID(to_table); + if (to_table) + query->to_table_id = getTableIdentifier(to_table); query->set(query->columns_list, columns_list); @@ -764,13 +765,14 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->is_populate = is_populate; query->replace_view = replace_view; - StorageID table_id{table}; + StorageID table_id = getTableIdentifier(table); query->database = table_id.database_name; query->table = table_id.table_name; query->uuid = table_id.uuid; query->cluster = cluster_str; - query->to_table_id = StorageID(to_table); + if (to_table) + query->to_table_id = getTableIdentifier(to_table); query->set(query->columns_list, columns_list); query->set(query->storage, storage); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 02a64897c23..e6f57a3dcc3 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -416,17 +416,10 @@ StorageID IStorage::getStorageID() const return storage_id; } -void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name) +void IStorage::renameInMemory(const StorageID & new_table_id) { std::lock_guard lock(id_mutex); - storage_id.database_name = new_database_name; - storage_id.table_name = new_table_name; -} - -void IStorage::resetStorageID(const StorageID & actual_table_id) -{ - std::lock_guard lock(id_mutex); - storage_id = actual_table_id; + storage_id = new_table_id; } } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 80e11c1c62a..7c1f5e2e85e 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -95,11 +95,6 @@ public: /// The name of the table. StorageID getStorageID() const; - /// This method is required to set actual storage id for storage created by table function. - /// Also it's required for moving table between Ordinary and Atomic databases. Do not use it for other purposes. - /// TODO refactor table names in table functions - void resetStorageID(const StorageID & actual_table_id); - /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } @@ -328,18 +323,18 @@ public: * Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately. * In this function, you need to rename the directory with the data, if any. * Called when the table structure is locked for write. + * Table UUID must remain unchanged, unless table moved between Ordinary and Atomic databases. */ - virtual void rename(const String & /*new_path_to_table_data*/, const String & new_database_name, const String & new_table_name, - TableStructureWriteLockHolder &) + virtual void rename(const String & /*new_path_to_table_data*/, const StorageID & new_table_id) { - renameInMemory(new_database_name, new_table_name); + renameInMemory(new_table_id); } /** * Just updates names of database and table without moving any data on disk * Can be called directly only from DatabaseAtomic. */ - virtual void renameInMemory(const String & new_database_name, const String & new_table_name); + virtual void renameInMemory(const StorageID & new_table_id); /** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters. * This method must fully execute the ALTER query, taking care of the locks itself. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 35a7fc79548..692952485ba 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1298,9 +1298,7 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re } } -void MergeTreeData::rename( - const String & new_table_path, const String & new_database_name, - const String & new_table_name, TableStructureWriteLockHolder &) +void MergeTreeData::rename(const String & new_table_path, const StorageID & new_table_id) { auto disks = getStoragePolicy()->getDisks(); @@ -1320,7 +1318,7 @@ void MergeTreeData::rename( global_context.dropCaches(); relative_data_path = new_table_path; - renameInMemory(new_database_name, new_table_name); + renameInMemory(new_table_id); } void MergeTreeData::dropAllData() diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fbc42de5517..624ca2cace0 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -511,8 +511,7 @@ public: /// Moves the entire data directory. /// Flushes the uncompressed blocks cache and the marks cache. /// Must be called with locked lockStructureForAlter(). - void rename(const String & new_table_path, const String & new_database_name, - const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_table_path, const StorageID & new_table_id) override; /// Check if the ALTER can be performed: /// - all needed columns are present. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1283cbce3f4..0fbb6741641 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -717,12 +717,11 @@ void StorageDistributed::flushClusterNodesAllData() node.second.flushAllData(); } -void StorageDistributed::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, - TableStructureWriteLockHolder &) +void StorageDistributed::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { if (!relative_data_path.empty()) renameOnDisk(new_path_to_table_data); - renameInMemory(new_database_name, new_table_name); + renameInMemory(new_table_id); } void StorageDistributed::renameOnDisk(const String & new_path_to_table_data) { diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 23f9f991dfe..8909d2470c5 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -82,7 +82,7 @@ public: /// Removes temporary data in local filesystem. void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; void renameOnDisk(const String & new_path_to_table_data); void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index b68a88fed00..446ded1e008 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -497,7 +497,7 @@ Strings StorageFile::getDataPaths() const return paths; } -void StorageFile::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageFile::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { if (!is_db_table) throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); @@ -512,7 +512,7 @@ void StorageFile::rename(const String & new_path_to_table_data, const String & n Poco::File(paths[0]).renameTo(path_new); paths[0] = std::move(path_new); - renameInMemory(new_database_name, new_table_name); + renameInMemory(new_table_id); } void StorageFile::truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 339c6b76eb9..815e6fffbe8 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -38,7 +38,7 @@ public: void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) override; - void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; Strings getDataPaths() const override; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index eb623561297..bc2bbb2ce67 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -514,7 +514,7 @@ void StorageLog::loadMarks() } -void StorageLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { std::unique_lock lock(rwlock); @@ -527,7 +527,7 @@ void StorageLog::rename(const String & new_path_to_table_data, const String & ne file.second.data_file_path = table_path + fileName(file.second.data_file_path); marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; - renameInMemory(new_database_name, new_table_name); + renameInMemory(new_table_id); } void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 79d7c960004..2c2abdb0275 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -34,11 +34,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename( - const String & new_path_to_table_data, - const String & new_database_name, - const String & new_table_name, - TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index f35688f2b93..190db614c49 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -327,13 +327,16 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co getTargetTable()->mutate(commands, context); } -void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name) +void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) { auto old_table_id = getStorageID(); - ///FIXME case when moving between DBs - if (has_inner_table && tryGetTargetTable() && !old_table_id.hasUUID()) + auto old_target_table_name = generateInnerTableName(old_table_id); + auto new_target_table_name = generateInnerTableName(new_table_id); + bool inner_same_name = old_table_id.database_name == new_table_id.database_name && + old_target_table_name == new_target_table_name; + + if (has_inner_table && tryGetTargetTable() && !inner_same_name) { - auto new_target_table_name = generateInnerTableName({new_database_name, new_table_name}); auto rename = std::make_shared(); ASTRenameQuery::Table from; @@ -353,7 +356,7 @@ void StorageMaterializedView::renameInMemory(const String & new_database_name, c target_table_id.table_name = new_target_table_name; } - IStorage::renameInMemory(new_database_name, new_table_name); + IStorage::renameInMemory(new_table_id); // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated DatabaseCatalog::instance().updateDependency(select_table_id, old_table_id, select_table_id, getStorageID()); } diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 3dc04900952..0f291646b0f 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -52,7 +52,7 @@ public: void mutate(const MutationCommands & commands, const Context & context) override; - void renameInMemory(const String & new_database_name, const String & new_table_name) override; + void renameInMemory(const StorageID & new_table_id) override; void shutdown() override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 455ee6fd86c..7c1c5e6a8a4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3624,11 +3624,9 @@ void StorageReplicatedMergeTree::drop() } -void StorageReplicatedMergeTree::rename( - const String & new_path_to_table_data, const String & new_database_name, - const String & new_table_name, TableStructureWriteLockHolder & lock) +void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - MergeTreeData::rename(new_path_to_table_data, new_database_name, new_table_name, lock); + MergeTreeData::rename(new_path_to_table_data, new_table_id); /// Update table name in zookeeper auto zookeeper = getZooKeeper(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index f635b4f7bc0..e0ce3405b9e 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -117,7 +117,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; - void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; bool supportsIndexForIn() const override { return true; } diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 72ae46787c8..572e118d892 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -197,15 +197,14 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) } -void StorageSetOrJoinBase::rename( - const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageSetOrJoinBase::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { /// Rename directory with data. String new_path = base_path + new_path_to_table_data; Poco::File(path).renameTo(new_path); path = new_path; - renameInMemory(new_database_name, new_table_name); + renameInMemory(new_table_id); } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index 39c1e3b7952..cf85dfd5d5b 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -19,7 +19,7 @@ class StorageSetOrJoinBase : public IStorage friend class SetOrJoinBlockOutputStream; public: - void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 445c16d73de..5c3e93fc7f8 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -237,7 +237,7 @@ StorageStripeLog::StorageStripeLog( } -void StorageStripeLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { std::unique_lock lock(rwlock); @@ -245,7 +245,7 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Strin table_path = new_path_to_table_data; file_checker.setPath(table_path + "sizes.json"); - renameInMemory(new_database_name, new_table_name); + renameInMemory(new_table_id); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 2ca4f69e742..c85f576bd43 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -35,11 +35,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename( - const String & new_path_to_table_data, - const String & new_database_name, - const String & new_table_name, - TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 400780cb072..5bdfa1fc374 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -375,7 +375,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type } -void StorageTinyLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) +void StorageTinyLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { std::unique_lock lock(rwlock); @@ -386,7 +386,7 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const String for (auto & file : files) file.second.data_file_path = table_path + fileName(file.second.data_file_path); - renameInMemory(new_database_name, new_table_name); + renameInMemory(new_table_id); } diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index 456fee0ef05..a8be3be2435 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -34,11 +34,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename( - const String & new_path_to_table_data, - const String & new_database_name, - const String & new_table_name, - TableStructureWriteLockHolder &) override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override; From 135197aa75847d0659187f8f414de7042326afe8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 Apr 2020 04:02:00 +0300 Subject: [PATCH 115/743] fixes --- src/Common/rename.cpp | 6 --- src/Common/rename.h | 10 +++++ src/Core/Settings.h | 1 + src/Databases/DatabaseAtomic.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 19 ++++----- src/Interpreters/DatabaseCatalog.h | 6 +-- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++ src/Parsers/ASTInsertQuery.cpp | 3 +- src/Storages/StorageMaterializedView.cpp | 39 ++++++++++--------- tests/integration/test_ttl_replicated/test.py | 3 +- .../01013_sync_replica_timeout_zookeeper.sh | 2 +- ...034_move_partition_from_table_zookeeper.sh | 20 ++++++---- ...ent_move_partition_from_table_zookeeper.sh | 2 +- 13 files changed, 67 insertions(+), 50 deletions(-) diff --git a/src/Common/rename.cpp b/src/Common/rename.cpp index 72bd5d1bb60..039c4304f69 100644 --- a/src/Common/rename.cpp +++ b/src/Common/rename.cpp @@ -54,17 +54,11 @@ namespace ErrorCodes #endif -#if !defined(__NR_renameat2) -[[noreturn]] -#endif void renameNoReplace(const std::string & old_path, const std::string & new_path) { renameat2(old_path, new_path, RENAME_NOREPLACE); } -#if !defined(__NR_renameat2) -[[noreturn]] -#endif void renameExchange(const std::string & old_path, const std::string & new_path) { renameat2(old_path, new_path, RENAME_EXCHANGE); diff --git a/src/Common/rename.h b/src/Common/rename.h index 1b42754507e..36e9fb7c73e 100644 --- a/src/Common/rename.h +++ b/src/Common/rename.h @@ -1,13 +1,23 @@ #pragma once #include +#if defined(_GNU_SOURCE) +#include +#endif + namespace DB { /// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception +#if !defined(__NR_renameat2) +[[noreturn]] +#endif void renameNoReplace(const std::string & old_path, const std::string & new_path); /// Atomically exchange oldpath and newpath. Throw exception if some of them does not exist +#if !defined(__NR_renameat2) +[[noreturn]] +#endif void renameExchange(const std::string & old_path, const std::string & new_path); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ba74d5eee2b..a7c6ad5c0be 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -394,6 +394,7 @@ struct Settings : public SettingsCollection 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(SettingDefaultDatabaseEngine, default_database_engine, /*DefaultDatabaseEngine::Ordinary*/ DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \ + M(SettingBool, allow_experimental_database_atomic, /*false*/ true, "Allow to create database with Engine=Atomic.", 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) \ M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(SettingUInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 8b6af677f5e..2117680d4d9 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator { public: - AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) + explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) : DatabaseTablesSnapshotIterator(std::move(base)) {} UUID uuid() const override { return table()->getStorageID().uuid; } }; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index bbe84baa466..4ec9b07e223 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -376,6 +376,7 @@ DatabaseCatalog::DatabaseCatalog(Context * global_context_) { if (!global_context) throw Exception("DatabaseCatalog is not initialized. It's a bug.", ErrorCodes::LOGICAL_ERROR); + drop_delay_s = global_context->getConfigRef().getInt("database_atomic_delay_before_drop_table_s", 60); } DatabaseCatalog & DatabaseCatalog::init(Context * global_context_) @@ -551,11 +552,11 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr drop_time = Poco::File(dropped_metadata_path).getLastModified().epochTime(); } - std::lock_guard lock(tables_marked_droped_mutex); + std::lock_guard lock(tables_marked_dropped_mutex); if (ignore_delay) - tables_marked_droped.push_front({table_id, table, dropped_metadata_path, 0}); + tables_marked_dropped.push_front({table_id, table, dropped_metadata_path, 0}); else - tables_marked_droped.push_back({table_id, table, dropped_metadata_path, drop_time}); + tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time}); } void DatabaseCatalog::dropTableDataTask() @@ -563,19 +564,19 @@ void DatabaseCatalog::dropTableDataTask() TableMarkedAsDropped table; try { - std::lock_guard lock(tables_marked_droped_mutex); + std::lock_guard lock(tables_marked_dropped_mutex); time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - auto it = std::find_if(tables_marked_droped.begin(), tables_marked_droped.end(), [current_time](const auto & elem) + auto it = std::find_if(tables_marked_dropped.begin(), tables_marked_dropped.end(), [&](const auto & elem) { bool not_in_use = !elem.table || elem.table.unique(); bool old_enough = elem.drop_time + drop_delay_s < current_time; return not_in_use && old_enough; }); - if (it != tables_marked_droped.end()) + if (it != tables_marked_dropped.end()) { table = std::move(*it); LOG_INFO(log, "Will try drop " + table.table_id.getNameForLogs()); - tables_marked_droped.erase(it); + tables_marked_dropped.erase(it); } } catch (...) @@ -595,8 +596,8 @@ void DatabaseCatalog::dropTableDataTask() tryLogCurrentException(log, "Cannot drop table " + table.table_id.getNameForLogs() + ". Will retry later."); { - std::lock_guard lock(tables_marked_droped_mutex); - tables_marked_droped.emplace_back(std::move(table)); + std::lock_guard lock(tables_marked_dropped_mutex); + tables_marked_dropped.emplace_back(std::move(table)); } } } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index cce83970876..338a81851ba 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -194,7 +194,6 @@ private: void dropTableDataTask(); void dropTableFinally(const TableMarkedAsDropped & table) const; - static constexpr time_t drop_delay_s = 10; static constexpr size_t reschedule_time_ms = 100; private: @@ -219,10 +218,11 @@ private: /// If you capture mutex and ddl_guards_mutex, then you need to grab them strictly in this order. mutable std::mutex ddl_guards_mutex; - TablesMarkedAsDropped tables_marked_droped; - mutable std::mutex tables_marked_droped_mutex; + TablesMarkedAsDropped tables_marked_dropped; + mutable std::mutex tables_marked_dropped_mutex; std::unique_ptr drop_task; + time_t drop_delay_s; }; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d66216519f5..fd9ac8fa5af 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -102,6 +102,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) engine->name = old_style_database ? "Ordinary" : "Atomic"; storage->set(storage->engine, engine); create.set(create.storage, storage); + + if (!context.getSettingsRef().allow_experimental_database_atomic) + throw Exception("Atomic is an experimental database engine. Enable allow_experimental_database_atomic to use it.", + ErrorCodes::UNKNOWN_DATABASE_ENGINE); } else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) { diff --git a/src/Parsers/ASTInsertQuery.cpp b/src/Parsers/ASTInsertQuery.cpp index 37e94c49668..5945d2752f3 100644 --- a/src/Parsers/ASTInsertQuery.cpp +++ b/src/Parsers/ASTInsertQuery.cpp @@ -26,8 +26,7 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s } else settings.ostr << (settings.hilite ? hilite_none : "") - << (!table_id.database_name.empty() ? backQuoteIfNeed(table_id.database_name) + "." : "") << backQuoteIfNeed(table_id.table_name) - << (table_id.hasUUID() ? " UUID " : "") << (table_id.hasUUID() ? quoteString(toString(table_id.uuid)) : ""); + << (!table_id.database_name.empty() ? backQuoteIfNeed(table_id.database_name) + "." : "") << backQuoteIfNeed(table_id.table_name); if (columns) { diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 190db614c49..b22165d50b4 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -330,30 +330,33 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) { auto old_table_id = getStorageID(); - auto old_target_table_name = generateInnerTableName(old_table_id); - auto new_target_table_name = generateInnerTableName(new_table_id); - bool inner_same_name = old_table_id.database_name == new_table_id.database_name && - old_target_table_name == new_target_table_name; - if (has_inner_table && tryGetTargetTable() && !inner_same_name) + if (has_inner_table && tryGetTargetTable()) { - auto rename = std::make_shared(); + auto old_target_table_name = generateInnerTableName(old_table_id); + auto new_target_table_name = generateInnerTableName(new_table_id); + if (old_table_id.database_name != new_table_id.database_name || + old_target_table_name != new_target_table_name) + { - ASTRenameQuery::Table from; - from.database = target_table_id.database_name; - from.table = target_table_id.table_name; + auto rename = std::make_shared(); - ASTRenameQuery::Table to; - to.database = target_table_id.database_name; - to.table = new_target_table_name; + ASTRenameQuery::Table from; + from.database = target_table_id.database_name; + from.table = target_table_id.table_name; - ASTRenameQuery::Element elem; - elem.from = from; - elem.to = to; - rename->elements.emplace_back(elem); + ASTRenameQuery::Table to; + to.database = target_table_id.database_name; + to.table = new_target_table_name; - InterpreterRenameQuery(rename, global_context).execute(); - target_table_id.table_name = new_target_table_name; + ASTRenameQuery::Element elem; + elem.from = from; + elem.to = to; + rename->elements.emplace_back(elem); + + InterpreterRenameQuery(rename, global_context).execute(); + target_table_id.table_name = new_target_table_name; + } } IStorage::renameInMemory(new_table_id); diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 03dd1d2aa89..78ff703f0ec 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -24,7 +24,8 @@ def started_cluster(): def drop_table(nodes, table_name): for node in nodes: - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE IF EXISTS {} NO DELAY".format(table_name)) + time.sleep(1) def test_ttl_columns(started_cluster): drop_table([node1, node2], "test_ttl") diff --git a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh index 9e846b42591..c0bc7e3574d 100755 --- a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh +++ b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh @@ -21,7 +21,7 @@ ${CLICKHOUSE_CLIENT} -n -q " timeout 10s ${CLICKHOUSE_CLIENT} -n -q " SET receive_timeout=1; SYSTEM SYNC REPLICA $R2 -" 2>&1 | fgrep -q "DB::Exception: SYNC REPLICA ${CLICKHOUSE_DATABASE}.$R2: command timed out!" && echo 'OK' || echo 'Failed!' +" 2>&1 | fgrep -q "Code: 159. DB::Exception" && echo 'OK' || echo 'Failed!' # By dropping tables all related SYNC REPLICA queries would be terminated as well ${CLICKHOUSE_CLIENT} -n -q " diff --git a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh index 9e077d673ee..aee0da15a67 100755 --- a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh @@ -22,8 +22,9 @@ function query_with_retry echo "Query '$1' failed with '$result'" } -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.dst;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.src NO DELAY;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.dst NO DELAY;" +sleep 1 $CLICKHOUSE_CLIENT --query="CREATE TABLE $CLICKHOUSE_DATABASE.src (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_DATABASE/src', '1') PARTITION BY p ORDER BY k;" $CLICKHOUSE_CLIENT --query="CREATE TABLE $CLICKHOUSE_DATABASE.dst (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_DATABASE/dst', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" @@ -50,8 +51,9 @@ $CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA $CLICKHOUSE_DATABASE.dst;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM $CLICKHOUSE_DATABASE.src;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM $CLICKHOUSE_DATABASE.dst;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.dst;" +$CLICKHOUSE_CLIENT --query="DROP TABLE $CLICKHOUSE_DATABASE.src NO DELAY;" +$CLICKHOUSE_CLIENT --query="DROP TABLE $CLICKHOUSE_DATABASE.dst NO DELAY;" +sleep 1 $CLICKHOUSE_CLIENT --query="SELECT 'MOVE incompatible schema missing column';" @@ -69,8 +71,9 @@ $CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA $CLICKHOUSE_DATABASE.dst;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM $CLICKHOUSE_DATABASE.src;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM $CLICKHOUSE_DATABASE.dst;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.dst;" +$CLICKHOUSE_CLIENT --query="DROP TABLE $CLICKHOUSE_DATABASE.src NO DELAY;" +$CLICKHOUSE_CLIENT --query="DROP TABLE $CLICKHOUSE_DATABASE.dst NO DELAY;" +sleep 1 $CLICKHOUSE_CLIENT --query="SELECT 'MOVE incompatible schema different order by';" @@ -89,6 +92,7 @@ $CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA $CLICKHOUSE_DATABASE.dst;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM $CLICKHOUSE_DATABASE.src;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM $CLICKHOUSE_DATABASE.dst;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS $CLICKHOUSE_DATABASE.dst;" +$CLICKHOUSE_CLIENT --query="DROP TABLE $CLICKHOUSE_DATABASE.src NO DELAY;" +$CLICKHOUSE_CLIENT --query="DROP TABLE $CLICKHOUSE_DATABASE.dst NO DELAY;" +sleep 1 diff --git a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh index bafa3a95b3b..5495feb89db 100755 --- a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh @@ -60,9 +60,9 @@ wait echo "DROP TABLE $CLICKHOUSE_DATABASE.src NO DELAY" | ${CLICKHOUSE_CLIENT} echo "DROP TABLE $CLICKHOUSE_DATABASE.dst NO DELAY" | ${CLICKHOUSE_CLIENT} +sleep 5 # Check for deadlocks echo "SELECT * FROM system.processes WHERE query_id LIKE 'query%'" | ${CLICKHOUSE_CLIENT} echo 'did not crash' -sleep 1 From 5b2b8d38fa50fd8b1f195c5890be8c103ad65b61 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 8 Apr 2020 11:41:13 +0300 Subject: [PATCH 116/743] Download part trough disk interface. --- src/Storages/MergeTree/DataPartsExchange.cpp | 29 ++++++++++---------- src/Storages/MergeTree/MergeTreeData.h | 3 ++ 2 files changed, 18 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 6373c85a15d..4e40d4a5977 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -258,19 +258,20 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( size_t files; readBinary(files, in); + auto disk = reservation->getDisk(); + static const String TMP_PREFIX = "tmp_fetch_"; String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_; - String relative_part_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name; - String absolute_part_path = Poco::Path(data.getFullPathOnDisk(reservation->getDisk()) + relative_part_path + "/").absolute().toString(); - Poco::File part_file(absolute_part_path); + String part_relative_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name; + String part_download_path = data.getRelativeDataPath() + part_relative_path + "/"; - if (part_file.exists()) - throw Exception("Directory " + absolute_part_path + " already exists.", ErrorCodes::DIRECTORY_ALREADY_EXISTS); + if (disk->exists(part_download_path)) + throw Exception("Directory " + fullPath(disk, part_download_path) + " already exists.", ErrorCodes::DIRECTORY_ALREADY_EXISTS); CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedFetch}; - part_file.createDirectory(); + disk->createDirectories(part_download_path); MergeTreeData::DataPart::Checksums checksums; for (size_t i = 0; i < files; ++i) @@ -283,21 +284,21 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( /// File must be inside "absolute_part_path" directory. /// Otherwise malicious ClickHouse replica may force us to write to arbitrary path. - String absolute_file_path = Poco::Path(absolute_part_path + file_name).absolute().toString(); - if (!startsWith(absolute_file_path, absolute_part_path)) - throw Exception("File path (" + absolute_file_path + ") doesn't appear to be inside part path (" + absolute_part_path + ")." + String absolute_file_path = Poco::Path(part_download_path + file_name).absolute().toString(); + if (!startsWith(absolute_file_path, part_download_path)) + throw Exception("File path (" + absolute_file_path + ") doesn't appear to be inside part path (" + part_download_path + ")." " This may happen if we are trying to download part from malicious replica or logical error.", ErrorCodes::INSECURE_PATH); - WriteBufferFromFile file_out(absolute_file_path); - HashingWriteBuffer hashing_out(file_out); + auto file_out = disk->writeFile(part_download_path + file_name); + HashingWriteBuffer hashing_out(*file_out); copyData(in, hashing_out, file_size, blocker.getCounter()); if (blocker.isCancelled()) { /// NOTE The is_cancelled flag also makes sense to check every time you read over the network, performing a poll with a not very large timeout. /// And now we check it only between read chunks (in the `copyData` function). - part_file.remove(true); + disk->removeRecursive(part_download_path); throw Exception("Fetching of part was cancelled", ErrorCodes::ABORTED); } @@ -305,7 +306,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( readPODBinary(expected_hash, in); if (expected_hash != hashing_out.getHash()) - throw Exception("Checksum mismatch for file " + absolute_part_path + file_name + " transferred from " + replica_path, + throw Exception("Checksum mismatch for file " + fullPath(disk, part_download_path + file_name) + " transferred from " + replica_path, ErrorCodes::CHECKSUM_DOESNT_MATCH); if (file_name != "checksums.txt" && @@ -315,7 +316,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( assertEOF(in); - MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, reservation->getDisk(), relative_part_path); + MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, reservation->getDisk(), part_relative_path); new_data_part->is_temp = true; new_data_part->modification_time = time(nullptr); new_data_part->loadColumnsChecksumsIndexes(true, false); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 125a90d26e0..eb2a0dd8774 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -625,6 +625,9 @@ public: return storage_settings.get(); } + /// Get relative table path + String getRelativeDataPath() const { return relative_data_path; } + /// Get table path on disk String getFullPathOnDisk(const DiskPtr & disk) const; From 4ec77fee8b5a49d2f98659a9c817a8c47bd866e9 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 8 Apr 2020 13:53:17 +0300 Subject: [PATCH 117/743] Integration test for ReplicatedMergeTree over S3. --- .../test_replicated_merge_tree_s3/__init__.py | 0 .../config.d/bg_processing_pool_conf.xml | 5 + .../configs/config.d/log_conf.xml | 12 ++ .../configs/config.d/storage_conf.xml | 28 +++++ .../configs/config.d/users.xml | 6 + .../configs/config.xml | 20 ++++ .../test_replicated_merge_tree_s3/test.py | 108 ++++++++++++++++++ 7 files changed, 179 insertions(+) create mode 100644 tests/integration/test_replicated_merge_tree_s3/__init__.py create mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.d/bg_processing_pool_conf.xml create mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.d/log_conf.xml create mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml create mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.d/users.xml create mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.xml create mode 100644 tests/integration/test_replicated_merge_tree_s3/test.py diff --git a/tests/integration/test_replicated_merge_tree_s3/__init__.py b/tests/integration/test_replicated_merge_tree_s3/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/bg_processing_pool_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/bg_processing_pool_conf.xml new file mode 100644 index 00000000000..a756c4434ea --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/bg_processing_pool_conf.xml @@ -0,0 +1,5 @@ + + 0.5 + 0.5 + 0.5 + diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/log_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/log_conf.xml new file mode 100644 index 00000000000..318a6bca95d --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/log_conf.xml @@ -0,0 +1,12 @@ + + 3 + + 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_replicated_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml new file mode 100644 index 00000000000..5b292446c6b --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml @@ -0,0 +1,28 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + local + / + + + + + +
+ s3 +
+ + hdd + +
+
+
+
+
diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/users.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/users.xml new file mode 100644 index 00000000000..a13b24b278d --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/users.xml @@ -0,0 +1,6 @@ + + + + + + diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.xml new file mode 100644 index 00000000000..24b7344df3a --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.xml @@ -0,0 +1,20 @@ + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py new file mode 100644 index 00000000000..53eb612c281 --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -0,0 +1,108 @@ +import logging +import random +import string +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +logging.getLogger().setLevel(logging.INFO) +logging.getLogger().addHandler(logging.StreamHandler()) + + +# Creates S3 bucket for tests and allows anonymous read-write access to it. +def prepare_s3_bucket(cluster): + minio_client = cluster.minio_client + + if minio_client.bucket_exists(cluster.minio_bucket): + minio_client.remove_bucket(cluster.minio_bucket) + + minio_client.make_bucket(cluster.minio_bucket) + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + + cluster.add_instance("node1", config_dir="configs", with_minio=True, with_zookeeper=True) + cluster.add_instance("node2", config_dir="configs") + cluster.add_instance("node3", config_dir="configs") + + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + yield cluster + finally: + cluster.shutdown() + + +FILES_OVERHEAD = 1 +FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files +FILES_OVERHEAD_PER_PART = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + + +def random_string(length): + letters = string.ascii_letters + return ''.join(random.choice(letters) for i in range(length)) + + +def generate_values(date_str, count, sign=1): + data = [[date_str, sign*(i + 1), random_string(10)] for i in range(count)] + data.sort(key=lambda tup: tup[1]) + return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data]) + + +def create_table(cluster): + create_table_statement = """ + CREATE TABLE s3_test ( + dt Date, + id Int64, + data String, + INDEX min_max (id) TYPE minmax GRANULARITY 3 + ) ENGINE=ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/test_mutations', '{instance}') + PARTITION BY dt + ORDER BY (dt, id) + SETTINGS + old_parts_lifetime=0, index_granularity=512 + """ + + for node in cluster.instances: + node.query(create_table_statement) + + +@pytest.fixture(autouse=True) +def drop_table(cluster): + yield + for node in cluster.instances: + node.query("DROP TABLE IF EXISTS s3_test") + + minio = cluster.minio_client + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 + + +def test_insert_select_replicated(cluster): + create_table(cluster) + + all_values = "" + for node_idx in range(1, 4): + node = cluster.instances["node" + str(node_idx)] + values = generate_values("2020-01-0" + str(node_idx), 4096) + node.query("INSERT INTO s3_test VALUES {}".format(values)) + if node_idx != 1: + all_values += "," + all_values += values + + # Wait for replication + time.sleep(10) + + for node_idx in range(1, 4): + node = cluster.instances["node" + str(node_idx)] + assert node.query("SELECT * FROM s3_test order by dt, id FORMAT Values") == all_values + + minio = cluster.minio_client + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 3 * (FILES_OVERHEAD + FILES_OVERHEAD_PER_PART * 3) From 89fe81ed62227f9643881389e8e393da25d19a98 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 8 Apr 2020 14:56:31 +0300 Subject: [PATCH 118/743] Integration test for ReplicatedMergeTree over S3 fixes. --- src/Storages/MergeTree/DataPartsExchange.cpp | 3 +-- .../configs/config.xml | 20 --------------- .../test_replicated_merge_tree_s3/test.py | 25 ++++++++----------- 3 files changed, 12 insertions(+), 36 deletions(-) delete mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.xml diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 4e40d4a5977..9ef7a4d37aa 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -254,7 +254,6 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in) { - size_t files; readBinary(files, in); @@ -285,7 +284,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( /// File must be inside "absolute_part_path" directory. /// Otherwise malicious ClickHouse replica may force us to write to arbitrary path. String absolute_file_path = Poco::Path(part_download_path + file_name).absolute().toString(); - if (!startsWith(absolute_file_path, part_download_path)) + if (!startsWith(absolute_file_path, Poco::Path(part_download_path).absolute().toString())) throw Exception("File path (" + absolute_file_path + ") doesn't appear to be inside part path (" + part_download_path + ")." " This may happen if we are trying to download part from malicious replica or logical error.", ErrorCodes::INSECURE_PATH); diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.xml deleted file mode 100644 index 24b7344df3a..00000000000 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.xml +++ /dev/null @@ -1,20 +0,0 @@ - - - 9000 - 127.0.0.1 - - - - true - none - - AcceptCertificateHandler - - - - - 500 - 5368709120 - ./clickhouse/ - users.xml - diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 53eb612c281..118a43a905e 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -25,9 +25,9 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node1", config_dir="configs", with_minio=True, with_zookeeper=True) - cluster.add_instance("node2", config_dir="configs") - cluster.add_instance("node3", config_dir="configs") + cluster.add_instance("node1", config_dir="configs", macros={'cluster': 'test1'}, with_minio=True, with_zookeeper=True) + cluster.add_instance("node2", config_dir="configs", macros={'cluster': 'test1'}, with_zookeeper=True) + cluster.add_instance("node3", config_dir="configs", macros={'cluster': 'test1'}, with_zookeeper=True) logging.info("Starting cluster...") cluster.start() @@ -64,25 +64,25 @@ def create_table(cluster): id Int64, data String, INDEX min_max (id) TYPE minmax GRANULARITY 3 - ) ENGINE=ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/test_mutations', '{instance}') + ) ENGINE=ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/s3', '{instance}') PARTITION BY dt ORDER BY (dt, id) SETTINGS old_parts_lifetime=0, index_granularity=512 """ - for node in cluster.instances: + for node in cluster.instances.values(): node.query(create_table_statement) @pytest.fixture(autouse=True) def drop_table(cluster): yield - for node in cluster.instances: - node.query("DROP TABLE IF EXISTS s3_test") + #for node in cluster.instances.values(): + # node.query("DROP TABLE IF EXISTS s3_test") - minio = cluster.minio_client - assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 + #minio = cluster.minio_client + #assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 def test_insert_select_replicated(cluster): @@ -92,17 +92,14 @@ def test_insert_select_replicated(cluster): for node_idx in range(1, 4): node = cluster.instances["node" + str(node_idx)] values = generate_values("2020-01-0" + str(node_idx), 4096) - node.query("INSERT INTO s3_test VALUES {}".format(values)) + node.query("INSERT INTO s3_test VALUES {}".format(values), settings={"insert_quorum": 3}) if node_idx != 1: all_values += "," all_values += values - # Wait for replication - time.sleep(10) - for node_idx in range(1, 4): node = cluster.instances["node" + str(node_idx)] - assert node.query("SELECT * FROM s3_test order by dt, id FORMAT Values") == all_values + assert node.query("SELECT * FROM s3_test order by dt, id FORMAT Values", settings={"select_sequential_consistency": 1}) == all_values minio = cluster.minio_client assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 3 * (FILES_OVERHEAD + FILES_OVERHEAD_PER_PART * 3) From eca6caa8db919f63d6dd61fc77e6f7dc67dad0f2 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 8 Apr 2020 15:48:16 +0300 Subject: [PATCH 119/743] Integration tests for MergeTree over S3 code cleanup. --- tests/integration/test_merge_tree_s3/test.py | 12 ++++++++---- .../configs/config.d/bg_processing_pool_conf.xml | 5 ----- .../configs/config.d/log_conf.xml | 12 ------------ .../configs/config.d/users.xml | 6 ------ .../test_replicated_merge_tree_s3/test.py | 15 +++++++++------ 5 files changed, 17 insertions(+), 33 deletions(-) delete mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.d/bg_processing_pool_conf.xml delete mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.d/log_conf.xml delete mode 100644 tests/integration/test_replicated_merge_tree_s3/configs/config.d/users.xml diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f69c09631e8..e12e31ebff2 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -84,7 +84,12 @@ def drop_table(cluster): minio = cluster.minio_client node.query("DROP TABLE IF EXISTS s3_test") - assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 + try: + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 + finally: + # Remove extra objects to prevent tests cascade failing + for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): + minio.remove_object(cluster.minio_bucket, obj.object_name) @pytest.mark.parametrize( @@ -210,7 +215,7 @@ def test_attach_detach_partition(cluster): assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE node.query("ALTER TABLE s3_test DETACH PARTITION '2020-01-04'") - node.query("SET allow_drop_detached=1; ALTER TABLE s3_test DROP DETACHED PARTITION '2020-01-04'") + node.query("ALTER TABLE s3_test DROP DETACHED PARTITION '2020-01-04'", settings={"allow_drop_detached": 1}) assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(0)" assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD @@ -245,8 +250,7 @@ def test_table_manipulations(cluster): assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE*2 node.query("RENAME TABLE s3_renamed TO s3_test") - # TODO: Doesn't work with min_max index. - #assert node.query("SET check_query_single_value_result='false'; CHECK TABLE s3_test FORMAT Values") == "(1)" + assert node.query("CHECK TABLE s3_test FORMAT Values") == "(1)" node.query("DETACH TABLE s3_test") node.query("ATTACH TABLE s3_test") diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/bg_processing_pool_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/bg_processing_pool_conf.xml deleted file mode 100644 index a756c4434ea..00000000000 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/bg_processing_pool_conf.xml +++ /dev/null @@ -1,5 +0,0 @@ - - 0.5 - 0.5 - 0.5 - diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/log_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/log_conf.xml deleted file mode 100644 index 318a6bca95d..00000000000 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/log_conf.xml +++ /dev/null @@ -1,12 +0,0 @@ - - 3 - - 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_replicated_merge_tree_s3/configs/config.d/users.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/users.xml deleted file mode 100644 index a13b24b278d..00000000000 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/users.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - - - - diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 118a43a905e..a8b7cf63e38 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -67,8 +67,6 @@ def create_table(cluster): ) ENGINE=ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/s3', '{instance}') PARTITION BY dt ORDER BY (dt, id) - SETTINGS - old_parts_lifetime=0, index_granularity=512 """ for node in cluster.instances.values(): @@ -78,11 +76,16 @@ def create_table(cluster): @pytest.fixture(autouse=True) def drop_table(cluster): yield - #for node in cluster.instances.values(): - # node.query("DROP TABLE IF EXISTS s3_test") + for node in cluster.instances.values(): + node.query("DROP TABLE IF EXISTS s3_test") - #minio = cluster.minio_client - #assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 + minio = cluster.minio_client + try: + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 + finally: + # Remove extra objects to prevent tests cascade failing + for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): + minio.remove_object(cluster.minio_bucket, obj.object_name) def test_insert_select_replicated(cluster): From 8d6f6417964726c97a287a02d409aea6cbc3c49a Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 8 Apr 2020 21:59:52 +0300 Subject: [PATCH 120/743] join over dictionary concept --- src/Interpreters/DictionaryReader.h | 188 ++++++++++++++++++++ src/Interpreters/ExpressionAnalyzer.cpp | 87 ++++----- src/Interpreters/ExpressionAnalyzer.h | 2 - src/Interpreters/HashJoin.cpp | 109 ++++++++++-- src/Interpreters/HashJoin.h | 9 +- src/Interpreters/InterpreterSelectQuery.cpp | 3 +- src/Interpreters/JoinedTables.cpp | 64 +++++++ src/Interpreters/JoinedTables.h | 2 + src/Interpreters/SyntaxAnalyzer.cpp | 40 +---- src/Interpreters/SyntaxAnalyzer.h | 3 +- src/Interpreters/TableJoin.h | 12 +- src/Storages/StorageDictionary.h | 2 + 12 files changed, 424 insertions(+), 97 deletions(-) create mode 100644 src/Interpreters/DictionaryReader.h diff --git a/src/Interpreters/DictionaryReader.h b/src/Interpreters/DictionaryReader.h new file mode 100644 index 00000000000..c1cd3e9ef1d --- /dev/null +++ b/src/Interpreters/DictionaryReader.h @@ -0,0 +1,188 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; + extern const int TYPE_MISMATCH; +} + +class DictionaryReader +{ +public: + struct FunctionWrapper + { + ExecutableFunctionPtr function; + ColumnNumbers arg_positions; + size_t result_pos = 0; + + FunctionWrapper(const IFunctionOverloadResolver & resolver, const ColumnsWithTypeAndName & arguments, Block & block, + const ColumnNumbers & arg_positions_, const String & column_name, TypeIndex expected_type) + : arg_positions(arg_positions_) + { + FunctionBasePtr prepare_function = resolver.build(arguments); + result_pos = block.columns(); + + ColumnWithTypeAndName result; + result.name = "get_" + column_name; + result.type = prepare_function->getReturnType(); + if (result.type->getTypeId() != expected_type) + throw Exception("Type mismatch in dictionary reader for: " + column_name, ErrorCodes::TYPE_MISMATCH); + + function = prepare_function->prepare(block, arg_positions, result_pos); + } + + void execute(Block & block, size_t rows) const + { + function->execute(block, arg_positions, result_pos, rows, false); + } + }; + + DictionaryReader(const String & dictionary_name, const Names & src_column_names, const NamesAndTypesList & result_columns, + const Context & context, size_t key_size = 1) + : result_header(makeResultBlock(result_columns)) + , key_position(key_size + result_header.columns()) + { + if (src_column_names.size() != result_columns.size()) + throw Exception("Columns number mismatch in dictionary reader", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); + + FunctionOverloadResolverPtr dict_has(FunctionFactory::instance().get("dictHas", context)); + FunctionOverloadResolverPtr dict_get(FunctionFactory::instance().get("dictGet", context)); + + ColumnWithTypeAndName dict_name; + ColumnWithTypeAndName key; + ColumnWithTypeAndName column_name; + + { + dict_name.name = "dict"; + dict_name.type = std::make_shared(); + dict_name.column = dict_name.type->createColumnConst(1, dictionary_name); + + /// TODO: composite key (key_size > 1) + key.name = "key"; + key.type = std::make_shared(); + + column_name.name = "column"; + column_name.type = std::make_shared(); + } + + /// dictHas('dict_name', id) + ColumnsWithTypeAndName arguments_has; + arguments_has.push_back(dict_name); + arguments_has.push_back(key); + + /// dictGet('dict_name', 'attr_name', id) + ColumnsWithTypeAndName arguments_get; + arguments_get.push_back(dict_name); + arguments_get.push_back(column_name); + arguments_get.push_back(key); + + sample_block.insert(dict_name); + + for (auto & columns_name : src_column_names) + { + ColumnWithTypeAndName name; + name.name = "col_" + columns_name; + name.type = std::make_shared(); + name.column = name.type->createColumnConst(1, columns_name); + + sample_block.insert(name); + } + + sample_block.insert(key); + + ColumnNumbers positions_has{0, key_position}; + function_has = std::make_unique( + *dict_has, arguments_has, sample_block, positions_has, "has", DataTypeUInt8().getTypeId()); + functions_get.reserve(result_header.columns()); + + for (size_t i = 0; i < result_header.columns(); ++i) + { + size_t column_name_pos = key_size + i; + auto & column = result_header.getByPosition(i); + ColumnNumbers positions_get{0, column_name_pos, key_position}; + functions_get.emplace_back(FunctionWrapper( + *dict_get, arguments_get, sample_block, positions_get, column.name, column.type->getTypeId())); + } + } + + void readKeys(const IColumn & keys, size_t size, Block & out_block, ColumnVector::Container & found, + std::vector & positions) const + { + Block working_block = sample_block; + size_t has_position = key_position + 1; + + /// set keys for dictHas() + ColumnWithTypeAndName & key_column = working_block.getByPosition(key_position); + key_column.column = keys.cloneResized(size); /// just a copy we cannot avoid + + /// calculate and extract dictHas() + function_has->execute(working_block, size); + ColumnWithTypeAndName & has_column = working_block.getByPosition(has_position); + auto mutable_has = (*std::move(has_column.column)).mutate(); + found.swap(typeid_cast &>(*mutable_has).getData()); + has_column.column = nullptr; + + /// set mapping form source keys to resulting rows in output block + positions.clear(); + positions.resize(size, 0); + size_t pos = 0; + for (size_t i = 0; i < size; ++i) + if (found[i]) + positions[i] = pos++; + + /// set keys for dictGet(): remove not found keys + key_column.column = key_column.column->filter(found, -1); + size_t rows = key_column.column->size(); + + /// calculate dictGet() + for (auto & func : functions_get) + func.execute(working_block, rows); + + /// make result: copy header block with correct names and move data columns + out_block = result_header.cloneEmpty(); + size_t first_get_position = has_position + 1; + for (size_t i = 0; i < out_block.columns(); ++i) + { + auto & src_column = working_block.getByPosition(first_get_position + i); + auto & dst_column = out_block.getByPosition(i); + dst_column.column = src_column.column; + src_column.column = nullptr; + } + } + +private: + Block result_header; + Block sample_block; /// dictionary name, column names, key, dictHas() result, dictGet() results + size_t key_position; + std::unique_ptr function_has; + std::vector functions_get; + + static Block makeResultBlock(const NamesAndTypesList & names) + { + Block block; + for (auto & nm : names) + { + ColumnWithTypeAndName column{nullptr, nm.type, nm.name}; + if (column.type->isNullable()) + column.type = typeid_cast(*column.type).getNestedType(); + block.insert(std::move(column)); + } + return block; + } +}; + +} diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6494918c532..3add5164316 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -31,11 +31,13 @@ #include #include #include +#include #include #include #include +#include #include #include @@ -502,25 +504,11 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b return true; } -static JoinPtr tryGetStorageJoin(const ASTTablesInSelectQueryElement & join_element, std::shared_ptr analyzed_join, - const Context & context) +static JoinPtr tryGetStorageJoin(std::shared_ptr analyzed_join) { - const auto & table_to_join = join_element.table_expression->as(); - - /// TODO This syntax does not support specifying a database name. - if (table_to_join.database_and_table_name) - { - auto table_id = context.resolveStorageID(table_to_join.database_and_table_name); - StoragePtr table = DatabaseCatalog::instance().tryGetTable(table_id); - - if (table) - { - auto * storage_join = dynamic_cast(table.get()); - if (storage_join) - return storage_join->getJoin(analyzed_join); - } - } - + if (auto * table = analyzed_join->joined_storage.get()) + if (auto * storage_join = dynamic_cast(table)) + return storage_join->getJoin(analyzed_join); return {}; } @@ -531,10 +519,22 @@ static ExpressionActionsPtr createJoinedBlockActions(const Context & context, co return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false); } -static std::shared_ptr makeJoin(std::shared_ptr analyzed_join, const Block & sample_block) +static std::shared_ptr makeJoin(std::shared_ptr analyzed_join, const Block & sample_block, + const Names & original_right_columns, const Context & context) { bool allow_merge_join = analyzed_join->allowMergeJoin(); + /// TODO: check keys + if (auto * storage = analyzed_join->joined_storage.get()) + { + if (auto * dict = dynamic_cast(storage)) + { + analyzed_join->dictionary_reader = std::make_shared( + dict->dictionaryName(), original_right_columns, sample_block.getNamesAndTypesList(), context); + return std::make_shared(analyzed_join, sample_block); + } + } + if (analyzed_join->forceHashJoin() || (analyzed_join->preferMergeJoin() && !allow_merge_join)) return std::make_shared(analyzed_join, sample_block); else if (analyzed_join->forceMergeJoin() || (analyzed_join->preferMergeJoin() && allow_merge_join)) @@ -550,48 +550,49 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQuer SubqueryForSet & subquery_for_join = subqueries_for_sets[join_subquery_id]; - /// Special case - if table name is specified on the right of JOIN, then the table has the type Join (the previously prepared mapping). + /// Use StorageJoin if any. if (!subquery_for_join.join) - subquery_for_join.join = tryGetStorageJoin(join_element, syntax->analyzed_join, context); + subquery_for_join.join = tryGetStorageJoin(syntax->analyzed_join); if (!subquery_for_join.join) { /// Actions which need to be calculated on joined block. ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(context, analyzedJoin()); + Names original_right_columns; if (!subquery_for_join.source) { - NamesWithAliases required_columns_with_aliases = - analyzedJoin().getRequiredColumns(joined_block_actions->getSampleBlock(), joined_block_actions->getRequiredColumns()); - makeSubqueryForJoin(join_element, std::move(required_columns_with_aliases), subquery_for_join); + NamesWithAliases required_columns_with_aliases = analyzedJoin().getRequiredColumns( + joined_block_actions->getSampleBlock(), joined_block_actions->getRequiredColumns()); + for (auto & pr : required_columns_with_aliases) + original_right_columns.push_back(pr.first); + + /** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs + * - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1, + * in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`. + * - this function shows the expression JOIN _data1. + */ + auto interpreter = interpretSubquery(join_element.table_expression, context, original_right_columns, query_options); + + subquery_for_join.makeSource(interpreter, std::move(required_columns_with_aliases)); } /// TODO You do not need to set this up when JOIN is only needed on remote servers. subquery_for_join.setJoinActions(joined_block_actions); /// changes subquery_for_join.sample_block inside - subquery_for_join.join = makeJoin(syntax->analyzed_join, subquery_for_join.sample_block); + subquery_for_join.join = makeJoin(syntax->analyzed_join, subquery_for_join.sample_block, original_right_columns, context); + + /// Do not make subquery for join over dictionary. + if (syntax->analyzed_join->dictionary_reader) + { + JoinPtr join = subquery_for_join.join; + subqueries_for_sets.erase(join_subquery_id); + return join; + } } return subquery_for_join.join; } -void SelectQueryExpressionAnalyzer::makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element, - NamesWithAliases && required_columns_with_aliases, - SubqueryForSet & subquery_for_set) const -{ - /** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs - * - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1, - * in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`. - * - this function shows the expression JOIN _data1. - */ - Names original_columns; - for (auto & pr : required_columns_with_aliases) - original_columns.push_back(pr.first); - - auto interpreter = interpretSubquery(join_element.table_expression, context, original_columns, query_options); - - subquery_for_set.makeSource(interpreter, std::move(required_columns_with_aliases)); -} - bool SelectQueryExpressionAnalyzer::appendPrewhere( ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) { diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 4322a897378..b7fda92e33f 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -276,8 +276,6 @@ private: SetPtr isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name); JoinPtr makeTableJoin(const ASTTablesInSelectQueryElement & join_element); - void makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element, NamesWithAliases && required_columns_with_aliases, - SubqueryForSet & subquery_for_set) const; const ASTSelectQuery * getAggregatingQuery() const; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index a3432ebebba..16187f10fa1 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -4,16 +4,21 @@ #include #include +#include #include #include #include +#include #include #include #include #include #include +#include + +#include #include #include @@ -21,8 +26,6 @@ #include #include #include -#include - namespace DB { @@ -282,6 +285,39 @@ static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes return KeyGetter(key_columns, key_sizes, nullptr); } +class KeyGetterForDict +{ +public: + using Mapped = JoinStuff::MappedOne; + using FindResult = ColumnsHashing::columns_hashing_impl::FindResultImpl; + + KeyGetterForDict(const ColumnRawPtrs & key_columns_, const Sizes & key_sizes_, void *) + : key_columns(key_columns_) + , key_sizes(key_sizes_) + {} + + FindResult findKey(const DictionaryReader & reader, size_t i, const Arena &) + { + if (!read_result) + { + reader.readKeys(*key_columns[0], key_sizes[0], read_result, found, positions); + result.block = &read_result; + /// TODO: check types and correct nullability + } + + result.row_num = positions[i]; + return FindResult(&result, found[i]); + } + +private: + const ColumnRawPtrs & key_columns; + const Sizes & key_sizes; + Block read_result; + Mapped result; + ColumnVector::Container found; + std::vector positions; +}; + template struct KeyGetterForTypeImpl; @@ -351,7 +387,7 @@ size_t HashJoin::getTotalRowCount() const for (const auto & block : data->blocks) res += block.rows(); } - else + else if (data->type != Type::DICT) { joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { res += map.getTotalRowCount(data->type); }); } @@ -368,7 +404,7 @@ size_t HashJoin::getTotalByteCount() const for (const auto & block : data->blocks) res += block.bytes(); } - else + else if (data->type != Type::DICT) { joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { res += map.getTotalByteCountImpl(data->type); }); res += data->pool.size(); @@ -400,7 +436,13 @@ void HashJoin::setSampleBlock(const Block & block) if (nullable_right_side) JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add); - if (strictness == ASTTableJoin::Strictness::Asof) + if (table_join->dictionary_reader) + { + data->type = Type::DICT; + std::get(data->maps).create(Type::DICT); + chooseMethod(key_columns, key_sizes); /// init key_sizes + } + else if (strictness == ASTTableJoin::Strictness::Asof) { if (kind != ASTTableJoin::Kind::Left and kind != ASTTableJoin::Kind::Inner) throw Exception("ASOF only supports LEFT and INNER as base joins", ErrorCodes::NOT_IMPLEMENTED); @@ -526,7 +568,8 @@ namespace switch (type) { case HashJoin::Type::EMPTY: break; - case HashJoin::Type::CROSS: break; /// Do nothing. We have already saved block, and it is enough. + case HashJoin::Type::CROSS: break; /// Do nothing. We have already saved block, and it is enough. + case HashJoin::Type::DICT: break; /// Noone should call it with Type::DICT. #define M(TYPE) \ case HashJoin::Type::TYPE: \ @@ -598,6 +641,8 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) { if (empty()) throw Exception("Logical error: HashJoin was not initialized", ErrorCodes::LOGICAL_ERROR); + if (overDictionary()) + throw Exception("Logical error: insert into hash-map in HashJoin over dictionary", ErrorCodes::LOGICAL_ERROR); /// There's no optimization for right side const columns. Remove constness if any. Block block = materializeBlock(source_block); @@ -932,8 +977,7 @@ IColumn::Filter switchJoinRightColumns(const Maps & maps_, AddedColumns & added_ case HashJoin::Type::TYPE: \ return joinRightColumnsSwitchNullability>::Type>(\ - *maps_.TYPE, added_columns, null_map);\ - break; + *maps_.TYPE, added_columns, null_map); APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -942,6 +986,20 @@ IColumn::Filter switchJoinRightColumns(const Maps & maps_, AddedColumns & added_ } } +template +IColumn::Filter dictionaryJoinRightColumns(const DictionaryReader & reader, AddedColumns & added_columns, const ConstNullMapPtr & null_map) +{ + if constexpr (KIND == ASTTableJoin::Kind::Left && + (STRICTNESS == ASTTableJoin::Strictness::Any || + STRICTNESS == ASTTableJoin::Strictness::Semi || + STRICTNESS == ASTTableJoin::Strictness::Anti)) + { + return joinRightColumnsSwitchNullability(reader, added_columns, null_map); + } + + throw Exception("Logical error: wrong JOIN combination", ErrorCodes::LOGICAL_ERROR); +} + } /// nameless @@ -1002,7 +1060,9 @@ void HashJoin::joinBlockImpl( bool has_required_right_keys = (required_right_keys.columns() != 0); added_columns.need_filter = need_filter || has_required_right_keys; - IColumn::Filter row_filter = switchJoinRightColumns(maps_, added_columns, data->type, null_map); + IColumn::Filter row_filter = overDictionary() ? + dictionaryJoinRightColumns(*table_join->dictionary_reader, added_columns, null_map) : + switchJoinRightColumns(maps_, added_columns, data->type, null_map); for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); @@ -1205,7 +1265,36 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) const Names & key_names_left = table_join->keyNamesLeft(); JoinCommon::checkTypesOfKeys(block, key_names_left, right_table_keys, key_names_right); - if (joinDispatch(kind, strictness, data->maps, [&](auto kind_, auto strictness_, auto & map) + if (overDictionary()) + { + using Kind = ASTTableJoin::Kind; + using Strictness = ASTTableJoin::Strictness; + + auto & map = std::get(data->maps); + if (kind == Kind::Left) + { + switch (strictness) + { + case Strictness::Any: + case Strictness::All: + joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); + break; + case Strictness::Semi: + joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); + break; + case Strictness::Anti: + joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); + break; + default: + throw Exception("Logical error: wrong JOIN combination", ErrorCodes::LOGICAL_ERROR); + } + } + else if (kind == Kind::Inner && strictness == Strictness::All) + joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); + else + throw Exception("Logical error: wrong JOIN combination", ErrorCodes::LOGICAL_ERROR); + } + else if (joinDispatch(kind, strictness, data->maps, [&](auto kind_, auto strictness_, auto & map) { joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); })) diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 24ad2b871c9..48e7e9e9c9a 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -27,6 +27,7 @@ namespace DB { class TableJoin; +class DictionaryReader; namespace JoinStuff { @@ -148,7 +149,8 @@ class HashJoin : public IJoin public: HashJoin(std::shared_ptr table_join_, const Block & right_sample_block, bool any_take_last_row_ = false); - bool empty() { return data->type == Type::EMPTY; } + bool empty() const { return data->type == Type::EMPTY; } + bool overDictionary() const { return data->type == Type::DICT; } /** Add block of data from right hand of JOIN to the map. * Returns false, if some limit was exceeded and you should not insert more data. @@ -220,12 +222,12 @@ public: { EMPTY, CROSS, + DICT, #define M(NAME) NAME, APPLY_FOR_JOIN_VARIANTS(M) #undef M }; - /** Different data structures, that are used to perform JOIN. */ template @@ -247,6 +249,7 @@ public: { case Type::EMPTY: break; case Type::CROSS: break; + case Type::DICT: break; #define M(NAME) \ case Type::NAME: NAME = std::make_unique(); break; @@ -261,6 +264,7 @@ public: { case Type::EMPTY: return 0; case Type::CROSS: return 0; + case Type::DICT: return 0; #define M(NAME) \ case Type::NAME: return NAME ? NAME->size() : 0; @@ -277,6 +281,7 @@ public: { case Type::EMPTY: return 0; case Type::CROSS: return 0; + case Type::DICT: return 0; #define M(NAME) \ case Type::NAME: return NAME ? NAME->getBufferSizeInBytes() : 0; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 35b33874ac1..6ebe15768c7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -305,12 +305,13 @@ InterpreterSelectQuery::InterpreterSelectQuery( max_streams = settings.max_threads; ASTSelectQuery & query = getSelectQuery(); + std::shared_ptr table_join = joined_tables.makeTableJoin(query); auto analyze = [&] (bool try_move_to_prewhere = true) { syntax_analyzer_result = SyntaxAnalyzer(*context).analyzeSelect( query_ptr, SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage), - options, joined_tables.tablesWithColumns(), required_result_column_names); + options, joined_tables.tablesWithColumns(), required_result_column_names, table_join); /// Save scalar sub queries's results in the query context if (context->hasQueryContext()) diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index cedf95bea06..c00704ced9c 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -1,18 +1,26 @@ #include +#include #include #include #include #include #include + #include #include #include +#include +#include + #include +#include #include #include #include #include #include +#include +#include namespace DB { @@ -26,6 +34,34 @@ namespace ErrorCodes namespace { +void replaceJoinedTable(const ASTSelectQuery & select_query) +{ + const ASTTablesInSelectQueryElement * join = select_query.join(); + if (!join || !join->table_expression) + return; + + /// TODO: Push down for CROSS JOIN is not OK [disabled] + const auto & table_join = join->table_join->as(); + if (table_join.kind == ASTTableJoin::Kind::Cross) + return; + + auto & table_expr = join->table_expression->as(); + if (table_expr.database_and_table_name) + { + const auto & table_id = table_expr.database_and_table_name->as(); + String expr = "(select * from " + table_id.name + ") as " + table_id.shortName(); + + // FIXME: since the expression "a as b" exposes both "a" and "b" names, which is not equivalent to "(select * from a) as b", + // we can't replace aliased tables. + // FIXME: long table names include database name, which we can't save within alias. + if (table_id.alias.empty() && table_id.isShort()) + { + ParserTableExpression parser; + table_expr = parseQuery(parser, expr, 0)->as(); + } + } +} + template void checkTablesWithColumns(const std::vector & tables_with_columns, const Context & context) { @@ -209,4 +245,32 @@ void JoinedTables::rewriteDistributedInAndJoins(ASTPtr & query) } } +std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & select_query) +{ + auto settings = context.getSettingsRef(); + auto table_join = std::make_shared(settings, context.getTemporaryVolume()); + + const ASTTablesInSelectQueryElement * ast_join = select_query.join(); + const auto & table_to_join = ast_join->table_expression->as(); + + /// TODO This syntax does not support specifying a database name. + if (table_to_join.database_and_table_name) + { + auto joined_table_id = context.resolveStorageID(table_to_join.database_and_table_name); + StoragePtr table = DatabaseCatalog::instance().tryGetTable(joined_table_id); + if (table) + { + if (dynamic_cast(table.get()) || + dynamic_cast(table.get())) + table_join->joined_storage = table; + } + } + + if (!table_join->joined_storage && + settings.enable_optimize_predicate_expression) + replaceJoinedTable(select_query); + + return table_join; +} + } diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index 66b3c8de609..6f5750823b0 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -10,6 +10,7 @@ namespace DB class ASTSelectQuery; class Context; +class TableJoin; struct SelectQueryOptions; /// Joined tables' columns resolver. @@ -28,6 +29,7 @@ public: StoragePtr getLeftTableStorage(); bool resolveTables(); void makeFakeTable(StoragePtr storage, const Block & source_header); + std::shared_ptr makeTableJoin(const ASTSelectQuery & select_query); const std::vector & tablesWithColumns() const { return tables_with_columns; } diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index 8a9a63206ba..7016d95f6ac 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -28,8 +28,6 @@ #include #include #include -#include -#include #include #include @@ -549,34 +547,6 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTSelectQuery & sele } } -void replaceJoinedTable(const ASTSelectQuery & select_query) -{ - const ASTTablesInSelectQueryElement * join = select_query.join(); - if (!join || !join->table_expression) - return; - - /// TODO: Push down for CROSS JOIN is not OK [disabled] - const auto & table_join = join->table_join->as(); - if (table_join.kind == ASTTableJoin::Kind::Cross) - return; - - auto & table_expr = join->table_expression->as(); - if (table_expr.database_and_table_name) - { - const auto & table_id = table_expr.database_and_table_name->as(); - String expr = "(select * from " + table_id.name + ") as " + table_id.shortName(); - - // FIXME: since the expression "a as b" exposes both "a" and "b" names, which is not equivalent to "(select * from a) as b", - // we can't replace aliased tables. - // FIXME: long table names include database name, which we can't save within alias. - if (table_id.alias.empty() && table_id.isShort()) - { - ParserTableExpression parser; - table_expr = parseQuery(parser, expr, 0)->as(); - } - } -} - std::vector getAggregates(ASTPtr & query, const ASTSelectQuery & select_query) { /// There can not be aggregate functions inside the WHERE and PREWHERE. @@ -783,7 +753,8 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect( SyntaxAnalyzerResult && result, const SelectQueryOptions & select_options, const std::vector & tables_with_columns, - const Names & required_result_columns) const + const Names & required_result_columns, + std::shared_ptr table_join) const { auto * select_query = query->as(); if (!select_query) @@ -795,14 +766,13 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect( const auto & settings = context.getSettingsRef(); const NameSet & source_columns_set = result.source_columns_set; - result.analyzed_join = std::make_shared(settings, context.getTemporaryVolume()); + result.analyzed_join = table_join; + if (!result.analyzed_join) /// ExpressionAnalyzer expects some not empty object here + result.analyzed_join = std::make_shared(); if (remove_duplicates) renameDuplicatedColumns(select_query); - if (settings.enable_optimize_predicate_expression) - replaceJoinedTable(*select_query); - /// TODO: Remove unneeded conversion std::vector tables_with_column_names; tables_with_column_names.reserve(tables_with_columns.size()); diff --git a/src/Interpreters/SyntaxAnalyzer.h b/src/Interpreters/SyntaxAnalyzer.h index 23e8a4b79aa..08afd14b83c 100644 --- a/src/Interpreters/SyntaxAnalyzer.h +++ b/src/Interpreters/SyntaxAnalyzer.h @@ -94,7 +94,8 @@ public: SyntaxAnalyzerResult && result, const SelectQueryOptions & select_options = {}, const std::vector & tables_with_columns = {}, - const Names & required_result_columns = {}) const; + const Names & required_result_columns = {}, + std::shared_ptr table_join = {}) const; private: const Context & context; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 0b5ed82411a..4cde414e270 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -19,6 +20,7 @@ class Context; class ASTSelectQuery; struct DatabaseAndTableWithAlias; class Block; +class DictionaryReader; struct Settings; @@ -42,10 +44,10 @@ class TableJoin friend class SyntaxAnalyzer; const SizeLimits size_limits; - const size_t default_max_bytes; - const bool join_use_nulls; + const size_t default_max_bytes = 0; + const bool join_use_nulls = false; const size_t max_joined_block_rows = 0; - JoinAlgorithm join_algorithm; + JoinAlgorithm join_algorithm = JoinAlgorithm::AUTO; const bool partial_merge_join_optimizations = false; const size_t partial_merge_join_rows_in_right_blocks = 0; @@ -69,6 +71,7 @@ class TableJoin VolumePtr tmp_volume; public: + TableJoin() = default; TableJoin(const Settings &, VolumePtr tmp_volume); /// for StorageJoin @@ -84,6 +87,9 @@ public: table_join.strictness = strictness; } + StoragePtr joined_storage; + std::shared_ptr dictionary_reader; + ASTTableJoin::Kind kind() const { return table_join.kind; } ASTTableJoin::Strictness strictness() const { return table_join.strictness; } bool sameStrictnessAndKind(ASTTableJoin::Strictness, ASTTableJoin::Kind) const; diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index fd6cb1902dc..85cddda399d 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -54,6 +54,8 @@ public: return description; } + const String & dictionaryName() const { return dictionary_name; } + private: using Ptr = MultiVersion::Version; From ab12ebb5cfa5914cb49ce7a49a62d1ab8755fbea Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 8 Apr 2020 22:58:27 +0300 Subject: [PATCH 121/743] add allowDictJoin() --- src/Interpreters/ExpressionAnalyzer.cpp | 38 +++++++++++++++++++------ 1 file changed, 30 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 3add5164316..fbf2b663f3b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -44,6 +44,7 @@ #include #include +#include #include #include @@ -519,20 +520,41 @@ static ExpressionActionsPtr createJoinedBlockActions(const Context & context, co return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false); } +static bool allowDictJoin(const TableJoin & table_join, const Context & context, String & dict_name) +{ + if (!table_join.joined_storage) + return false; + + const Names & right_keys = table_join.keyNamesRight(); + if (right_keys.size() != 1) + return false; + + const String & key_name = right_keys[0]; /// TODO: compound name + + auto * dict = dynamic_cast(table_join.joined_storage.get()); + if (!dict) + return false; + + dict_name = dict->dictionaryName(); + auto dictionary = context.getExternalDictionariesLoader().getDictionary(dict_name); + if (!dictionary) + return false; + + const DictionaryStructure & structure = dictionary->getStructure(); + return structure.id && (structure.id->name == key_name); /// key is UInt64 +} + static std::shared_ptr makeJoin(std::shared_ptr analyzed_join, const Block & sample_block, const Names & original_right_columns, const Context & context) { bool allow_merge_join = analyzed_join->allowMergeJoin(); - /// TODO: check keys - if (auto * storage = analyzed_join->joined_storage.get()) + String dict_name; + if (allowDictJoin(*analyzed_join, context, dict_name)) { - if (auto * dict = dynamic_cast(storage)) - { - analyzed_join->dictionary_reader = std::make_shared( - dict->dictionaryName(), original_right_columns, sample_block.getNamesAndTypesList(), context); - return std::make_shared(analyzed_join, sample_block); - } + analyzed_join->dictionary_reader = std::make_shared( + dict_name, original_right_columns, sample_block.getNamesAndTypesList(), context); + return std::make_shared(analyzed_join, sample_block); } if (analyzed_join->forceHashJoin() || (analyzed_join->preferMergeJoin() && !allow_merge_join)) From c0051d9cd960ed872b6c3b6688c97ab29bb88259 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Thu, 9 Apr 2020 00:48:00 +0300 Subject: [PATCH 122/743] fixed bug with ClickHouseDictionarySource & test for all sources added --- .../ClickHouseDictionarySource.cpp | 3 + .../__init__.py | 0 .../configs/config.xml | 30 +++++++ .../dictionaries/ClickHouseSourceConfig.xml | 48 +++++++++++ .../dictionaries/ExecutableSourceConfig.xml | 45 ++++++++++ .../configs/dictionaries/FileSourceConfig.xml | 45 ++++++++++ .../configs/dictionaries/HTTPSourceConfig.xml | 54 ++++++++++++ .../configs/dictionaries/source.csv | 3 + .../configs/users.xml | 23 +++++ .../http_server.py | 86 +++++++++++++++++++ .../test_dictionary_custom_settings/test.py | 62 +++++++++++++ 11 files changed, 399 insertions(+) create mode 100644 tests/integration/test_dictionary_custom_settings/__init__.py create mode 100644 tests/integration/test_dictionary_custom_settings/configs/config.xml create mode 100644 tests/integration/test_dictionary_custom_settings/configs/dictionaries/ClickHouseSourceConfig.xml create mode 100644 tests/integration/test_dictionary_custom_settings/configs/dictionaries/ExecutableSourceConfig.xml create mode 100644 tests/integration/test_dictionary_custom_settings/configs/dictionaries/FileSourceConfig.xml create mode 100644 tests/integration/test_dictionary_custom_settings/configs/dictionaries/HTTPSourceConfig.xml create mode 100644 tests/integration/test_dictionary_custom_settings/configs/dictionaries/source.csv create mode 100644 tests/integration/test_dictionary_custom_settings/configs/users.xml create mode 100644 tests/integration/test_dictionary_custom_settings/http_server.py create mode 100644 tests/integration/test_dictionary_custom_settings/test.py diff --git a/dbms/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/Dictionaries/ClickHouseDictionarySource.cpp index 45895ae93b2..8d6c5b205e0 100644 --- a/dbms/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/Dictionaries/ClickHouseDictionarySource.cpp @@ -74,6 +74,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {}); /// Processors are not supported here yet. + context.setSettings(context_.getSettings()); context.setSetting("experimental_use_processors", false); /// Query context is needed because some code in executeQuery function may assume it exists. /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. @@ -217,6 +218,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) bool /* check_config */) -> DictionarySourcePtr { Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + + std::cerr << "initialization: " << context_local_copy.getSettings().max_bytes_to_read << '\n'; /// Note that processors are not supported yet (see constructor), /// hence it is not possible to override experimental_use_processors setting return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context_local_copy); diff --git a/tests/integration/test_dictionary_custom_settings/__init__.py b/tests/integration/test_dictionary_custom_settings/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_dictionary_custom_settings/configs/config.xml b/tests/integration/test_dictionary_custom_settings/configs/config.xml new file mode 100644 index 00000000000..1e4c14585a9 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/config.xml @@ -0,0 +1,30 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ClickHouseSourceConfig.xml b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ClickHouseSourceConfig.xml new file mode 100644 index 00000000000..2191c8ded8a --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ClickHouseSourceConfig.xml @@ -0,0 +1,48 @@ + + + test_clickhouse + + + + localhost + 9000 + default + + default + source
+
+ + + 1 + + + + + 600 + + + + + + + + id + + + first + String + + + + second + String + + + + third + String + + + +
+
diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ExecutableSourceConfig.xml b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ExecutableSourceConfig.xml new file mode 100644 index 00000000000..3191118c4e9 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ExecutableSourceConfig.xml @@ -0,0 +1,45 @@ + + + test_executable + + + + cat /etc/clickhouse-server/config.d/source.csv + CSVWithNames + + + + 0 + 0 + + + + + 600 + + + + + + + + id + + + first + String + + + + second + String + + + + third + String + + + + + diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/FileSourceConfig.xml b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/FileSourceConfig.xml new file mode 100644 index 00000000000..ff7baf29be0 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/FileSourceConfig.xml @@ -0,0 +1,45 @@ + + + test_file + + + + /etc/clickhouse-server/config.d/source.csv + CSVWithNames + + + + 0 + 0 + + + + + 600 + + + + + + + + id + + + first + String + + + + second + String + + + + third + String + + + + + diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/HTTPSourceConfig.xml b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/HTTPSourceConfig.xml new file mode 100644 index 00000000000..dc03974c4b6 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/HTTPSourceConfig.xml @@ -0,0 +1,54 @@ + + + test_http + + + http://localhost:5555/source.csv + CSVWithNames + + foo + bar + + +
+ api-key + secret +
+
+
+ + + 0 + 0 + + + + + 600 + + + + + + + + id + + + first + String + + + + second + String + + + + third + String + + + +
+
diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/source.csv b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/source.csv new file mode 100644 index 00000000000..23d113e5225 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/source.csv @@ -0,0 +1,3 @@ +id,first,second,third +1,'a,"b,c +2,'d,"e,f diff --git a/tests/integration/test_dictionary_custom_settings/configs/users.xml b/tests/integration/test_dictionary_custom_settings/configs/users.xml new file mode 100644 index 00000000000..6061af8e33d --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/users.xml @@ -0,0 +1,23 @@ + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_dictionary_custom_settings/http_server.py b/tests/integration/test_dictionary_custom_settings/http_server.py new file mode 100644 index 00000000000..c7920a9024d --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/http_server.py @@ -0,0 +1,86 @@ +# -*- coding: utf-8 -*- +import argparse +from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer +import socket +import ssl +import csv + + +# Decorator used to see if authentication works for external dictionary who use a HTTP source. +def check_auth(fn): + def wrapper(req): + auth_header = req.headers.get('authorization', None) + api_key = req.headers.get('api-key', None) + if not auth_header or auth_header != 'Basic Zm9vOmJhcg==' or not api_key or api_key != 'secret': + req.send_response(401) + else: + fn(req) + return wrapper + + +def start_server(server_address, data_path, schema, cert_path, address_family): + class TSVHTTPHandler(BaseHTTPRequestHandler): + @check_auth + def do_GET(self): + self.__send_headers() + self.__send_data() + + @check_auth + def do_POST(self): + ids = self.__read_and_decode_post_ids() + print "ids=", ids + self.__send_headers() + self.__send_data(ids) + + def __send_headers(self): + self.send_response(200) + self.send_header('Content-type', 'text/csv') + self.end_headers() + + def __send_data(self, only_ids = None): + with open(data_path, 'r') as fl: + reader = csv.reader(fl, delimiter='\t') + for row in reader: + if not only_ids or (row[0] in only_ids): + self.wfile.write('\t'.join(row) + '\n') + + def __read_and_decode_post_ids(self): + data = self.__read_and_decode_post_data() + return filter(None, data.split()) + + def __read_and_decode_post_data(self): + transfer_encoding = self.headers.get("Transfer-encoding") + decoded = "" + if transfer_encoding == "chunked": + while True: + s = self.rfile.readline() + chunk_length = int(s, 16) + if not chunk_length: + break + decoded += self.rfile.read(chunk_length) + self.rfile.readline() + else: + content_length = int(self.headers.get("Content-Length", 0)) + decoded = self.rfile.read(content_length) + return decoded + + if address_family == "ipv6": + HTTPServer.address_family = socket.AF_INET6 + httpd = HTTPServer(server_address, TSVHTTPHandler) + if schema == "https": + httpd.socket = ssl.wrap_socket(httpd.socket, certfile=cert_path, server_side=True) + httpd.serve_forever() + + +if __name__ == "__main__": + parser = argparse.ArgumentParser(description="Simple HTTP server returns data from file") + parser.add_argument("--host", default="localhost") + parser.add_argument("--port", default=5555, type=int) + parser.add_argument("--data-path", required=True) + parser.add_argument("--schema", choices=("http", "https"), required=True) + parser.add_argument("--cert-path", default="./fake_cert.pem") + parser.add_argument('--address-family', choices=("ipv4", "ipv6"), default="ipv4") + + args = parser.parse_args() + + start_server((args.host, args.port), args.data_path, args.schema, args.cert_path, args.address_family) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py new file mode 100644 index 00000000000..a9a7c19d4af --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -0,0 +1,62 @@ +import os +import pytest + +from helpers.cluster import ClickHouseCluster + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +config_dir = os.path.join(SCRIPT_DIR, './configs') +DICTIONARY_FILES = [ + 'configs/dictionaries/FileSourceConfig.xml', + 'configs/dictionaries/ExecutableSourceConfig.xml', + 'configs/dictionaries/source.csv', + 'configs/dictionaries/HTTPSourceConfig.xml', + 'configs/dictionaries/ClickHouseSourceConfig.xml' +] + +cluster = ClickHouseCluster(__file__, base_configs_dir=config_dir) +instance = cluster.add_instance('node', main_configs=DICTIONARY_FILES, config_dir=config_dir) + +def prepare(): + node = instance + path = "/source.csv" + + script_dir = os.path.dirname(os.path.realpath(__file__)) + node.copy_file_to_container(os.path.join(script_dir, './http_server.py'), '/http_server.py') + node.copy_file_to_container(os.path.join(script_dir, 'configs/dictionaries/source.csv'), './source.csv') + node.exec_in_container([ + "bash", + "-c", + "python2 /http_server.py --data-path={tbl} --schema=http --host=localhost --port=5555".format( + tbl=path) + ], detach=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + prepare() + yield cluster + finally: + cluster.shutdown() + +def test_work(start_cluster): + query = instance.query + + assert query("SELECT dictGetString('test_file', 'first', toUInt64(1))") == "\\\'a\n" + assert query("SELECT dictGetString('test_file', 'second', toUInt64(1))") == "\"b\n" + assert query("SELECT dictGetString('test_executable', 'first', toUInt64(1))") == "\\\'a\n" + assert query("SELECT dictGetString('test_executable', 'second', toUInt64(1))") == "\"b\n" + + caught_exception = '' + try: + instance.query("CREATE TABLE source (id UInt64, first String, second String, third String) ENGINE=File(CSVWithNames);") + instance.query("INSERT INTO default.source VALUES (1, 'aaa', 'bbb', 'cccc'), (2, 'ddd', 'eee', 'fff')") + instance.query("SELECT dictGetString('test_clickhouse', 'second', toUInt64(1))") + except Exception as e: + caught_exception = str(e) + + assert caught_exception.find("Limit for result exceeded") != -1 + + assert query("SELECT dictGetString('test_http', 'first', toUInt64(1))") == "\\\'a\n" + assert query("SELECT dictGetString('test_http', 'second', toUInt64(1))") == "\"b\n" \ No newline at end of file From f2fda8570848b3ea7f24e7742c97700dfbb32386 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Thu, 9 Apr 2020 00:53:48 +0300 Subject: [PATCH 123/743] documentation edit --- .../dicts/external_dicts_dict_sources.md | 19 +++++++++++++++++++ .../dicts/external_dicts_dict_sources.md | 19 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/docs/en/query_language/dicts/external_dicts_dict_sources.md b/docs/en/query_language/dicts/external_dicts_dict_sources.md index 1756936febf..9a67d4bde9d 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/en/query_language/dicts/external_dicts_dict_sources.md @@ -30,6 +30,25 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration The source is configured in the `source` section. +For source types +[Local file](#dicts-external_dicts_dict_sources-local_file), +[Executable file](#dicts-external_dicts_dict_sources-executable), +[HTTP(s)](#dicts-external_dicts_dict_sources-http), +[ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) +optional format settings are available: + +``` xml + + + /opt/dictionaries/os.tsv + TabSeparated + + + 0 + + +``` + Types of sources (`source_type`): - [Local file](#dicts-external_dicts_dict_sources-local_file) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_sources.md b/docs/ru/query_language/dicts/external_dicts_dict_sources.md index 8b9961ee7fc..5b13a438179 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_sources.md @@ -30,6 +30,25 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration Источник настраивается в разделе `source`. +Для типов источников +[Локальный файл](#dicts-external_dicts_dict_sources-local_file), +[Исполняемый файл](#dicts-external_dicts_dict_sources-executable), +[HTTP(s)](#dicts-external_dicts_dict_sources-http), +[ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) +доступны дополнительные настройки форматирования: + +``` xml + + + /opt/dictionaries/os.tsv + TabSeparated + + + 0 + + +``` + Типы источников (`source_type`): - [Локальный файл](#dicts-external_dicts_dict_sources-local_file) From eea558b713cd4d8d0582dd08a504d5b2f32a017d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Thu, 9 Apr 2020 00:57:20 +0300 Subject: [PATCH 124/743] minor changes --- dbms/Dictionaries/DictionaryStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/Dictionaries/DictionaryStructure.cpp b/dbms/Dictionaries/DictionaryStructure.cpp index fe4b1474e84..5528be7a2bb 100644 --- a/dbms/Dictionaries/DictionaryStructure.cpp +++ b/dbms/Dictionaries/DictionaryStructure.cpp @@ -356,4 +356,4 @@ std::vector DictionaryStructure::getAttributes( return res_attributes; } -} \ No newline at end of file +} From 140cd88c201c3fa2c65a2e8230881eb233a86d23 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 9 Apr 2020 02:59:39 +0300 Subject: [PATCH 125/743] fixes --- src/Interpreters/Context.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 24 ++++++++++++++++--- src/Interpreters/DatabaseCatalog.h | 6 +++-- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++++ src/Storages/StorageReplicatedMergeTree.h | 2 ++ ...213_alter_rename_primary_key_zookeeper.sql | 3 ++- 6 files changed, 35 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 80b50194dda..a9c1d122275 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -404,7 +404,7 @@ struct ContextShared if (system_logs) system_logs->shutdown(); - DatabaseCatalog::instance().shutdown(); + DatabaseCatalog::shutdown(); /// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference). /// TODO: Get rid of this. diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index b6f92926116..69b57e97c6e 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes extern const int DATABASE_NOT_EMPTY; extern const int DATABASE_ACCESS_DENIED; extern const int LOGICAL_ERROR; + extern const int NULL_POINTER_DEREFERENCE; } TemporaryTableHolder::TemporaryTableHolder(const Context & context_, @@ -107,6 +108,7 @@ StoragePtr TemporaryTableHolder::getTable() const void DatabaseCatalog::loadDatabases() { + drop_delay_s = global_context->getConfigRef().getInt("database_atomic_delay_before_drop_table_s", 60); auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); @@ -117,7 +119,7 @@ void DatabaseCatalog::loadDatabases() (*drop_task)->activateAndSchedule(); } -void DatabaseCatalog::shutdown() +void DatabaseCatalog::shutdownImpl() { if (drop_task) (*drop_task)->deactivate(); @@ -378,8 +380,7 @@ DatabaseCatalog::DatabaseCatalog(Context * global_context_) : global_context(global_context_), log(&Poco::Logger::get("DatabaseCatalog")) { if (!global_context) - throw Exception("DatabaseCatalog is not initialized. It's a bug.", ErrorCodes::LOGICAL_ERROR); - drop_delay_s = global_context->getConfigRef().getInt("database_atomic_delay_before_drop_table_s", 60); + throw Exception("DatabaseCatalog is not initialized. It's a bug.", ErrorCodes::NULL_POINTER_DEREFERENCE); } DatabaseCatalog & DatabaseCatalog::init(Context * global_context_) @@ -393,6 +394,23 @@ DatabaseCatalog & DatabaseCatalog::instance() return init(nullptr); } +void DatabaseCatalog::shutdown() +{ + try + { + instance().shutdownImpl(); + } + catch (const Exception & e) + { + /// If catalog was not initialized yet by init(global_context), instance() throws NULL_POINTER_DEREFERENCE. + /// It can happen if some exception was thrown on first steps of startup (e.g. command line arguments parsing). + /// Ignore it. + if (e.code() == ErrorCodes::NULL_POINTER_DEREFERENCE) + return; + throw; + } +} + DatabasePtr DatabaseCatalog::getDatabase(const String & database_name, const Context & local_context) const { String resolved_database = local_context.resolveDatabase(database_name); diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 338a81851ba..0838a80ec0c 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -102,9 +102,9 @@ public: static DatabaseCatalog & init(Context * global_context_); static DatabaseCatalog & instance(); + static void shutdown(); void loadDatabases(); - void shutdown(); /// Get an object that protects the table from concurrently executing multiple DDL operations. std::unique_ptr getDDLGuard(const String & database, const String & table); @@ -166,6 +166,8 @@ private: void assertDatabaseExistsUnlocked(const String & database_name) const; void assertDatabaseDoesntExistUnlocked(const String & database_name) const; + void shutdownImpl(); + struct UUIDToStorageMapPart { @@ -222,7 +224,7 @@ private: mutable std::mutex tables_marked_dropped_mutex; std::unique_ptr drop_task; - time_t drop_delay_s; + time_t drop_delay_s = 60; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3c78972d372..b0097a92c48 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2880,11 +2880,15 @@ void StorageReplicatedMergeTree::startup() move_parts_task_handle = pool.createTask([this] { return movePartsTask(); }); pool.startTask(move_parts_task_handle); } + need_shutdown.store(true); } void StorageReplicatedMergeTree::shutdown() { + if (!need_shutdown.load()) + return; + clearOldPartsFromFilesystem(true); /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. fetcher.blocker.cancelForever(); @@ -2917,6 +2921,7 @@ void StorageReplicatedMergeTree::shutdown() std::unique_lock lock(data_parts_exchange_endpoint->rwlock); } data_parts_exchange_endpoint.reset(); + need_shutdown.store(false); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 294830c4567..3d9891a605c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -288,6 +288,8 @@ private: /// True if replica was created for existing table with fixed granularity bool other_replicas_fixed_granularity = false; + std::atomic_bool need_shutdown{false}; + template void foreachCommittedParts(const Func & func) const; diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql index e03ef67212e..b5ad162fdfb 100644 --- a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql +++ b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql @@ -23,7 +23,8 @@ ALTER TABLE table_for_rename_pk RENAME COLUMN key3 TO renamed_key3; --{serverErr ALTER TABLE table_for_rename_pk RENAME COLUMN key2 TO renamed_key2; --{serverError 44} -DROP TABLE IF EXISTS table_for_rename_pk; +DROP TABLE IF EXISTS table_for_rename_pk NO DELAY; +SELECT sleep(1) FORMAT Null; DROP TABLE IF EXISTS table_for_rename_with_primary_key; From a5ac19bf1a385473da57a4982c91f8557c1d216f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Thu, 9 Apr 2020 17:33:59 +0300 Subject: [PATCH 126/743] deleted debug output --- src/Dictionaries/ClickHouseDictionarySource.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index a9aad0739b1..aa06e1b8660 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -230,7 +230,6 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) { Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); - std::cerr << "initialization: " << context_local_copy.getSettings().max_bytes_to_read << '\n'; /// Note that processors are not supported yet (see constructor), /// hence it is not possible to override experimental_use_processors setting return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context_local_copy); From 3c4ac2f04c66fab059bcd373b27bc80fe17bb4d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Thu, 9 Apr 2020 17:38:01 +0300 Subject: [PATCH 127/743] minor changes --- src/Dictionaries/DictionaryStructure.cpp | 2 -- src/Dictionaries/DictionaryStructure.h | 7 ------- 2 files changed, 9 deletions(-) diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 5528be7a2bb..f8b8fbd6aab 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -20,7 +20,6 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int TYPE_MISMATCH; extern const int BAD_ARGUMENTS; - extern const int NO_ELEMENTS_IN_CONFIG; } namespace @@ -194,7 +193,6 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration } attributes = getAttributes(config, config_prefix); - if (attributes.empty()) throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS}; } diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index cd9d41f67ee..2893dea2e4f 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include @@ -102,7 +101,6 @@ struct DictionaryStructure final std::optional id; std::optional> key; std::vector attributes; - Settings settings; std::optional range_min; std::optional range_max; bool has_expressions = false; @@ -120,11 +118,6 @@ private: const std::string & config_prefix, const bool hierarchy_allowed = true, const bool allow_null_values = true); - - void getSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - Settings & dict_settings); }; } From 10aad522787942f2066e1780e10f96d8b6e4678c Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Thu, 9 Apr 2020 17:38:32 +0300 Subject: [PATCH 128/743] fix crash --- src/Interpreters/JoinedTables.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index c00704ced9c..e6843a3089e 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -247,6 +247,9 @@ void JoinedTables::rewriteDistributedInAndJoins(ASTPtr & query) std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & select_query) { + if (tables_with_columns.size() < 2) + return {}; + auto settings = context.getSettingsRef(); auto table_join = std::make_shared(settings, context.getTemporaryVolume()); From ee36750482d6ddca2a3247679293e5fb4a622b29 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 9 Apr 2020 18:36:13 +0300 Subject: [PATCH 129/743] Set storage policy explicitly in MergeTree over S3 tests. --- .../configs/config.d/storage_conf.xml | 4 ++-- tests/integration/test_merge_tree_s3/test.py | 4 +++- .../configs/config.d/storage_conf.xml | 11 ++--------- .../integration/test_replicated_merge_tree_s3/test.py | 10 ++++------ 4 files changed, 11 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index 5b292446c6b..d097675ca63 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -13,7 +13,7 @@
- +
s3 @@ -22,7 +22,7 @@ hdd - + diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index e12e31ebff2..4beb33604be 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -67,7 +67,9 @@ def create_table(cluster, table_name, additional_settings=None): PARTITION BY dt ORDER BY (dt, id) SETTINGS - old_parts_lifetime=0, index_granularity=512 + storage_policy = 's3', + old_parts_lifetime=0, + index_granularity=512 """.format(table_name) if additional_settings: diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml index 5b292446c6b..b32770095fc 100644 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml @@ -7,22 +7,15 @@ minio minio123 - - local - / - - +
s3
- - hdd -
-
+
diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index a8b7cf63e38..8689e7ccf5d 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -67,6 +67,7 @@ def create_table(cluster): ) ENGINE=ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/s3', '{instance}') PARTITION BY dt ORDER BY (dt, id) + SETTINGS storage_policy = 's3' """ for node in cluster.instances.values(): @@ -80,12 +81,9 @@ def drop_table(cluster): node.query("DROP TABLE IF EXISTS s3_test") minio = cluster.minio_client - try: - assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 - finally: - # Remove extra objects to prevent tests cascade failing - for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): - minio.remove_object(cluster.minio_bucket, obj.object_name) + # Remove extra objects to prevent tests cascade failing + for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): + minio.remove_object(cluster.minio_bucket, obj.object_name) def test_insert_select_replicated(cluster): From a6d0aacd534fa939d815a06c4068d892886f2e67 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 9 Apr 2020 21:02:27 +0300 Subject: [PATCH 130/743] fix test --- .../test_force_deduplication/test.py | 66 ++++++++++++------- 1 file changed, 43 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_force_deduplication/test.py b/tests/integration/test_force_deduplication/test.py index 0969f538988..5ad964ecb27 100644 --- a/tests/integration/test_force_deduplication/test.py +++ b/tests/integration/test_force_deduplication/test.py @@ -19,31 +19,47 @@ def start_cluster(): finally: cluster.shutdown() +def get_counts(): + src = int(node.query("SELECT count() FROM test")) + a = int(node.query("SELECT count() FROM test_mv_a")) + b = int(node.query("SELECT count() FROM test_mv_b")) + c = int(node.query("SELECT count() FROM test_mv_c")) + return src, a, b, c + + def test_basic(start_cluster): + node.query( + ''' + CREATE TABLE test (A Int64) ENGINE = ReplicatedMergeTree ('/clickhouse/test/tables/test','1') ORDER BY tuple(); + CREATE MATERIALIZED VIEW test_mv_a Engine=ReplicatedMergeTree ('/clickhouse/test/tables/test_mv_a','1') order by tuple() AS SELECT A FROM test; + CREATE MATERIALIZED VIEW test_mv_b Engine=ReplicatedMergeTree ('/clickhouse/test/tables/test_mv_b','1') partition by A order by tuple() AS SELECT A FROM test; + CREATE MATERIALIZED VIEW test_mv_c Engine=ReplicatedMergeTree ('/clickhouse/test/tables/test_mv_c','1') order by tuple() AS SELECT A FROM test; + INSERT INTO test values(999); + INSERT INTO test values(999); + ''' + ) with pytest.raises(QueryRuntimeException): node.query( ''' - CREATE TABLE test (A Int64) ENGINE = ReplicatedMergeTree ('/clickhouse/test/tables/test','1') ORDER BY tuple(); - CREATE MATERIALIZED VIEW test_mv_a Engine=ReplicatedMergeTree ('/clickhouse/test/tables/test_mv_a','1') order by tuple() AS SELECT A FROM test; - CREATE MATERIALIZED VIEW test_mv_b Engine=ReplicatedMergeTree ('/clickhouse/test/tables/test_mv_b','1') partition by A order by tuple() AS SELECT A FROM test; - CREATE MATERIALIZED VIEW test_mv_c Engine=ReplicatedMergeTree ('/clickhouse/test/tables/test_mv_c','1') order by tuple() AS SELECT A FROM test; - INSERT INTO test values(999); - INSERT INTO test values(999); SET max_partitions_per_insert_block = 3; INSERT INTO test SELECT number FROM numbers(10); ''' ) - assert int(node.query("SELECT count() FROM test")) == 11 - assert int(node.query("SELECT count() FROM test_mv_a")) == 11 - assert int(node.query("SELECT count() FROM test_mv_b")) == 1 - assert int(node.query("SELECT count() FROM test_mv_c")) == 1 + old_src, old_a, old_b, old_c = get_counts() + # number of rows in test_mv_a and test_mv_c depends on order of inserts into views + assert old_src == 11 + assert old_a in (1, 11) + assert old_b == 1 + assert old_c in (1, 11) node.query("INSERT INTO test SELECT number FROM numbers(10)") - assert int(node.query("SELECT count() FROM test")) == 11 - assert int(node.query("SELECT count() FROM test_mv_a")) == 11 - assert int(node.query("SELECT count() FROM test_mv_b")) == 1 - assert int(node.query("SELECT count() FROM test_mv_c")) == 1 + src, a, b, c = get_counts() + # no changes because of deduplication in source table + assert src == old_src + assert a == old_a + assert b == old_b + assert c == old_c node.query( ''' @@ -51,10 +67,11 @@ def test_basic(start_cluster): INSERT INTO test SELECT number FROM numbers(10); ''' ) - assert int(node.query("SELECT count() FROM test")) == 11 - assert int(node.query("SELECT count() FROM test_mv_a")) == 21 # first insert was succesfull with disabled dedup.. - assert int(node.query("SELECT count() FROM test_mv_b")) == 11 - assert int(node.query("SELECT count() FROM test_mv_c")) == 11 + src, a, b, c = get_counts() + assert src == 11 + assert a == old_a + 10 # first insert could be succesfull with disabled dedup + assert b == 11 + assert c == old_c + 10 with pytest.raises(QueryRuntimeException): node.query( @@ -71,8 +88,11 @@ def test_basic(start_cluster): INSERT INTO test SELECT number FROM numbers(100,10); ''' ) - - assert int(node.query("SELECT count() FROM test")) == 21 - assert int(node.query("SELECT count() FROM test_mv_a")) == 31 - assert int(node.query("SELECT count() FROM test_mv_b")) == 21 - assert int(node.query("SELECT count() FROM test_mv_c")) == 21 + + src, a, b, c = get_counts() + assert src == 21 + assert a == old_a + 20 + assert b == 21 + assert c == old_c + 20 + + From 0cd49d746f7aa7be71e801fa589923a74a489b60 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Thu, 9 Apr 2020 23:00:57 +0300 Subject: [PATCH 131/743] several fixes --- src/Interpreters/DictionaryReader.h | 2 ++ src/Interpreters/ExpressionAnalyzer.cpp | 39 +++++++++++++------------ src/Interpreters/TableJoin.cpp | 23 +++++++++++++++ src/Interpreters/TableJoin.h | 1 + 4 files changed, 46 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/DictionaryReader.h b/src/Interpreters/DictionaryReader.h index c1cd3e9ef1d..bb13758f46c 100644 --- a/src/Interpreters/DictionaryReader.h +++ b/src/Interpreters/DictionaryReader.h @@ -41,6 +41,7 @@ public: result.type = prepare_function->getReturnType(); if (result.type->getTypeId() != expected_type) throw Exception("Type mismatch in dictionary reader for: " + column_name, ErrorCodes::TYPE_MISMATCH); + block.insert(result); function = prepare_function->prepare(block, arg_positions, result_pos); } @@ -113,6 +114,7 @@ public: { size_t column_name_pos = key_size + i; auto & column = result_header.getByPosition(i); + arguments_get[1].column = DataTypeString().createColumnConst(1, src_column_names[i]); ColumnNumbers positions_get{0, column_name_pos, key_position}; functions_get.emplace_back(FunctionWrapper( *dict_get, arguments_get, sample_block, positions_get, column.name, column.type->getTypeId())); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index fbf2b663f3b..2ce7b373178 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -520,18 +520,9 @@ static ExpressionActionsPtr createJoinedBlockActions(const Context & context, co return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false); } -static bool allowDictJoin(const TableJoin & table_join, const Context & context, String & dict_name) +static bool allowDictJoin(StoragePtr joined_storage, const Context & context, String & dict_name, String & key_name) { - if (!table_join.joined_storage) - return false; - - const Names & right_keys = table_join.keyNamesRight(); - if (right_keys.size() != 1) - return false; - - const String & key_name = right_keys[0]; /// TODO: compound name - - auto * dict = dynamic_cast(table_join.joined_storage.get()); + auto * dict = dynamic_cast(joined_storage.get()); if (!dict) return false; @@ -541,20 +532,30 @@ static bool allowDictJoin(const TableJoin & table_join, const Context & context, return false; const DictionaryStructure & structure = dictionary->getStructure(); - return structure.id && (structure.id->name == key_name); /// key is UInt64 + if (structure.id) + { + key_name = structure.id->name; + return true; + } + return false; } -static std::shared_ptr makeJoin(std::shared_ptr analyzed_join, const Block & sample_block, - const Names & original_right_columns, const Context & context) +static std::shared_ptr makeJoin(std::shared_ptr analyzed_join, const Block & sample_block, const Context & context) { bool allow_merge_join = analyzed_join->allowMergeJoin(); + /// HashJoin with Dictionary optimisation String dict_name; - if (allowDictJoin(*analyzed_join, context, dict_name)) + String key_name; + if (analyzed_join->joined_storage && allowDictJoin(analyzed_join->joined_storage, context, dict_name, key_name)) { - analyzed_join->dictionary_reader = std::make_shared( - dict_name, original_right_columns, sample_block.getNamesAndTypesList(), context); - return std::make_shared(analyzed_join, sample_block); + Names original_names; + NamesAndTypesList result_columns; + if (analyzed_join->allowDictJoin(key_name, sample_block, original_names, result_columns)) + { + analyzed_join->dictionary_reader = std::make_shared(dict_name, original_names, result_columns, context); + return std::make_shared(analyzed_join, sample_block); + } } if (analyzed_join->forceHashJoin() || (analyzed_join->preferMergeJoin() && !allow_merge_join)) @@ -601,7 +602,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQuer /// TODO You do not need to set this up when JOIN is only needed on remote servers. subquery_for_join.setJoinActions(joined_block_actions); /// changes subquery_for_join.sample_block inside - subquery_for_join.join = makeJoin(syntax->analyzed_join, subquery_for_join.sample_block, original_right_columns, context); + subquery_for_join.join = makeJoin(syntax->analyzed_join, subquery_for_join.sample_block, context); /// Do not make subquery for join over dictionary. if (syntax->analyzed_join->dictionary_reader) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 30b5e8e4483..5e57c740bf1 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -238,4 +238,27 @@ bool TableJoin::allowMergeJoin() const return allow_merge_join; } +bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_block, Names & names, NamesAndTypesList & result_columns) const +{ + const Names & right_keys = keyNamesRight(); + if (right_keys.size() != 1) + return false; + + for (auto & col : sample_block) + { + String original = original_names.find(col.name)->second; + if (col.name == right_keys[0]) + { + if (original != dict_key) + return false; /// JOIN key != Dictionary key + continue; /// do not extract key column + } + + names.push_back(original); + result_columns.push_back({col.name, col.type}); + } + + return true; +} + } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 4cde414e270..c8c51918e27 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -96,6 +96,7 @@ public: const SizeLimits & sizeLimits() const { return size_limits; } VolumePtr getTemporaryVolume() { return tmp_volume; } bool allowMergeJoin() const; + bool allowDictJoin(const String & dict_key, const Block & sample_block, Names &, NamesAndTypesList &) const; bool preferMergeJoin() const { return join_algorithm == JoinAlgorithm::PREFER_PARTIAL_MERGE; } bool forceMergeJoin() const { return join_algorithm == JoinAlgorithm::PARTIAL_MERGE; } bool forceHashJoin() const { return join_algorithm == JoinAlgorithm::HASH; } From cd2eac9f930de2ad9b7320ae2b79467f434e5f97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Thu, 9 Apr 2020 23:52:53 +0300 Subject: [PATCH 132/743] codestyle fixed --- src/Dictionaries/ClickHouseDictionarySource.cpp | 2 +- src/Dictionaries/DictionarySourceFactory.cpp | 4 ++-- src/Dictionaries/DictionarySourceHelpers.cpp | 8 ++++---- src/Dictionaries/DictionarySourceHelpers.h | 6 ++---- src/Dictionaries/ExecutableDictionarySource.cpp | 1 - 5 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index aa06e1b8660..5ad4bcd3af0 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -229,7 +229,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) bool /* check_config */) -> DictionarySourcePtr { Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); - + /// Note that processors are not supported yet (see constructor), /// hence it is not possible to override experimental_use_processors setting return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context_local_copy); diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index 8431e065dd4..25931e6a724 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -84,11 +84,11 @@ DictionarySourcePtr DictionarySourceFactory::create( { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); - + if (keys.empty() || keys.size() > 2) throw Exception{name + ": element dictionary.source should have one or two child elements", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; - + const auto & source_type = keys.front(); const auto found = registered_sources.find(source_type); diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 7dfa67b5167..bbf6fb8fb1b 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -53,12 +53,12 @@ void formatKeys( } Context copyContextAndApplySettings( - const std::string & config_prefix, - const Context & context, + const std::string & config_prefix, + const Context & context, const Poco::Util::AbstractConfiguration & config) { Context local_context(context); - if (config.has(config_prefix + ".settings")) + if (config.has(config_prefix + ".settings")) { const auto prefix = config_prefix + ".settings"; Settings settings; @@ -66,7 +66,7 @@ Context copyContextAndApplySettings( settings.loadSettingsFromConfig(prefix, config); local_context.setSettings(settings); } - + return local_context; } diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 7dc5f319432..ac173b0178d 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -14,8 +14,6 @@ using BlockOutputStreamPtr = std::shared_ptr; struct DictionaryStructure; class Context; - - /// Write keys to block output stream. /// For simple key @@ -30,8 +28,8 @@ void formatKeys( /// Used for applying settings to copied context in some register[...]Source functions Context copyContextAndApplySettings( - const std::string & config_prefix, - const Context & context, + const std::string & config_prefix, + const Context & context, const Poco::Util::AbstractConfiguration & config); } diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 6f4cd747b87..34943d62b44 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -15,7 +15,6 @@ #include "DictionarySourceHelpers.h" #include "DictionaryStructure.h" #include "registerDictionaries.h" -#include "DictionarySourceHelpers.h" namespace DB { From 22a55c49516a7c7ad273959e1cdd293eacd5e6ff Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 18:45:29 -0300 Subject: [PATCH 133/743] Doc. merge_tree_settings --- .../settings/merge_tree_settings.md | 75 +++++++++++++++++++ 1 file changed, 75 insertions(+) create mode 100644 docs/ru/operations/settings/merge_tree_settings.md diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md new file mode 100644 index 00000000000..4bb8791ef82 --- /dev/null +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -0,0 +1,75 @@ +# Настройки MergeTree таблиц {#merge-tree-settings} + +Значения по умолчанию (для всех таблиц) задаются в config.xml в секции merge_tree. + +Пример: +```text + + 5 + +``` + +Эти значения можно задать (перекрыть) у таблиц задав в секции `Settings` у команды `CREATE TABLE`. + +Пример: +```sql +CREATE TABLE foo +( + `A` Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS max_suspicious_broken_parts = 500; +``` + +Или изменить с помощью команды `ALTER TABLE ... MODIFY SETTING`. + +Пример: +```sql +ALTER TABLE foo + MODIFY SETTING max_suspicious_broken_parts = 100; +``` + + +## parts_to_throw_insert {#parts-to-throw-insert} + +Eсли число кусков в партиции превышает значение `parts_to_throw_insert` INSERT прерывается с исключением 'Too many parts (300). Merges are processing significantly slower than inserts'. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: 300. + +Для достижения максимальной производительности запросов `SELECT` необходимо минимизировать количество обрабатываемых кусков, см. [Дизайн MergeTree](../../development/architecture.md#merge-tree). +Можно установить большее значение 600 (1200), это уменьшит вероятность возникновения ошибки 'Too many parts', но в тоже время вы позже заметите возможную проблему со слияниями. + + +## parts_to_delay_insert {#parts-to-delay-insert} + +Eсли число кусков в партиции превышает значение `parts_to_delay_insert` INSERT искусственно замедляется. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: 150. + +ClickHouse искусственно выполняет `INSERT` дольше (добавляет 'sleep'), чтобы фоновый механизм слияния успевал слиять куски быстрее чем они добавляются. + + +## max_delay_to_insert {#max-delay-to-insert} + +Время в секундах на которое будет замедлен `INSERT`, если число кусков в партиции превышает значение [parts_to_delay_insert](#parts-to-delay-insert) + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: 1. + +`INSERT` будет замедлен на max_delay_to_insert/(количество кусков в партиции - parts_to_delay_insert). +Т.е. если в партиции уже 299 кусков и parts_to_delay_insert = 150, max_delay_to_insert = 1, `INSERT` замедлится на 1/(299-150) (~0.5) секунд. + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/merge_tree_settings/) From bc8b724f5efd00b22a5e2e8293a50a5167282b5a Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 19:05:38 -0300 Subject: [PATCH 134/743] Update merge_tree_settings.md fix max_delay_to_insert formula --- docs/ru/operations/settings/merge_tree_settings.md | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index 4bb8791ef82..b0714368ed2 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -9,7 +9,7 @@ ``` -Эти значения можно задать (перекрыть) у таблиц задав в секции `Settings` у команды `CREATE TABLE`. +Эти значения можно задать (перекрыть) у таблиц в секции `SETTINGS` у команды `CREATE TABLE`. Пример: ```sql @@ -33,7 +33,7 @@ ALTER TABLE foo ## parts_to_throw_insert {#parts-to-throw-insert} -Eсли число кусков в партиции превышает значение `parts_to_throw_insert` INSERT прерывается с исключением 'Too many parts (300). Merges are processing significantly slower than inserts'. +Eсли число кусков в партиции превышает значение `parts_to_throw_insert` INSERT прерывается с исключением 'Too many parts (N). Merges are processing significantly slower than inserts'. Возможные значения: @@ -47,7 +47,7 @@ Eсли число кусков в партиции превышает знач ## parts_to_delay_insert {#parts-to-delay-insert} -Eсли число кусков в партиции превышает значение `parts_to_delay_insert` INSERT искусственно замедляется. +Eсли число кусков в партиции превышает значение `parts_to_delay_insert` `INSERT` искусственно замедляется. Возможные значения: @@ -68,8 +68,11 @@ ClickHouse искусственно выполняет `INSERT` дольше (д Значение по умолчанию: 1. -`INSERT` будет замедлен на max_delay_to_insert/(количество кусков в партиции - parts_to_delay_insert). -Т.е. если в партиции уже 299 кусков и parts_to_delay_insert = 150, max_delay_to_insert = 1, `INSERT` замедлится на 1/(299-150) (~0.5) секунд. +Величина задержи (в миллисекундах) для `INSERT` вычисляется по формуле +`pow(max_delay_to_insert * 1000, (1 + parts_count_in_partition - parts_to_delay_insert) / (parts_to_throw_insert - parts_to_delay_insert))` + +Т.е. если в партиции уже 299 кусков и parts_to_throw_insert =300, parts_to_delay_insert = 150, max_delay_to_insert = 1, `INSERT` замедлится на `pow( 1 * 1000, (1 + 299 - 150) / (300 - 150) ) = 1000` миллисекунд. + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/merge_tree_settings/) From 6a10d0df00caf9c952d87d18a87f96303bf9af91 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 19:08:35 -0300 Subject: [PATCH 135/743] Update merge_tree_settings.md fix max_delay_to_insert formula --- docs/ru/operations/settings/merge_tree_settings.md | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index b0714368ed2..5adda56a76b 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -69,9 +69,14 @@ ClickHouse искусственно выполняет `INSERT` дольше (д Значение по умолчанию: 1. Величина задержи (в миллисекундах) для `INSERT` вычисляется по формуле -`pow(max_delay_to_insert * 1000, (1 + parts_count_in_partition - parts_to_delay_insert) / (parts_to_throw_insert - parts_to_delay_insert))` -Т.е. если в партиции уже 299 кусков и parts_to_throw_insert =300, parts_to_delay_insert = 150, max_delay_to_insert = 1, `INSERT` замедлится на `pow( 1 * 1000, (1 + 299 - 150) / (300 - 150) ) = 1000` миллисекунд. +```code +max_k = parts_to_throw_insert - parts_to_delay_insert +k = 1 + parts_count_in_partition - parts_to_delay_insert +delay_milliseconds = pow(max_delay_to_insert * 1000, k / max_k) +``` + +Т.е. если в партиции уже 299 кусков и parts_to_throw_insert = 300, parts_to_delay_insert = 150, max_delay_to_insert = 1, `INSERT` замедлится на `pow( 1 * 1000, (1 + 299 - 150) / (300 - 150) ) = 1000` миллисекунд. From 825785283c2daae26d72b363c7d77232d20592e1 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 19:17:12 -0300 Subject: [PATCH 136/743] Update merge_tree_settings.md --- docs/ru/operations/settings/merge_tree_settings.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index 5adda56a76b..e9ec07c9fc8 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -33,7 +33,7 @@ ALTER TABLE foo ## parts_to_throw_insert {#parts-to-throw-insert} -Eсли число кусков в партиции превышает значение `parts_to_throw_insert` INSERT прерывается с исключением 'Too many parts (N). Merges are processing significantly slower than inserts'. +Eсли число кусков в партиции превышает значение `parts_to_throw_insert` INSERT прерывается с исключением `Too many parts (N). Merges are processing significantly slower than inserts`. Возможные значения: @@ -60,7 +60,7 @@ ClickHouse искусственно выполняет `INSERT` дольше (д ## max_delay_to_insert {#max-delay-to-insert} -Время в секундах на которое будет замедлен `INSERT`, если число кусков в партиции превышает значение [parts_to_delay_insert](#parts-to-delay-insert) +Величина в секундах, которая используется для расчета задержки `INSERT`, если число кусков в партиции превышает значение [parts_to_delay_insert](#parts-to-delay-insert). Возможные значения: @@ -68,7 +68,7 @@ ClickHouse искусственно выполняет `INSERT` дольше (д Значение по умолчанию: 1. -Величина задержи (в миллисекундах) для `INSERT` вычисляется по формуле +Величина задержи (в миллисекундах) для `INSERT` вычисляется по формуле: ```code max_k = parts_to_throw_insert - parts_to_delay_insert From c4c78f8e73f15292d07eef2fd79402baa26f92d8 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 19:19:42 -0300 Subject: [PATCH 137/743] Update merge_tree_settings.md --- docs/ru/operations/settings/merge_tree_settings.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index e9ec07c9fc8..4da8f126708 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -42,7 +42,8 @@ Eсли число кусков в партиции превышает знач Значение по умолчанию: 300. Для достижения максимальной производительности запросов `SELECT` необходимо минимизировать количество обрабатываемых кусков, см. [Дизайн MergeTree](../../development/architecture.md#merge-tree). -Можно установить большее значение 600 (1200), это уменьшит вероятность возникновения ошибки 'Too many parts', но в тоже время вы позже заметите возможную проблему со слияниями. + +Можно установить большее значение 600 (1200), это уменьшит вероятность возникновения ошибки `Too many parts`, но в тоже время вы позже обнаружите возможную проблему со слияниями (например из-за недостатка места на диске), и деградацию производительности `SELECT`. ## parts_to_delay_insert {#parts-to-delay-insert} From 58067438cdb7b010308a1faa343ca84cd988f8aa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Apr 2020 01:22:43 +0300 Subject: [PATCH 138/743] add fallback for renameat2 --- src/Common/rename.cpp | 79 ++++++++++++++++--- src/Common/rename.h | 12 +-- ...lized_view_and_too_many_parts_zookeeper.sh | 8 +- 3 files changed, 74 insertions(+), 25 deletions(-) diff --git a/src/Common/rename.cpp b/src/Common/rename.cpp index 039c4304f69..76e24db3954 100644 --- a/src/Common/rename.cpp +++ b/src/Common/rename.cpp @@ -1,23 +1,52 @@ #include #include +#include -#if defined(_GNU_SOURCE) +#if defined(linux) || defined(__linux) || defined(__linux__) #include #include #include +#include #endif namespace DB { -#if defined(__NR_renameat2) namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ATOMIC_RENAME_FAIL; extern const int SYSTEM_ERROR; + extern const int UNSUPPORTED_METHOD; + extern const int FILE_ALREADY_EXISTS; } +static bool supportsRenameat2Impl() +{ +#if defined(__NR_renameat2) + /// renameat2 is available in linux since 3.15 + struct utsname sysinfo; + if (uname(&sysinfo)) + return false; + char * point = nullptr; + long v_major = strtol(sysinfo.release, &point, 10); + + errno = 0; + if (errno || *point != '.' || v_major < 3) + return false; + if (3 < v_major) + return true; + + errno = 0; + long v_minor = strtol(point + 1, nullptr, 10); + return !errno && 15 <= v_minor; +#else + return false; +#endif +} + +#if defined(__NR_renameat2) + static void renameat2(const std::string & old_path, const std::string & new_path, int flags) { if (old_path.empty() || new_path.empty()) @@ -39,29 +68,55 @@ static void renameat2(const std::string & old_path, const std::string & new_path } #else -#define RENAME_NOREPLACE 0 -#define RENAME_EXCHANGE 0 +#define RENAME_NOREPLACE -1 +#define RENAME_EXCHANGE -1 -namespace ErrorCodes -{ - extern const int UNSUPPORTED_METHOD; -} - -[[noreturn]] static void renameat2(const std::string &, const std::string &, int) +[[noreturn]] +static void renameat2(const std::string &, const std::string &, int) { throw Exception("Compiled without renameat2() support", ErrorCodes::UNSUPPORTED_METHOD); } #endif +static void renameNoReplaceFallback(const std::string & old_path, const std::string & new_path) +{ + /// NOTE it's unsafe + if (Poco::File{new_path}.exists()) + throw Exception("File " + new_path + " exists", ErrorCodes::FILE_ALREADY_EXISTS); + Poco::File{old_path}.renameTo(new_path); +} + +/// Do not use [[noreturn]] to avoid warnings like "code will never be executed" in other places +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wmissing-noreturn" +static void renameExchangeFallback(const std::string &, const std::string &) +{ + throw Exception("System call renameat2() is not supported", ErrorCodes::UNSUPPORTED_METHOD); +} +#pragma GCC diagnostic pop + + +bool supportsRenameat2() +{ + static bool supports = supportsRenameat2Impl(); + return supports; +} + void renameNoReplace(const std::string & old_path, const std::string & new_path) { - renameat2(old_path, new_path, RENAME_NOREPLACE); + if (supportsRenameat2()) + renameat2(old_path, new_path, RENAME_NOREPLACE); + else + renameNoReplaceFallback(old_path, new_path); } void renameExchange(const std::string & old_path, const std::string & new_path) { - renameat2(old_path, new_path, RENAME_EXCHANGE); + if (supportsRenameat2()) + renameat2(old_path, new_path, RENAME_EXCHANGE); + else + renameExchangeFallback(old_path, new_path); } } diff --git a/src/Common/rename.h b/src/Common/rename.h index 36e9fb7c73e..333f85541f1 100644 --- a/src/Common/rename.h +++ b/src/Common/rename.h @@ -1,23 +1,17 @@ #pragma once #include -#if defined(_GNU_SOURCE) -#include -#endif namespace DB { +/// Returns true, if the following functions supported by the system +bool supportsRenameat2(); + /// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception -#if !defined(__NR_renameat2) -[[noreturn]] -#endif void renameNoReplace(const std::string & old_path, const std::string & new_path); /// Atomically exchange oldpath and newpath. Throw exception if some of them does not exist -#if !defined(__NR_renameat2) -[[noreturn]] -#endif void renameExchange(const std::string & old_path, const std::string & new_path); } diff --git a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh index dca2912b032..182e0ffcecd 100755 --- a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh +++ b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh @@ -23,10 +23,10 @@ fi echo ${CLICKHOUSE_CLIENT} --query "SELECT _table, d FROM merge('${CLICKHOUSE_DATABASE}', '^[abc]\$') ORDER BY _table" -${CLICKHOUSE_CLIENT} --query "DROP TABLE root NO DELAY" +${CLICKHOUSE_CLIENT} --query "DROP TABLE root" ${CLICKHOUSE_CLIENT} --query "DROP TABLE a NO DELAY" -${CLICKHOUSE_CLIENT} --query "DROP TABLE b NO DELAY" -${CLICKHOUSE_CLIENT} --query "DROP TABLE c NO DELAY" +${CLICKHOUSE_CLIENT} --query "DROP TABLE b" +${CLICKHOUSE_CLIENT} --query "DROP TABLE c" sleep 2 # Deduplication check for non-replicated root table @@ -36,6 +36,6 @@ ${CLICKHOUSE_CLIENT} --query "CREATE MATERIALIZED VIEW a (d UInt64) ENGINE = Rep ${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; ${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; ${CLICKHOUSE_CLIENT} --query "SELECT * FROM a"; -${CLICKHOUSE_CLIENT} --query "DROP TABLE root NO DELAY" +${CLICKHOUSE_CLIENT} --query "DROP TABLE root" ${CLICKHOUSE_CLIENT} --query "DROP TABLE a NO DELAY" sleep 1 From 03ed9d59b0cc0c534e1b9d7f76d1d40b5d55fad1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Apr 2020 03:08:43 +0300 Subject: [PATCH 139/743] add symlinks --- src/Databases/DatabaseAtomic.cpp | 76 ++++++++++++++++++++++++++------ src/Databases/DatabaseAtomic.h | 11 ++++- 2 files changed, 72 insertions(+), 15 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 2117680d4d9..7a54234cb06 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -29,7 +29,9 @@ public: DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_) : DatabaseOrdinary(name_, metadata_path_, "store/", "DatabaseAtomic (" + name_ + ")", context_) + , path_to_table_symlinks(context_.getPath() + "data/" + escapeForFileName(name_) + "/") { + Poco::File(path_to_table_symlinks).createDirectories(); } String DatabaseAtomic::getTableDataPath(const String & table_name) const @@ -52,6 +54,7 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const void DatabaseAtomic::drop(const Context &) { + Poco::File(path_to_table_symlinks).remove(true); Poco::File(getMetadataPath()).remove(true); } @@ -64,6 +67,7 @@ void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, assertDetachedTableNotInUse(table->getStorageID().uuid); DatabaseWithDictionaries::attachTableUnlocked(name, table, relative_table_path); table_name_to_path.emplace(std::make_pair(name, relative_table_path)); + tryCreateSymlink(name, relative_table_path); } StoragePtr DatabaseAtomic::detachTable(const String & name) @@ -74,6 +78,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) table_name_to_path.erase(name); detached_tables.emplace(table->getStorageID().uuid, table); not_in_use = cleenupDetachedTables(); + tryRemoveSymlink(name); return table; } @@ -90,6 +95,7 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool DatabaseWithDictionaries::detachTableUnlocked(table_name); /// Should never throw table_name_to_path.erase(table_name); } + tryRemoveSymlink(table_name); DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); } @@ -106,31 +112,23 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n } auto & other_db = dynamic_cast(to_database); - StoragePtr table = tryGetTable(context, table_name); - StoragePtr other_table; - if (exchange) - other_table = other_db.tryGetTable(context, to_table_name); - - if (!table) - throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - if (exchange && !other_table) - throw Exception("Table " + backQuote(other_db.getDatabaseName()) + "." + backQuote(to_table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - String old_metadata_path = getObjectMetadataPath(table_name); String new_metadata_path = to_database.getObjectMetadataPath(to_table_name); - auto detach = [](DatabaseAtomic & db, const String & table_name_) + auto detach = [this](DatabaseAtomic & db, const String & table_name_) { auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second; db.tables.erase(table_name_); db.table_name_to_path.erase(table_name_); + tryRemoveSymlink(table_name_); return table_data_path_saved; }; - auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_) + auto attach = [this](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_) { db.tables.emplace(table_name_, table_); db.table_name_to_path.emplace(table_name_, table_data_path_); + tryCreateSymlink(table_name_, table_data_path_); }; String table_data_path; @@ -155,6 +153,11 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n db_lock = std::unique_lock{mutex}; } + StoragePtr table = getTableUnlocked(table_name); + StoragePtr other_table; + if (exchange) + other_table = other_db.getTableUnlocked(to_table_name); + if (exchange) renameExchange(old_metadata_path, new_metadata_path); else @@ -199,7 +202,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora Poco::File(table_metadata_tmp_path).remove(); throw; } - + tryCreateSymlink(query.table, table_data_path); } void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) @@ -251,5 +254,52 @@ DatabaseTablesIteratorPtr DatabaseAtomic::getTablesWithDictionaryTablesIterator( return std::make_unique(std::move(typeid_cast(*base_iter))); } +void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) +{ + /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken + if (has_force_restore_data_flag) + Poco::File(path_to_table_symlinks).remove(true); + + DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag); + + if (has_force_restore_data_flag) + { + NameToPathMap table_names; + { + std::lock_guard lock{mutex}; + table_names = table_name_to_path; + } + for (const auto & table : table_names) + tryCreateSymlink(table.first, table.second); + } +} + +void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & actual_data_path) +{ + try + { + String link = path_to_table_symlinks + escapeForFileName(table_name); + String data = global_context.getPath() + actual_data_path; + Poco::File{data}.linkTo(link, Poco::File::LINK_SYMBOLIC); + } + catch (...) + { + tryLogCurrentException(log); + } +} + +void DatabaseAtomic::tryRemoveSymlink(const String & table_name) +{ + try + { + String path = path_to_table_symlinks + escapeForFileName(table_name); + Poco::File{path}.remove(); + } + catch (...) + { + tryLogCurrentException(log); + } +} + } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index e68e924d008..73a4c0166a1 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -36,19 +36,26 @@ public: DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const FilterByNameFunction & filter_by_dictionary_name) override; + void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; + private: void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path) override; void assertDetachedTableNotInUse(const UUID & uuid); - typedef std::map DetachedTables; + typedef std::unordered_map DetachedTables; DetachedTables cleenupDetachedTables(); + void tryCreateSymlink(const String & table_name, const String & actual_data_path); + void tryRemoveSymlink(const String & table_name); + //TODO store path in DatabaseWithOwnTables::tables - std::map table_name_to_path; + typedef std::unordered_map NameToPathMap; + NameToPathMap table_name_to_path; DetachedTables detached_tables; + const String path_to_table_symlinks; }; } From 14e22ab6b355fe93a9e570ec565a10680f22435f Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 21:16:48 -0300 Subject: [PATCH 140/743] old_parts_lifetime --- docs/ru/operations/settings/merge_tree_settings.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index 4da8f126708..98fef837740 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -79,6 +79,20 @@ delay_milliseconds = pow(max_delay_to_insert * 1000, k / max_k) Т.е. если в партиции уже 299 кусков и parts_to_throw_insert = 300, parts_to_delay_insert = 150, max_delay_to_insert = 1, `INSERT` замедлится на `pow( 1 * 1000, (1 + 299 - 150) / (300 - 150) ) = 1000` миллисекунд. +## old_parts_lifetime {#old_parts_lifetime} + +Время (в секундах) хранения неактивных кусков, для защиты от потери данных при спонтанной перезагрузке сервера или О.С. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: 480. + +После слияния нескольких кусков в новый кусок, ClickHouse помечает исходные куски как неактивные и удаляет после `old_parts_lifetime` секунд. +Неактивные куски удаляются если они не используются в текущих запросах, т.е. если счетчик ссылок куска -- `refcount` равен нулю. +Неактивные куски удаляются не сразу, потому что при записи нового куска не вызывается `fsync`, т.е. некоторое время новый кусок находится только в оперативной памяти сервера (кеше О.С.). Т.о. при спонтанной перезагрузке сервера, новый (смерженный) кусок может быть потерян или испорчен, в этом случае ClickHouse при загрузке при проверке целостности кусков обнаружит это и вернет неактивные куски в активные и позже заново их смержит. Сломанный кусок в этом случае переименовывается (добавляется префикс broken) и перемещается в папку detached. +Стандартное значение Linux dirty_expire_centisecs - 30 секунд (максимальное время, которое записанные данные хранятся только в оперативной памяти), но при больших нагрузках на дисковую систему, данные могут быть записаны намного позже (30 сек.), экспериментально было найдено время - 480 секунд, за которое почти гарантировано новый кусок будет записан на диск и безопасно удалять неактивные куски. [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/merge_tree_settings/) From eb57e54a21a00d22bb3ddfde66e91d57e36f7af6 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 21:23:12 -0300 Subject: [PATCH 141/743] Update merge_tree_settings.md --- docs/ru/operations/settings/merge_tree_settings.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index 98fef837740..e3be061526d 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -89,9 +89,11 @@ delay_milliseconds = pow(max_delay_to_insert * 1000, k / max_k) Значение по умолчанию: 480. -После слияния нескольких кусков в новый кусок, ClickHouse помечает исходные куски как неактивные и удаляет после `old_parts_lifetime` секунд. +После слияния нескольких кусков в новый кусок, ClickHouse помечает исходные куски как неактивные и удаляет их после `old_parts_lifetime` секунд. Неактивные куски удаляются если они не используются в текущих запросах, т.е. если счетчик ссылок куска -- `refcount` равен нулю. -Неактивные куски удаляются не сразу, потому что при записи нового куска не вызывается `fsync`, т.е. некоторое время новый кусок находится только в оперативной памяти сервера (кеше О.С.). Т.о. при спонтанной перезагрузке сервера, новый (смерженный) кусок может быть потерян или испорчен, в этом случае ClickHouse при загрузке при проверке целостности кусков обнаружит это и вернет неактивные куски в активные и позже заново их смержит. Сломанный кусок в этом случае переименовывается (добавляется префикс broken) и перемещается в папку detached. + +Неактивные куски удаляются не сразу, потому что при записи нового куска не вызывается `fsync`, т.е. некоторое время новый кусок находится только в оперативной памяти сервера (кеше О.С.). Т.о. при спонтанной перезагрузке сервера, новый (смерженный) кусок может быть потерян или испорчен. В этом случае ClickHouse при загрузке при проверке целостности кусков обнаружит это и вернет неактивные куски в список активных и позже заново их смержит. Сломанный кусок в этом случае переименовывается (добавляется префикс broken_) и перемещается в папку detached. + Стандартное значение Linux dirty_expire_centisecs - 30 секунд (максимальное время, которое записанные данные хранятся только в оперативной памяти), но при больших нагрузках на дисковую систему, данные могут быть записаны намного позже (30 сек.), экспериментально было найдено время - 480 секунд, за которое почти гарантировано новый кусок будет записан на диск и безопасно удалять неактивные куски. From af73f555817ae39d1abaf82504804dd91f39d128 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 21:24:37 -0300 Subject: [PATCH 142/743] Update merge_tree_settings.md --- docs/ru/operations/settings/merge_tree_settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index e3be061526d..46bf78dda0f 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -79,7 +79,7 @@ delay_milliseconds = pow(max_delay_to_insert * 1000, k / max_k) Т.е. если в партиции уже 299 кусков и parts_to_throw_insert = 300, parts_to_delay_insert = 150, max_delay_to_insert = 1, `INSERT` замедлится на `pow( 1 * 1000, (1 + 299 - 150) / (300 - 150) ) = 1000` миллисекунд. -## old_parts_lifetime {#old_parts_lifetime} +## old_parts_lifetime {#old-parts-lifetime} Время (в секундах) хранения неактивных кусков, для защиты от потери данных при спонтанной перезагрузке сервера или О.С. From aba149d8a26def16efb71afeb95ba03c2f94283f Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 9 Apr 2020 22:35:28 -0300 Subject: [PATCH 143/743] Update merge_tree_settings.md --- docs/ru/operations/settings/merge_tree_settings.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index 46bf78dda0f..afd88694def 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -33,7 +33,7 @@ ALTER TABLE foo ## parts_to_throw_insert {#parts-to-throw-insert} -Eсли число кусков в партиции превышает значение `parts_to_throw_insert` INSERT прерывается с исключением `Too many parts (N). Merges are processing significantly slower than inserts`. +Eсли число кусков в партиции превышает значение `parts_to_throw_insert`, INSERT прерывается с исключением `Too many parts (N). Merges are processing significantly slower than inserts`. Возможные значения: @@ -43,12 +43,12 @@ Eсли число кусков в партиции превышает знач Для достижения максимальной производительности запросов `SELECT` необходимо минимизировать количество обрабатываемых кусков, см. [Дизайн MergeTree](../../development/architecture.md#merge-tree). -Можно установить большее значение 600 (1200), это уменьшит вероятность возникновения ошибки `Too many parts`, но в тоже время вы позже обнаружите возможную проблему со слияниями (например из-за недостатка места на диске), и деградацию производительности `SELECT`. +Можно установить большее значение 600 (1200), это уменьшит вероятность возникновения ошибки `Too many parts`, но в тоже время вы позже обнаружите возможную проблему со слияниями (например, из-за недостатка места на диске) и деградацию производительности `SELECT`. ## parts_to_delay_insert {#parts-to-delay-insert} -Eсли число кусков в партиции превышает значение `parts_to_delay_insert` `INSERT` искусственно замедляется. +Eсли число кусков в партиции превышает значение `parts_to_delay_insert`, `INSERT` искусственно замедляется. Возможные значения: @@ -56,7 +56,7 @@ Eсли число кусков в партиции превышает знач Значение по умолчанию: 150. -ClickHouse искусственно выполняет `INSERT` дольше (добавляет 'sleep'), чтобы фоновый механизм слияния успевал слиять куски быстрее чем они добавляются. +ClickHouse искусственно выполняет `INSERT` дольше (добавляет 'sleep'), чтобы фоновый механизм слияния успевал слиять куски быстрее, чем они добавляются. ## max_delay_to_insert {#max-delay-to-insert} @@ -90,11 +90,11 @@ delay_milliseconds = pow(max_delay_to_insert * 1000, k / max_k) Значение по умолчанию: 480. После слияния нескольких кусков в новый кусок, ClickHouse помечает исходные куски как неактивные и удаляет их после `old_parts_lifetime` секунд. -Неактивные куски удаляются если они не используются в текущих запросах, т.е. если счетчик ссылок куска -- `refcount` равен нулю. +Неактивные куски удаляются, если они не используются в текущих запросах, т.е. если счетчик ссылок куска -- `refcount` равен нулю. -Неактивные куски удаляются не сразу, потому что при записи нового куска не вызывается `fsync`, т.е. некоторое время новый кусок находится только в оперативной памяти сервера (кеше О.С.). Т.о. при спонтанной перезагрузке сервера, новый (смерженный) кусок может быть потерян или испорчен. В этом случае ClickHouse при загрузке при проверке целостности кусков обнаружит это и вернет неактивные куски в список активных и позже заново их смержит. Сломанный кусок в этом случае переименовывается (добавляется префикс broken_) и перемещается в папку detached. +Неактивные куски удаляются не сразу, потому что при записи нового куска не вызывается `fsync`, т.е. некоторое время новый кусок находится только в оперативной памяти сервера (кеше О.С.). Т.о. при спонтанной перезагрузке сервера новый (смерженный) кусок может быть потерян или испорчен. В этом случае ClickHouse в процессе старта при проверке целостности кусков обнаружит проблему, вернет неактивные куски в список активных и позже заново их смержит. Сломанный кусок в этом случае переименовывается (добавляется префикс broken_) и перемещается в папку detached. Если проверка целостности не обнаруживает проблем в смерженном куске, то исходные неактивные куски переименовываются (добавляется префикс ignored_) и перемещаются в папку detached. -Стандартное значение Linux dirty_expire_centisecs - 30 секунд (максимальное время, которое записанные данные хранятся только в оперативной памяти), но при больших нагрузках на дисковую систему, данные могут быть записаны намного позже (30 сек.), экспериментально было найдено время - 480 секунд, за которое почти гарантировано новый кусок будет записан на диск и безопасно удалять неактивные куски. +Стандартное значение Linux dirty_expire_centisecs - 30 секунд (максимальное время, которое записанные данные хранятся только в оперативной памяти), но при больших нагрузках на дисковую систему, данные могут быть записаны намного позже. Экспериментально было найдено время - 480 секунд, за которое гарантированно новый кусок будет записан на диск. [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/merge_tree_settings/) From 50decc755d8aa594d74ad9393aca37ea6d614ce9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Apr 2020 04:35:37 +0300 Subject: [PATCH 144/743] fix rename materialized view --- src/Databases/DatabaseAtomic.cpp | 16 ++++++++++- src/Storages/StorageMaterializedView.cpp | 35 ++++++++++-------------- src/Storages/StorageMaterializedView.h | 1 + 3 files changed, 31 insertions(+), 21 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 7a54234cb06..66183f450fb 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -111,6 +112,7 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n return; } auto & other_db = dynamic_cast(to_database); + bool inside_database = this == &other_db; String old_metadata_path = getObjectMetadataPath(table_name); String new_metadata_path = to_database.getObjectMetadataPath(to_table_name); @@ -131,10 +133,18 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n tryCreateSymlink(table_name_, table_data_path_); }; + auto assertCanMoveMatView = [&](const StoragePtr & table_) + { + if (inside_database) + return; + if (const auto * mv = dynamic_cast(table_.get())) + if (mv->hasInnerTable()) + throw Exception("Cannot move MaterializedView with inner table to other database", ErrorCodes::NOT_IMPLEMENTED); + }; + String table_data_path; String other_table_data_path; - bool inside_database = this == &other_db; if (inside_database && table_name == to_table_name) return; @@ -154,9 +164,13 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n } StoragePtr table = getTableUnlocked(table_name); + assertCanMoveMatView(table); StoragePtr other_table; if (exchange) + { other_table = other_db.getTableUnlocked(to_table_name); + assertCanMoveMatView(other_table); + } if (exchange) renameExchange(old_metadata_path, new_metadata_path); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index ec7831c706a..35e7b59e1ab 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -330,33 +330,28 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) { auto old_table_id = getStorageID(); + bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID(); - if (has_inner_table && tryGetTargetTable()) + if (has_inner_table && tryGetTargetTable() && !from_atomic_to_atomic_database) { - auto old_target_table_name = generateInnerTableName(old_table_id); auto new_target_table_name = generateInnerTableName(new_table_id); - if (old_table_id.database_name != new_table_id.database_name || - old_target_table_name != new_target_table_name) - { + auto rename = std::make_shared(); - auto rename = std::make_shared(); + ASTRenameQuery::Table from; + from.database = target_table_id.database_name; + from.table = target_table_id.table_name; - ASTRenameQuery::Table from; - from.database = target_table_id.database_name; - from.table = target_table_id.table_name; + ASTRenameQuery::Table to; + to.database = target_table_id.database_name; + to.table = new_target_table_name; - ASTRenameQuery::Table to; - to.database = target_table_id.database_name; - to.table = new_target_table_name; + ASTRenameQuery::Element elem; + elem.from = from; + elem.to = to; + rename->elements.emplace_back(elem); - ASTRenameQuery::Element elem; - elem.from = from; - elem.to = to; - rename->elements.emplace_back(elem); - - InterpreterRenameQuery(rename, global_context).execute(); - target_table_id.table_name = new_target_table_name; - } + InterpreterRenameQuery(rename, global_context).execute(); + target_table_id.table_name = new_target_table_name; } IStorage::renameInMemory(new_table_id); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 175529fd940..97496d33f94 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -21,6 +21,7 @@ public: ASTPtr getSelectQuery() const { return select->clone(); } ASTPtr getInnerQuery() const { return inner_query->clone(); } + bool hasInnerTable() const { return has_inner_table; } NameAndTypePair getColumn(const String & column_name) const override; bool hasColumn(const String & column_name) const override; From d3acce8afa18f5122ad4b49c2966d7c5f4f4ad2e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 5 Apr 2020 16:07:05 +0300 Subject: [PATCH 145/743] Fix build. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../Transforms/MergingSortedTransform.cpp | 331 ------------------ .../Transforms/MergingSortedTransform.h | 160 --------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 4 files changed, 2 insertions(+), 493 deletions(-) delete mode 100644 src/Processors/Transforms/MergingSortedTransform.cpp delete mode 100644 src/Processors/Transforms/MergingSortedTransform.h diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 514efb90a00..4b85b222949 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -72,6 +72,7 @@ #include #include +#include #include #include #include @@ -84,7 +85,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Processors/Transforms/MergingSortedTransform.cpp b/src/Processors/Transforms/MergingSortedTransform.cpp deleted file mode 100644 index b9e74277023..00000000000 --- a/src/Processors/Transforms/MergingSortedTransform.cpp +++ /dev/null @@ -1,331 +0,0 @@ -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -MergingSortedTransform::MergingSortedTransform( - const Block & header, - size_t num_inputs, - const SortDescription & description_, - size_t max_block_size_, - UInt64 limit_, - bool quiet_, - bool have_all_inputs_) - : IProcessor(InputPorts(num_inputs, header), {header}) - , description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) - , have_all_inputs(have_all_inputs_) - , merged_data(header), source_chunks(num_inputs), cursors(num_inputs) -{ - auto & sample = outputs.front().getHeader(); - /// Replace column names in description to positions. - for (auto & column_description : description) - { - has_collation |= column_description.collator != nullptr; - if (!column_description.column_name.empty()) - { - column_description.column_number = sample.getPositionByName(column_description.column_name); - column_description.column_name.clear(); - } - } -} - -void MergingSortedTransform::addInput() -{ - if (have_all_inputs) - throw Exception("MergingSortedTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); - - inputs.emplace_back(outputs.front().getHeader(), this); - source_chunks.emplace_back(); - cursors.emplace_back(); -} - -void MergingSortedTransform::setHaveAllInputs() -{ - if (have_all_inputs) - throw Exception("MergingSortedTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); - - have_all_inputs = true; -} - -IProcessor::Status MergingSortedTransform::prepare() -{ - if (!have_all_inputs) - return Status::NeedData; - - auto & output = outputs.front(); - - /// Special case for no inputs. - if (inputs.empty()) - { - output.finish(); - return Status::Finished; - } - - /// Check can output. - - if (output.isFinished()) - { - for (auto & in : inputs) - in.close(); - - return Status::Finished; - } - - /// Do not disable inputs, so it will work in the same way as with AsynchronousBlockInputStream, like before. - bool is_port_full = !output.canPush(); - - /// Special case for single input. - if (inputs.size() == 1) - { - auto & input = inputs.front(); - if (input.isFinished()) - { - output.finish(); - return Status::Finished; - } - - input.setNeeded(); - - if (input.hasData()) - { - if (!is_port_full) - output.push(input.pull()); - - return Status::PortFull; - } - - return Status::NeedData; - } - - /// Push if has data. - if (merged_data.mergedRows() && !is_port_full) - output.push(merged_data.pull()); - - if (!is_initialized) - { - /// Check for inputs we need. - bool all_inputs_has_data = true; - auto it = inputs.begin(); - for (size_t i = 0; it != inputs.end(); ++i, ++it) - { - auto & input = *it; - if (input.isFinished()) - continue; - - if (!cursors[i].empty()) - { - // input.setNotNeeded(); - continue; - } - - input.setNeeded(); - - if (!input.hasData()) - { - all_inputs_has_data = false; - continue; - } - - auto chunk = input.pull(); - if (!chunk.hasRows()) - { - - if (!input.isFinished()) - all_inputs_has_data = false; - - continue; - } - - updateCursor(std::move(chunk), i); - } - - if (!all_inputs_has_data) - return Status::NeedData; - - if (has_collation) - queue_with_collation = SortingHeap(cursors); - else - queue_without_collation = SortingHeap(cursors); - - is_initialized = true; - return Status::Ready; - } - else - { - if (is_finished) - { - - if (is_port_full) - return Status::PortFull; - - for (auto & input : inputs) - input.close(); - - outputs.front().finish(); - - return Status::Finished; - } - - if (need_data) - { - auto & input = *std::next(inputs.begin(), next_input_to_read); - if (!input.isFinished()) - { - input.setNeeded(); - - if (!input.hasData()) - return Status::NeedData; - - auto chunk = input.pull(); - if (!chunk.hasRows() && !input.isFinished()) - return Status::NeedData; - - updateCursor(std::move(chunk), next_input_to_read); - - if (has_collation) - queue_with_collation.push(cursors[next_input_to_read]); - else - queue_without_collation.push(cursors[next_input_to_read]); - } - - need_data = false; - } - - if (is_port_full) - return Status::PortFull; - - return Status::Ready; - } -} - -void MergingSortedTransform::work() -{ - if (has_collation) - merge(queue_with_collation); - else - merge(queue_without_collation); -} - -template -void MergingSortedTransform::merge(TSortingHeap & queue) -{ - /// Returns MergeStatus which we should return if we are going to finish now. - auto can_read_another_row = [&, this]() - { - if (limit && merged_data.totalMergedRows() >= limit) - { - //std::cerr << "Limit reached\n"; - is_finished = true; - return false; - } - - return merged_data.mergedRows() < max_block_size; - }; - - /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` - while (queue.isValid()) - { - /// Shouldn't happen at first iteration, but check just in case. - if (!can_read_another_row()) - return; - - auto current = queue.current(); - - /** And what if the block is totally less or equal than the rest for the current cursor? - * Or is there only one data source left in the queue? Then you can take the entire block on current cursor. - */ - if (current.impl->isFirst() - && (queue.size() == 1 - || (queue.size() >= 2 && current.totallyLessOrEquals(queue.nextChild())))) - { - //std::cerr << "current block is totally less or equals\n"; - - /// If there are already data in the current block, we first return it. We'll get here again the next time we call the merge function. - if (merged_data.mergedRows() != 0) - { - //std::cerr << "merged rows is non-zero\n"; - return; - } - - /// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl) - size_t source_num = current.impl->order; - insertFromChunk(source_num); - queue.removeTop(); - return; - } - - //std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n"; - //std::cerr << "Inserting row\n"; - merged_data.insertRow(current->all_columns, current->pos); - - if (out_row_sources_buf) - { - /// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl) - RowSourcePart row_source(current.impl->order); - out_row_sources_buf->write(row_source.data); - } - - if (!current->isLast()) - { -// std::cerr << "moving to next row\n"; - queue.next(); - } - else - { - /// We will get the next block from the corresponding source, if there is one. - queue.removeTop(); - -// std::cerr << "It was last row, fetching next block\n"; - need_data = true; - next_input_to_read = current.impl->order; - - if (limit && merged_data.totalMergedRows() >= limit) - is_finished = true; - - return; - } - } - is_finished = true; -} - -void MergingSortedTransform::insertFromChunk(size_t source_num) -{ - if (source_num >= cursors.size()) - throw Exception("Logical error in MergingSortedTrandform", ErrorCodes::LOGICAL_ERROR); - - //std::cerr << "copied columns\n"; - - auto num_rows = source_chunks[source_num].getNumRows(); - - UInt64 total_merged_rows_after_insertion = merged_data.mergedRows() + num_rows; - if (limit && total_merged_rows_after_insertion > limit) - { - num_rows = total_merged_rows_after_insertion - limit; - merged_data.insertFromChunk(std::move(source_chunks[source_num]), num_rows); - is_finished = true; - } - else - { - merged_data.insertFromChunk(std::move(source_chunks[source_num]), 0); - need_data = true; - next_input_to_read = source_num; - } - source_chunks[source_num] = Chunk(); - - if (out_row_sources_buf) - { - RowSourcePart row_source(source_num); - for (size_t i = 0; i < num_rows; ++i) - out_row_sources_buf->write(row_source.data); - } -} - - -} diff --git a/src/Processors/Transforms/MergingSortedTransform.h b/src/Processors/Transforms/MergingSortedTransform.h deleted file mode 100644 index 914ad543c75..00000000000 --- a/src/Processors/Transforms/MergingSortedTransform.h +++ /dev/null @@ -1,160 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class MergingSortedTransform : public IProcessor -{ -public: - MergingSortedTransform( - const Block & header, - size_t num_inputs, - const SortDescription & description_, - size_t max_block_size, - UInt64 limit = 0, - bool quiet = false, - bool have_all_inputs = true); - - String getName() const override { return "MergingSortedTransform"; } - Status prepare() override; - void work() override; - - void addInput(); - void setHaveAllInputs(); - -protected: - - class MergedData - { - public: - explicit MergedData(const Block & header) - { - columns.reserve(header.columns()); - for (const auto & column : header) - columns.emplace_back(column.type->createColumn()); - } - - void insertRow(const ColumnRawPtrs & raw_columns, size_t row) - { - size_t num_columns = raw_columns.size(); - for (size_t i = 0; i < num_columns; ++i) - columns[i]->insertFrom(*raw_columns[i], row); - - ++total_merged_rows; - ++merged_rows; - } - - void insertFromChunk(Chunk && chunk, size_t limit_rows) - { - if (merged_rows) - throw Exception("Cannot insert to MergedData from Chunk because MergedData is not empty.", - ErrorCodes::LOGICAL_ERROR); - - auto num_rows = chunk.getNumRows(); - columns = chunk.mutateColumns(); - if (limit_rows && num_rows > limit_rows) - { - num_rows = limit_rows; - for (auto & column : columns) - column = (*column->cut(0, num_rows)).mutate(); - } - - total_merged_rows += num_rows; - merged_rows = num_rows; - } - - Chunk pull() - { - MutableColumns empty_columns; - empty_columns.reserve(columns.size()); - - for (const auto & column : columns) - empty_columns.emplace_back(column->cloneEmpty()); - - empty_columns.swap(columns); - Chunk chunk(std::move(empty_columns), merged_rows); - merged_rows = 0; - - return chunk; - } - - UInt64 totalMergedRows() const { return total_merged_rows; } - UInt64 mergedRows() const { return merged_rows; } - - private: - UInt64 total_merged_rows = 0; - UInt64 merged_rows = 0; - MutableColumns columns; - }; - - /// Settings - SortDescription description; - const size_t max_block_size; - UInt64 limit; - bool has_collation = false; - bool quiet = false; - - std::atomic have_all_inputs; - - MergedData merged_data; - - /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) - /// If it is not nullptr then it should be populated during execution - WriteBuffer * out_row_sources_buf = nullptr; - - /// Chunks currently being merged. - std::vector source_chunks; - - SortCursorImpls cursors; - - SortingHeap queue_without_collation; - SortingHeap queue_with_collation; - -private: - - /// Processor state. - bool is_initialized = false; - bool is_finished = false; - bool need_data = false; - size_t next_input_to_read = 0; - - template - void merge(TSortingHeap & queue); - - void insertFromChunk(size_t source_num); - - void updateCursor(Chunk chunk, size_t source_num) - { - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); - - chunk.setColumns(std::move(columns), num_rows); - - auto & source_chunk = source_chunks[source_num]; - - if (source_chunk.empty()) - { - source_chunk = std::move(chunk); - cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num); - has_collation |= cursors[source_num].has_collation; - } - else - { - source_chunk = std::move(chunk); - cursors[source_num].reset(source_chunk.getColumns(), {}); - } - } -}; - -} diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 370286117ae..4a7a411f45a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -58,7 +58,7 @@ namespace std #include #include #include -#include +#include #include #include #include From 8726ad9e3b51b55b36bf072df93215b40a5a7e3e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Apr 2020 16:50:51 +0300 Subject: [PATCH 146/743] Try fix collapsing. --- src/Processors/Merges/CollapsingSortedTransform.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Merges/CollapsingSortedTransform.cpp b/src/Processors/Merges/CollapsingSortedTransform.cpp index 009aed0983f..82787ec5e18 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.cpp +++ b/src/Processors/Merges/CollapsingSortedTransform.cpp @@ -219,6 +219,8 @@ void CollapsingSortedTransform::merge() throw Exception("Incorrect data: Sign = " + toString(sign) + " (must be 1 or -1).", ErrorCodes::INCORRECT_DATA); + ++current_pos; + if (!current->isLast()) { queue.next(); From fe442d8c9ae42caf15fe0f201b5f682823d1ecf4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Apr 2020 17:21:49 +0300 Subject: [PATCH 147/743] Remove special case for simple input in IMergingTransform. --- src/Processors/Merges/IMergingTransform.cpp | 29 --------------------- src/Processors/Merges/IMergingTransform.h | 1 - 2 files changed, 30 deletions(-) diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 2c5a7affd40..23985e286ff 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -60,31 +60,6 @@ void IMergingTransform::prepareOutputChunk(MergedData & merged_data) output_chunk = merged_data.pull(); } -IProcessor::Status IMergingTransform::prepareSingleInput() -{ - auto & input = inputs.front(); - auto & output = outputs.front(); - - if (input.isFinished()) - { - output.finish(); - onFinish(); - return Status::Finished; - } - - input.setNeeded(); - - if (input.hasData()) - { - if (output.canPush()) - output.push(input.pull()); - - return Status::PortFull; - } - - return Status::NeedData; -} - IProcessor::Status IMergingTransform::prepareInitializeInputs() { /// Add information about inputs. @@ -168,10 +143,6 @@ IProcessor::Status IMergingTransform::prepare() return Status::Finished; } - /// Special case for single input. - if (inputs.size() == 1) - return prepareSingleInput(); - /// Do not disable inputs, so it will work in the same way as with AsynchronousBlockInputStream, like before. bool is_port_full = !output.canPush(); diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index a1046d207ad..72a9d043046 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -62,7 +62,6 @@ private: std::vector input_states; - Status prepareSingleInput(); Status prepareInitializeInputs(); }; From 71fab516f252acd72285bf040db2d2d087d57edb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Apr 2020 18:03:38 +0300 Subject: [PATCH 148/743] Fix AggregatingSorted for simple aggregate functions. --- .../Merges/AggregatingSortedTransform.cpp | 28 ++++++++++++++--- .../Merges/AggregatingSortedTransform.h | 4 +-- .../Merges/CollapsingSortedTransform.h | 5 +-- .../Merges/GraphiteRollupSortedTransform.h | 5 +-- .../Merges/ReplacingSortedTransform.h | 5 +-- src/Processors/Merges/RowRef.h | 31 ++++++++++++++++++- .../Merges/VersionedCollapsingTransform.h | 5 +-- 7 files changed, 67 insertions(+), 16 deletions(-) diff --git a/src/Processors/Merges/AggregatingSortedTransform.cpp b/src/Processors/Merges/AggregatingSortedTransform.cpp index 68f60aed8a2..fe86d01b390 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/src/Processors/Merges/AggregatingSortedTransform.cpp @@ -68,6 +68,24 @@ namespace return def; } + + MutableColumns getMergedColumns(const Block & header, const AggregatingSortedTransform::ColumnsDefinition & def) + { + MutableColumns columns; + columns.resize(header.columns()); + + for (auto & desc : def.columns_to_simple_aggregate) + { + auto & type = header.getByPosition(desc.column_number).type; + columns[desc.column_number] = recursiveRemoveLowCardinality(type)->createColumn(); + } + + for (size_t i = 0; i < columns.size(); ++i) + if (!columns[i]) + columns[i] = header.getByPosition(i).type->createColumn(); + + return columns; + } } AggregatingSortedTransform::AggregatingSortedTransform( @@ -75,7 +93,7 @@ AggregatingSortedTransform::AggregatingSortedTransform( SortDescription description_, size_t max_block_size) : IMergingTransform(num_inputs, header, header, true) , columns_definition(defineColumns(header, description_)) - , merged_data(header.cloneEmptyColumns(), false, max_block_size) + , merged_data(getMergedColumns(header, columns_definition), false, max_block_size) , description(std::move(description_)) , source_chunks(num_inputs) , cursors(num_inputs) @@ -106,7 +124,7 @@ void AggregatingSortedTransform::updateCursor(Chunk chunk, size_t source_num) column = column->convertToFullColumnIfConst(); for (auto & desc : columns_definition.columns_to_simple_aggregate) - if (desc.type_to_convert) + if (desc.inner_type) columns[desc.column_number] = recursiveRemoveLowCardinality(columns[desc.column_number]); chunk.setColumns(std::move(columns), num_rows); @@ -145,10 +163,10 @@ void AggregatingSortedTransform::work() for (auto & desc : columns_definition.columns_to_simple_aggregate) { - if (desc.type_to_convert) + if (desc.inner_type) { - auto & from_type = header.getByPosition(desc.column_number).type; - auto & to_type = desc.type_to_convert; + auto & from_type = desc.inner_type; + auto & to_type = header.getByPosition(desc.column_number).type; columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type); } } diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index bb950a68257..247d92d99e9 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -129,13 +129,13 @@ public: size_t column_number = 0; IColumn * column = nullptr; - const DataTypePtr type_to_convert; + const DataTypePtr inner_type; AlignedBuffer state; bool created = false; SimpleAggregateDescription(AggregateFunctionPtr function_, const size_t column_number_, DataTypePtr type) - : function(std::move(function_)), column_number(column_number_), type_to_convert(std::move(type)) + : function(std::move(function_)), column_number(column_number_), inner_type(std::move(type)) { add_function = function->getAddressOfAddFunction(); state.reset(function->sizeOfData(), function->alignOfData()); diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index 46e3fb2e693..7e64d3253fc 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -64,14 +64,15 @@ private: SortingHeap queue; bool is_queue_initialized = false; + /// Allocator must be destroyed after all RowRefs. + detail::SharedChunkAllocator chunk_allocator; + using RowRef = detail::RowRefWithOwnedChunk; static constexpr size_t max_row_refs = 4; /// first_negative, last_positive, last, current. RowRef first_negative_row; RowRef last_positive_row; RowRef last_row; - detail::SharedChunkAllocator chunk_allocator; - size_t count_positive = 0; /// The number of positive rows for the current primary key. size_t count_negative = 0; /// The number of negative rows for the current primary key. bool last_is_positive = false; /// true if the last row for the current primary key is positive. diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index 4dd394198ad..ce9fb6e019c 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -237,12 +237,13 @@ private: /// Path name of current bucket StringRef current_group_path; + /// Allocator must be destroyed after all RowRefs. + detail::SharedChunkAllocator chunk_allocator; + static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row. /// Last row with maximum version for current primary key (time bucket). RowRef current_subgroup_newest_row; - detail::SharedChunkAllocator chunk_allocator; - /// Time of last read row time_t current_time = 0; time_t current_time_rounded = 0; diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index ff447ee8da6..ffbfe92be96 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -50,6 +50,9 @@ private: SortingHeap queue; bool is_queue_initialized = false; + /// Allocator must be destroyed after all RowRefs. + detail::SharedChunkAllocator chunk_allocator; + using RowRef = detail::RowRefWithOwnedChunk; static constexpr size_t max_row_refs = 3; /// last, current, selected. RowRef last_row; @@ -57,8 +60,6 @@ private: RowRef selected_row; /// Last row with maximum version for current primary key. size_t max_pos = 0; /// The position (into current_row_sources) of the row with the highest version. - detail::SharedChunkAllocator chunk_allocator; - /// Sources of rows with the current primary key. PODArray current_row_sources; diff --git a/src/Processors/Merges/RowRef.h b/src/Processors/Merges/RowRef.h index 48ce92dbdbd..5aeae952067 100644 --- a/src/Processors/Merges/RowRef.h +++ b/src/Processors/Merges/RowRef.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include @@ -51,6 +53,9 @@ class SharedChunkAllocator public: explicit SharedChunkAllocator(size_t max_chunks) { + if (max_chunks == 0) + max_chunks = 1; + chunks.resize(max_chunks); free_chunks.reserve(max_chunks); @@ -74,12 +79,36 @@ public: return SharedChunkPtr(&chunks[pos]); } + ~SharedChunkAllocator() + { + if (free_chunks.size() != chunks.size()) + { + LOG_ERROR(&Logger::get("SharedChunkAllocator"), + "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: " + << StackTrace().toString()); + + return; + } + } + private: std::vector chunks; std::vector free_chunks; - void release(SharedChunk * ptr) + void release(SharedChunk * ptr) noexcept { + if (chunks.empty()) + { + /// This may happen if allocator was removed before chunks. + /// Log message and exit, because we don't want to throw exception in destructor. + + LOG_ERROR(&Logger::get("SharedChunkAllocator"), + "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: " + << StackTrace().toString()); + + return; + } + /// Release memory. It is not obligatory. ptr->clear(); ptr->all_columns.clear(); diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index 04a18142072..214fb3b2f0d 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -53,14 +53,15 @@ private: SortingHeap queue; bool is_queue_initialized = false; + /// Allocator must be destroyed after all RowRefs. + detail::SharedChunkAllocator chunk_allocator; + using RowRef = detail::RowRefWithOwnedChunk; const size_t max_rows_in_queue; /// Rows with the same primary key and sign. FixedSizeDequeWithGaps current_keys; Int8 sign_in_queue = 0; - detail::SharedChunkAllocator chunk_allocator; - std::queue current_row_sources; /// Sources of rows with the current primary key void insertGap(size_t gap_size); From f96a5fe923f6eb7beb5d2e07909cb80758ccba15 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Apr 2020 19:57:49 +0300 Subject: [PATCH 149/743] Fix build, --- src/Processors/Merges/CollapsingSortedTransform.cpp | 2 +- src/Processors/Merges/CollapsingSortedTransform.h | 6 +++--- .../Merges/GraphiteRollupSortedTransform.cpp | 2 +- src/Processors/Merges/GraphiteRollupSortedTransform.h | 6 +++--- src/Processors/Merges/ReplacingSortedTransform.cpp | 2 +- src/Processors/Merges/ReplacingSortedTransform.h | 6 +++--- .../Merges/VersionedCollapsingTransform.cpp | 4 ++-- src/Processors/Merges/VersionedCollapsingTransform.h | 11 ++++++----- 8 files changed, 20 insertions(+), 19 deletions(-) diff --git a/src/Processors/Merges/CollapsingSortedTransform.cpp b/src/Processors/Merges/CollapsingSortedTransform.cpp index 82787ec5e18..d1099184952 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.cpp +++ b/src/Processors/Merges/CollapsingSortedTransform.cpp @@ -29,9 +29,9 @@ CollapsingSortedTransform::CollapsingSortedTransform( , description(std::move(description_)) , sign_column_number(header.getPositionByName(sign_column)) , out_row_sources_buf(out_row_sources_buf_) + , chunk_allocator(num_inputs + max_row_refs) , source_chunks(num_inputs) , cursors(num_inputs) - , chunk_allocator(num_inputs + max_row_refs) { } diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index 7e64d3253fc..e35df8ac6d7 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -56,6 +56,9 @@ private: /// If it is not nullptr then it should be populated during execution WriteBuffer * out_row_sources_buf = nullptr; + /// Allocator must be destroyed after all RowRefs. + detail::SharedChunkAllocator chunk_allocator; + /// Chunks currently being merged. using SourceChunks = std::vector; SourceChunks source_chunks; @@ -64,9 +67,6 @@ private: SortingHeap queue; bool is_queue_initialized = false; - /// Allocator must be destroyed after all RowRefs. - detail::SharedChunkAllocator chunk_allocator; - using RowRef = detail::RowRefWithOwnedChunk; static constexpr size_t max_row_refs = 4; /// first_negative, last_positive, last, current. RowRef first_negative_row; diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.cpp b/src/Processors/Merges/GraphiteRollupSortedTransform.cpp index 5ada52790a3..564b697139b 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.cpp +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.cpp @@ -33,10 +33,10 @@ GraphiteRollupSortedTransform::GraphiteRollupSortedTransform( : IMergingTransform(num_inputs, header, header, true) , merged_data(header.cloneEmptyColumns(), false, max_block_size) , description(std::move(description_)) + , chunk_allocator(num_inputs + max_row_refs) , source_chunks(num_inputs) , cursors(num_inputs) , params(std::move(params_)), time_of_merge(time_of_merge_) - , chunk_allocator(num_inputs + max_row_refs) { size_t max_size_of_aggregate_state = 0; size_t max_alignment_of_aggregate_state = 1; diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index ce9fb6e019c..70112dd853a 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -204,6 +204,9 @@ private: GraphiteRollupMergedData merged_data; SortDescription description; + /// Allocator must be destroyed after all RowRefs. + detail::SharedChunkAllocator chunk_allocator; + /// Chunks currently being merged. using SourceChunks = std::vector; SourceChunks source_chunks; @@ -237,9 +240,6 @@ private: /// Path name of current bucket StringRef current_group_path; - /// Allocator must be destroyed after all RowRefs. - detail::SharedChunkAllocator chunk_allocator; - static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row. /// Last row with maximum version for current primary key (time bucket). RowRef current_subgroup_newest_row; diff --git a/src/Processors/Merges/ReplacingSortedTransform.cpp b/src/Processors/Merges/ReplacingSortedTransform.cpp index d45e6f04aa6..1b29b4842ed 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.cpp +++ b/src/Processors/Merges/ReplacingSortedTransform.cpp @@ -19,9 +19,9 @@ ReplacingSortedTransform::ReplacingSortedTransform( , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) + , chunk_allocator(num_inputs + max_row_refs) , source_chunks(num_inputs) , cursors(num_inputs) - , chunk_allocator(num_inputs + max_row_refs) { if (!version_column.empty()) version_column_number = header.getPositionByName(version_column); diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index ffbfe92be96..0b4c2b2feac 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -42,6 +42,9 @@ private: /// If it is not nullptr then it should be populated during execution WriteBuffer * out_row_sources_buf = nullptr; + /// Allocator must be destroyed after all RowRefs. + detail::SharedChunkAllocator chunk_allocator; + /// Chunks currently being merged. using SourceChunks = std::vector; SourceChunks source_chunks; @@ -50,9 +53,6 @@ private: SortingHeap queue; bool is_queue_initialized = false; - /// Allocator must be destroyed after all RowRefs. - detail::SharedChunkAllocator chunk_allocator; - using RowRef = detail::RowRefWithOwnedChunk; static constexpr size_t max_row_refs = 3; /// last, current, selected. RowRef last_row; diff --git a/src/Processors/Merges/VersionedCollapsingTransform.cpp b/src/Processors/Merges/VersionedCollapsingTransform.cpp index c11cf46a6ca..cb0e085c8d1 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.cpp +++ b/src/Processors/Merges/VersionedCollapsingTransform.cpp @@ -22,11 +22,11 @@ VersionedCollapsingTransform::VersionedCollapsingTransform( , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , description(std::move(description_)) , out_row_sources_buf(out_row_sources_buf_) + , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 1) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer + , chunk_allocator(num_inputs + max_rows_in_queue + 1) /// +1 just in case (for current_row) , source_chunks(num_inputs) , cursors(num_inputs) - , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 1) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer , current_keys(max_rows_in_queue) - , chunk_allocator(num_inputs + max_rows_in_queue + 1) /// +1 just in case (for current_row) { sign_column_number = header.getPositionByName(sign_column_); } diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index 214fb3b2f0d..85993bde134 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -45,6 +45,12 @@ private: /// If it is not nullptr then it should be populated during execution WriteBuffer * out_row_sources_buf = nullptr; + using RowRef = detail::RowRefWithOwnedChunk; + const size_t max_rows_in_queue; + + /// Allocator must be destroyed after all RowRefs. + detail::SharedChunkAllocator chunk_allocator; + /// Chunks currently being merged. using SourceChunks = std::vector; SourceChunks source_chunks; @@ -53,11 +59,6 @@ private: SortingHeap queue; bool is_queue_initialized = false; - /// Allocator must be destroyed after all RowRefs. - detail::SharedChunkAllocator chunk_allocator; - - using RowRef = detail::RowRefWithOwnedChunk; - const size_t max_rows_in_queue; /// Rows with the same primary key and sign. FixedSizeDequeWithGaps current_keys; Int8 sign_in_queue = 0; From cbd1f6a524c0332f679da3551cdfce19e623a5fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Apr 2020 22:27:50 +0300 Subject: [PATCH 150/743] Added comments. --- src/Processors/Merges/AggregatingSortedTransform.h | 6 ++++++ src/Processors/Merges/CollapsingSortedTransform.h | 2 +- src/Processors/Merges/ReplacingSortedTransform.h | 4 ++++ src/Processors/Merges/SummingSortedTransform.h | 5 +++++ src/Processors/Merges/VersionedCollapsingTransform.h | 5 +++++ 5 files changed, 21 insertions(+), 1 deletion(-) diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index 247d92d99e9..da26527d435 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -15,6 +15,12 @@ namespace DB class ColumnAggregateFunction; +/** Merges several sorted ports to one. + * During this for each group of consecutive identical values of the primary key (the columns by which the data is sorted), + * merges them into one row. When merging, the data is pre-aggregated - merge of states of aggregate functions, + * corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type, + * when merged, the first value is selected. + */ class AggregatingSortedTransform : public IMergingTransform { public: diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index e35df8ac6d7..3a3c1540c06 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -12,7 +12,7 @@ namespace DB { -/** Merges several sorted inputs to one. +/** Merges several sorted ports to one. * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), * keeps no more than one row with the value of the column `sign_column = -1` ("negative row") * and no more than a row with the value of the column `sign_column = 1` ("positive row"). diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index 0b4c2b2feac..194e81ab6db 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -13,6 +13,10 @@ namespace DB { +/** Merges several sorted ports into one. + * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), + * keeps row with max `version` value. + */ class ReplacingSortedTransform final : public IMergingTransform { public: diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index 2e07ae12115..cc651197647 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -14,6 +14,11 @@ namespace DB { +/** Merges several sorted ports into one. + * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), + * collapses them into one row, summing all the numeric columns except the primary key. + * If in all numeric columns, except for the primary key, the result is zero, it deletes the row. + */ class SummingSortedTransform final : public IMergingTransform { public: diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index 85993bde134..875377a544b 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -15,6 +15,11 @@ namespace DB { +/** Merges several sorted ports to one. + * For each group of consecutive identical values of the sorting key + * (the columns by which the data is sorted, including specially specified version column), + * merges any pair of consecutive rows with opposite sign. + */ class VersionedCollapsingTransform final : public IMergingTransform { public: From 2ce753a3165b1f5136bfdb5e5b517b793f59b2df Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Apr 2020 16:54:15 +0300 Subject: [PATCH 151/743] Fix tests. --- src/Processors/Merges/IMergingTransform.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 23985e286ff..2f815d69d9c 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -55,6 +55,9 @@ void IMergingTransform::requestDataForInput(size_t input_number) void IMergingTransform::prepareOutputChunk(MergedData & merged_data) { + if (need_data) + return; + has_output_chunk = (is_finished && merged_data.mergedRows()) || merged_data.hasEnoughRows(); if (has_output_chunk) output_chunk = merged_data.pull(); From fc605d411d0eb534d6d8c3883d423d33a534172f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Apr 2020 20:08:25 +0300 Subject: [PATCH 152/743] Added IMergingAlgorithm. --- src/Processors/Merges/IMergingAlgorithm.h | 33 +++ src/Processors/Merges/IMergingTransform.cpp | 198 +++++++++++++++++ src/Processors/Merges/IMergingTransform.h | 56 +++++ .../Merges/MergingSortedAlgorithm.cpp | 198 +++++++++++++++++ .../Merges/MergingSortedAlgorithm.h | 59 +++++ .../Merges/MergingSortedTransform.cpp | 201 +----------------- .../Merges/MergingSortedTransform.h | 34 +-- 7 files changed, 553 insertions(+), 226 deletions(-) create mode 100644 src/Processors/Merges/IMergingAlgorithm.h create mode 100644 src/Processors/Merges/MergingSortedAlgorithm.cpp create mode 100644 src/Processors/Merges/MergingSortedAlgorithm.h diff --git a/src/Processors/Merges/IMergingAlgorithm.h b/src/Processors/Merges/IMergingAlgorithm.h new file mode 100644 index 00000000000..1dbe88e1370 --- /dev/null +++ b/src/Processors/Merges/IMergingAlgorithm.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class IMergingAlgorithm +{ +public: + struct Status + { + Chunk chunk; + bool is_finished = false; + ssize_t required_source = -1; + + explicit Status(Chunk chunk_) : chunk(std::move(chunk_)) {} + explicit Status(Chunk chunk_, bool is_finished_) : chunk(std::move(chunk_)), is_finished(is_finished_) {} + explicit Status(size_t source) : required_source(source) {} + }; + + virtual void initialize(Chunks chunks) = 0; + virtual void consume(Chunk chunk, size_t source_num) = 0; + virtual Status merge() = 0; + + virtual ~IMergingAlgorithm() = default; +}; + +template +concept MergingAlgorithm = std::is_base_of::value && std::is_move_constructible::value; + +} diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 2f815d69d9c..295d7361d19 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -200,4 +200,202 @@ IProcessor::Status IMergingTransform::prepare() return Status::Ready; } + +template +IMergingTransform2::IMergingTransform2( + Algorithm algorithm, + size_t num_inputs, + const Block & input_header, + const Block & output_header, + bool have_all_inputs_) + : IProcessor(InputPorts(num_inputs, input_header), {output_header}) + , algorithm(std::move(algorithm)) + , have_all_inputs(have_all_inputs_) +{ +} + +template +void IMergingTransform2::onNewInput() +{ + throw Exception("onNewInput is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); +} + +template +void IMergingTransform2::addInput() +{ + if (have_all_inputs) + throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); + + inputs.emplace_back(outputs.front().getHeader(), this); + onNewInput(); +} + +template +void IMergingTransform2::setHaveAllInputs() +{ + if (have_all_inputs) + throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); + + have_all_inputs = true; +} + +template +IProcessor::Status IMergingTransform2::prepareInitializeInputs() +{ + /// Add information about inputs. + if (input_states.empty()) + { + input_states.reserve(inputs.size()); + for (auto & input : inputs) + input_states.emplace_back(input); + + init_chunks.resize(inputs.size()); + } + + /// Check for inputs we need. + bool all_inputs_has_data = true; + auto it = inputs.begin(); + for (size_t i = 0; it != inputs.end(); ++i, ++it) + { + auto & input = *it; + if (input.isFinished()) + continue; + + if (input_states[i].is_initialized) + { + // input.setNotNeeded(); + continue; + } + + input.setNeeded(); + + if (!input.hasData()) + { + all_inputs_has_data = false; + continue; + } + + auto chunk = input.pull(); + if (!chunk.hasRows()) + { + + if (!input.isFinished()) + all_inputs_has_data = false; + + continue; + } + + init_chunks[i] = std::move(chunk); + input_states[i].is_initialized = true; + } + + if (!all_inputs_has_data) + return Status::NeedData; + + is_initialized = true; + return Status::Ready; +} + +template +IProcessor::Status IMergingTransform2::prepare() +{ + if (!have_all_inputs) + return Status::NeedData; + + auto & output = outputs.front(); + + /// Special case for no inputs. + if (inputs.empty()) + { + output.finish(); + onFinish(); + return Status::Finished; + } + + /// Check can output. + + if (output.isFinished()) + { + for (auto & in : inputs) + in.close(); + + onFinish(); + return Status::Finished; + } + + /// Do not disable inputs, so it will work in the same way as with AsynchronousBlockInputStream, like before. + bool is_port_full = !output.canPush(); + + /// Push if has data. + if (has_output_chunk && !is_port_full) + { + output.push(std::move(output_chunk)); + has_output_chunk = false; + } + + if (!is_initialized) + return prepareInitializeInputs(); + + if (is_finished) + { + + if (is_port_full) + return Status::PortFull; + + for (auto & input : inputs) + input.close(); + + outputs.front().finish(); + + onFinish(); + return Status::Finished; + } + + if (need_data) + { + auto & input = input_states[next_input_to_read].port; + if (!input.isFinished()) + { + input.setNeeded(); + + if (!input.hasData()) + return Status::NeedData; + + auto chunk = input.pull(); + if (!chunk.hasRows() && !input.isFinished()) + return Status::NeedData; + + algorithm.consume(std::move(chunk), next_input_to_read); + } + + need_data = false; + } + + if (is_port_full) + return Status::PortFull; + + return Status::Ready; +} + +template +void IMergingTransform2::work() +{ + if (!init_chunks.empty()) + algorithm.initialize(std::move(init_chunks)); + + IMergingAlgorithm::Status status = algorithm.merge(); + + if (status.chunk && status.chunk.hasRows()) + { + has_output_chunk = true; + output_chunk = std::move(status.chunk); + } + + if (status.required_source >= 0) + next_input_to_read = status.required_source; + + if (status.is_finished) + is_finished = true; +} + } diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 72a9d043046..9122e28a913 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -65,4 +66,59 @@ private: Status prepareInitializeInputs(); }; +/// Base class for merging transforms. +template +class IMergingTransform2 : public IProcessor +{ +public: + IMergingTransform2( + Algorithm algorithm, + size_t num_inputs, + const Block & input_header, + const Block & output_header, + //size_t max_block_size, + //bool use_average_block_size, /// For adaptive granularity. Return chunks with the same avg size as inputs. + bool have_all_inputs_); + + /// Methods to add additional input port. It is possible to do only before the first call of `prepare`. + void addInput(); + /// Need to be called after all inputs are added. (only if have_all_inputs was not specified). + void setHaveAllInputs(); + + Status prepare() override; + void work() override; + +protected: + virtual void onNewInput(); /// Is called when new input is added. Only if have_all_inputs = false. + virtual void onFinish() {} /// Is called when all data is processed. + + /// Profile info. + Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + Algorithm algorithm; + +private: + /// Processor state. + Chunk output_chunk; + bool has_output_chunk = false; + bool is_finished = false; + bool is_initialized = false; + bool need_data = false; + size_t next_input_to_read = 0; + + std::atomic have_all_inputs; + + struct InputState + { + explicit InputState(InputPort & port_) : port(port_) {} + + InputPort & port; + bool is_initialized = false; + }; + + std::vector input_states; + Chunks init_chunks; + + Status prepareInitializeInputs(); +}; + } diff --git a/src/Processors/Merges/MergingSortedAlgorithm.cpp b/src/Processors/Merges/MergingSortedAlgorithm.cpp new file mode 100644 index 00000000000..556bde169d3 --- /dev/null +++ b/src/Processors/Merges/MergingSortedAlgorithm.cpp @@ -0,0 +1,198 @@ +#include +#include +#include + +namespace DB +{ + +MergingSortedAlgorithm::MergingSortedAlgorithm( + const Block & header, + size_t num_inputs, + SortDescription description_, + size_t max_block_size, + UInt64 limit_, + WriteBuffer * out_row_sources_buf_, + bool use_average_block_sizes) + : merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) + , description(std::move(description_)) + , limit(limit_) + , out_row_sources_buf(out_row_sources_buf_) + , source_chunks(num_inputs) + , cursors(num_inputs) +{ + /// Replace column names in description to positions. + for (auto & column_description : description) + { + has_collation |= column_description.collator != nullptr; + if (!column_description.column_name.empty()) + { + column_description.column_number = header.getPositionByName(column_description.column_name); + column_description.column_name.clear(); + } + } +} + +void MergingSortedAlgorithm::addInput() +{ + source_chunks.emplace_back(); + cursors.emplace_back(); +} + +static void prepareChunk(Chunk & chunk) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); +} + +void MergingSortedAlgorithm::updateCursor(size_t source_num) +{ + auto & source_chunk = source_chunks[source_num]; + cursors[source_num].reset(source_chunk.getColumns(), {}); +} + +void MergingSortedAlgorithm::initialize(Chunks chunks) +{ + source_chunks = std::move(chunks); + + for (size_t source_num = 0; source_num < source_chunks.size(); ++source_num) + { + auto & chunk = source_chunks[source_num]; + + if (!chunk) + continue; + + prepareChunk(chunk); + cursors[source_num] = SortCursorImpl(chunk.getColumns(), description, source_num); + } + + if (has_collation) + queue_with_collation = SortingHeap(cursors); + else + queue_without_collation = SortingHeap(cursors); +} + +void MergingSortedAlgorithm::consume(Chunk chunk, size_t source_num) +{ + prepareChunk(chunk); + source_chunks[source_num] = std::move(chunk); + cursors[source_num].reset(source_chunks[source_num].getColumns(), {}); +} + +IMergingAlgorithm::Status MergingSortedAlgorithm::merge() +{ + if (has_collation) + return mergeImpl(queue_with_collation); + else + return mergeImpl(queue_without_collation); +} + +template +IMergingAlgorithm::Status MergingSortedAlgorithm::mergeImpl(TSortingHeap & queue) +{ + /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` + while (queue.isValid()) + { + if (merged_data.hasEnoughRows()) + return Status(merged_data.pull()); + + auto current = queue.current(); + + /** And what if the block is totally less or equal than the rest for the current cursor? + * Or is there only one data source left in the queue? Then you can take the entire block on current cursor. + */ + if (current.impl->isFirst() + && (queue.size() == 1 + || (queue.size() >= 2 && current.totallyLessOrEquals(queue.nextChild())))) + { + //std::cerr << "current block is totally less or equals\n"; + + /// If there are already data in the current block, we first return it. + /// We'll get here again the next time we call the merge function. + if (merged_data.mergedRows() != 0) + { + //std::cerr << "merged rows is non-zero\n"; + // merged_data.flush(); + return Status(merged_data.pull()); + } + + /// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl) + size_t source_num = current.impl->order; + queue.removeTop(); + return insertFromChunk(source_num); + } + + //std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n"; + //std::cerr << "Inserting row\n"; + merged_data.insertRow(current->all_columns, current->pos, current->rows); + + if (out_row_sources_buf) + { + /// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl) + RowSourcePart row_source(current.impl->order); + out_row_sources_buf->write(row_source.data); + } + + if (limit && merged_data.totalMergedRows() >= limit) + return Status(merged_data.pull(), true); + + if (!current->isLast()) + { + //std::cerr << "moving to next row\n"; + queue.next(); + } + else + { + /// We will get the next block from the corresponding source, if there is one. + queue.removeTop(); + //std::cerr << "It was last row, fetching next block\n"; + return Status(current.impl->order); + } + } + + return Status(merged_data.pull(), true); +} + +IMergingAlgorithm::Status MergingSortedAlgorithm::insertFromChunk(size_t source_num) +{ + if (source_num >= cursors.size()) + throw Exception("Logical error in MergingSortedTransform", ErrorCodes::LOGICAL_ERROR); + + //std::cerr << "copied columns\n"; + + auto num_rows = source_chunks[source_num].getNumRows(); + + UInt64 total_merged_rows_after_insertion = merged_data.mergedRows() + num_rows; + bool is_finished = limit && total_merged_rows_after_insertion > limit; + + if (is_finished) + { + num_rows = total_merged_rows_after_insertion - limit; + merged_data.insertFromChunk(std::move(source_chunks[source_num]), num_rows); + } + else + merged_data.insertFromChunk(std::move(source_chunks[source_num]), 0); + + source_chunks[source_num] = Chunk(); + + /// Write order of rows for other columns + /// this data will be used in gather stream + if (out_row_sources_buf) + { + RowSourcePart row_source(source_num); + for (size_t i = 0; i < num_rows; ++i) + out_row_sources_buf->write(row_source.data); + } + + auto status = Status(merged_data.pull(), is_finished); + + if (!is_finished) + status.required_source = source_num; + + return status; +} + +} diff --git a/src/Processors/Merges/MergingSortedAlgorithm.h b/src/Processors/Merges/MergingSortedAlgorithm.h new file mode 100644 index 00000000000..5c4087ed0bd --- /dev/null +++ b/src/Processors/Merges/MergingSortedAlgorithm.h @@ -0,0 +1,59 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +class MergingSortedAlgorithm final : public IMergingAlgorithm +{ +public: + MergingSortedAlgorithm( + const Block & header, + size_t num_inputs, + SortDescription description_, + size_t max_block_size, + UInt64 limit_, + WriteBuffer * out_row_sources_buf_, + bool use_average_block_sizes); + + MergingSortedAlgorithm(MergingSortedAlgorithm && other) = default; + + void addInput(); + + void initialize(Chunks chunks) override; + void consume(Chunk chunk, size_t source_num) override; + Status merge() override; + + const MergedData & getMergedData() const { return merged_data; } + +private: + MergedData merged_data; + + /// Settings + SortDescription description; + UInt64 limit; + bool has_collation = false; + + /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) + /// If it is not nullptr then it should be populated during execution + WriteBuffer * out_row_sources_buf = nullptr; + + /// Chunks currently being merged. + std::vector source_chunks; + + SortCursorImpls cursors; + + SortingHeap queue_without_collation; + SortingHeap queue_with_collation; + + void updateCursor(size_t source_num); + Status insertFromChunk(size_t source_num); + + template + Status mergeImpl(TSortingHeap & queue); +}; + +} diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 7b7e4fcf62c..2657077c143 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -23,206 +23,17 @@ MergingSortedTransform::MergingSortedTransform( bool quiet_, bool use_average_block_sizes, bool have_all_inputs_) - : IMergingTransform(num_inputs, header, header, have_all_inputs_) - , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) - , description(std::move(description_)) - , limit(limit_) + : IMergingTransform2( + MergingSortedAlgorithm(header, num_inputs, std::move(description_), max_block_size, + limit_, out_row_sources_buf_, use_average_block_sizes), + num_inputs, header, header, have_all_inputs_) , quiet(quiet_) - , out_row_sources_buf(out_row_sources_buf_) - , source_chunks(num_inputs) - , cursors(num_inputs) { - auto & sample = outputs.front().getHeader(); - /// Replace column names in description to positions. - for (auto & column_description : description) - { - has_collation |= column_description.collator != nullptr; - if (!column_description.column_name.empty()) - { - column_description.column_number = sample.getPositionByName(column_description.column_name); - column_description.column_name.clear(); - } - } } void MergingSortedTransform::onNewInput() { - source_chunks.emplace_back(); - cursors.emplace_back(); -} - -void MergingSortedTransform::initializeInputs() -{ - if (has_collation) - queue_with_collation = SortingHeap(cursors); - else - queue_without_collation = SortingHeap(cursors); - - is_queue_initialized = true; -} - -void MergingSortedTransform::consume(Chunk chunk, size_t input_number) -{ - updateCursor(std::move(chunk), input_number); - - if (is_queue_initialized) - { - if (has_collation) - queue_with_collation.push(cursors[input_number]); - else - queue_without_collation.push(cursors[input_number]); - } -} - -void MergingSortedTransform::updateCursor(Chunk chunk, size_t source_num) -{ - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); - - chunk.setColumns(std::move(columns), num_rows); - - auto & source_chunk = source_chunks[source_num]; - - if (source_chunk.empty()) - { - source_chunk = std::move(chunk); - cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num); - has_collation |= cursors[source_num].has_collation; - } - else - { - source_chunk = std::move(chunk); - cursors[source_num].reset(source_chunk.getColumns(), {}); - } -} - -void MergingSortedTransform::work() -{ - if (has_collation) - merge(queue_with_collation); - else - merge(queue_without_collation); - - prepareOutputChunk(merged_data); -} - -template -void MergingSortedTransform::merge(TSortingHeap & queue) -{ - /// Returns MergeStatus which we should return if we are going to finish now. - auto can_read_another_row = [&, this]() - { - if (limit && merged_data.totalMergedRows() >= limit) - { - //std::cerr << "Limit reached\n"; - is_finished = true; - return false; - } - - return !merged_data.hasEnoughRows(); - }; - - /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` - while (queue.isValid()) - { - if (!can_read_another_row()) - return; - - auto current = queue.current(); - - /** And what if the block is totally less or equal than the rest for the current cursor? - * Or is there only one data source left in the queue? Then you can take the entire block on current cursor. - */ - if (current.impl->isFirst() - && (queue.size() == 1 - || (queue.size() >= 2 && current.totallyLessOrEquals(queue.nextChild())))) - { - //std::cerr << "current block is totally less or equals\n"; - - /// If there are already data in the current block, we first return it. - /// We'll get here again the next time we call the merge function. - if (merged_data.mergedRows() != 0) - { - //std::cerr << "merged rows is non-zero\n"; - merged_data.flush(); - return; - } - - /// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl) - size_t source_num = current.impl->order; - insertFromChunk(source_num); - queue.removeTop(); - return; - } - - //std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n"; - //std::cerr << "Inserting row\n"; - merged_data.insertRow(current->all_columns, current->pos, current->rows); - - if (out_row_sources_buf) - { - /// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl) - RowSourcePart row_source(current.impl->order); - out_row_sources_buf->write(row_source.data); - } - - if (!current->isLast()) - { - //std::cerr << "moving to next row\n"; - queue.next(); - } - else - { - /// We will get the next block from the corresponding source, if there is one. - queue.removeTop(); - - //std::cerr << "It was last row, fetching next block\n"; - requestDataForInput(current.impl->order); - - if (limit && merged_data.totalMergedRows() >= limit) - is_finished = true; - - return; - } - } - - is_finished = true; -} - -void MergingSortedTransform::insertFromChunk(size_t source_num) -{ - if (source_num >= cursors.size()) - throw Exception("Logical error in MergingSortedTransform", ErrorCodes::LOGICAL_ERROR); - - //std::cerr << "copied columns\n"; - - auto num_rows = source_chunks[source_num].getNumRows(); - - UInt64 total_merged_rows_after_insertion = merged_data.mergedRows() + num_rows; - if (limit && total_merged_rows_after_insertion > limit) - { - num_rows = total_merged_rows_after_insertion - limit; - merged_data.insertFromChunk(std::move(source_chunks[source_num]), num_rows); - is_finished = true; - } - else - { - merged_data.insertFromChunk(std::move(source_chunks[source_num]), 0); - requestDataForInput(source_num); - } - - source_chunks[source_num] = Chunk(); - - /// Write order of rows for other columns - /// this data will be used in gather stream - if (out_row_sources_buf) - { - RowSourcePart row_source(source_num); - for (size_t i = 0; i < num_rows; ++i) - out_row_sources_buf->write(row_source.data); - } + algorithm.addInput(); } void MergingSortedTransform::onFinish() @@ -230,6 +41,8 @@ void MergingSortedTransform::onFinish() if (quiet) return; + auto & merged_data = algorithm.getMergedData(); + auto * log = &Logger::get("MergingSortedTransform"); double seconds = total_stopwatch.elapsedSeconds(); diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 49bfe228106..8763fbe1aa2 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -1,16 +1,14 @@ #pragma once #include -#include -#include -#include +#include namespace DB { /// Merges several sorted inputs into one sorted output. -class MergingSortedTransform final : public IMergingTransform +class MergingSortedTransform final : public IMergingTransform2 { public: MergingSortedTransform( @@ -25,41 +23,13 @@ public: bool have_all_inputs_ = true); String getName() const override { return "MergingSortedTransform"; } - void work() override; protected: void onNewInput() override; - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; void onFinish() override; private: - MergedData merged_data; - - /// Settings - SortDescription description; - UInt64 limit; - bool has_collation = false; bool quiet = false; - - /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) - /// If it is not nullptr then it should be populated during execution - WriteBuffer * out_row_sources_buf = nullptr; - - /// Chunks currently being merged. - std::vector source_chunks; - - SortCursorImpls cursors; - - SortingHeap queue_without_collation; - SortingHeap queue_with_collation; - bool is_queue_initialized = false; - - template - void merge(TSortingHeap & queue); - - void insertFromChunk(size_t source_num); - void updateCursor(Chunk chunk, size_t source_num); }; } From f7fdfe4ed2b32d975a72d2948bc8d9024816bf9b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Apr 2020 20:25:44 +0300 Subject: [PATCH 153/743] Fix build. --- src/Processors/Merges/IMergingAlgorithm.h | 6 +- src/Processors/Merges/IMergingTransform.cpp | 61 +++---------- src/Processors/Merges/IMergingTransform.h | 86 ++++++++++++++----- .../Merges/MergingSortedAlgorithm.cpp | 9 +- 4 files changed, 90 insertions(+), 72 deletions(-) diff --git a/src/Processors/Merges/IMergingAlgorithm.h b/src/Processors/Merges/IMergingAlgorithm.h index 1dbe88e1370..51ee58cedd2 100644 --- a/src/Processors/Merges/IMergingAlgorithm.h +++ b/src/Processors/Merges/IMergingAlgorithm.h @@ -24,10 +24,12 @@ public: virtual void consume(Chunk chunk, size_t source_num) = 0; virtual Status merge() = 0; + IMergingAlgorithm(IMergingAlgorithm &&) = default; virtual ~IMergingAlgorithm() = default; }; -template -concept MergingAlgorithm = std::is_base_of::value && std::is_move_constructible::value; +// TODO: use when compile with clang which could support it +// template +// concept MergingAlgorithm = std::is_base_of::value && std::is_move_constructible::value; } diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 295d7361d19..11ab063dce7 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -200,28 +200,22 @@ IProcessor::Status IMergingTransform::prepare() return Status::Ready; } - -template -IMergingTransform2::IMergingTransform2( - Algorithm algorithm, - size_t num_inputs, - const Block & input_header, - const Block & output_header, - bool have_all_inputs_) - : IProcessor(InputPorts(num_inputs, input_header), {output_header}) - , algorithm(std::move(algorithm)) - , have_all_inputs(have_all_inputs_) +IMergingTransformBase::IMergingTransformBase( + size_t num_inputs, + const Block & input_header, + const Block & output_header, + bool have_all_inputs_) + : IProcessor(InputPorts(num_inputs, input_header), {output_header}) + , have_all_inputs(have_all_inputs_) { } -template -void IMergingTransform2::onNewInput() +void IMergingTransformBase::onNewInput() { throw Exception("onNewInput is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } -template -void IMergingTransform2::addInput() +void IMergingTransformBase::addInput() { if (have_all_inputs) throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); @@ -230,8 +224,7 @@ void IMergingTransform2::addInput() onNewInput(); } -template -void IMergingTransform2::setHaveAllInputs() +void IMergingTransformBase::setHaveAllInputs() { if (have_all_inputs) throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); @@ -239,8 +232,7 @@ void IMergingTransform2::setHaveAllInputs() have_all_inputs = true; } -template -IProcessor::Status IMergingTransform2::prepareInitializeInputs() +IProcessor::Status IMergingTransformBase::prepareInitializeInputs() { /// Add information about inputs. if (input_states.empty()) @@ -296,8 +288,7 @@ IProcessor::Status IMergingTransform2::prepareInitializeInputs() return Status::Ready; } -template -IProcessor::Status IMergingTransform2::prepare() +IProcessor::Status IMergingTransformBase::prepare() { if (!have_all_inputs) return Status::NeedData; @@ -327,11 +318,8 @@ IProcessor::Status IMergingTransform2::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - if (has_output_chunk && !is_port_full) - { + if (output_chunk && !is_port_full) output.push(std::move(output_chunk)); - has_output_chunk = false; - } if (!is_initialized) return prepareInitializeInputs(); @@ -365,7 +353,7 @@ IProcessor::Status IMergingTransform2::prepare() if (!chunk.hasRows() && !input.isFinished()) return Status::NeedData; - algorithm.consume(std::move(chunk), next_input_to_read); + input_chunk = std::move(chunk); } need_data = false; @@ -377,25 +365,4 @@ IProcessor::Status IMergingTransform2::prepare() return Status::Ready; } -template -void IMergingTransform2::work() -{ - if (!init_chunks.empty()) - algorithm.initialize(std::move(init_chunks)); - - IMergingAlgorithm::Status status = algorithm.merge(); - - if (status.chunk && status.chunk.hasRows()) - { - has_output_chunk = true; - output_chunk = std::move(status.chunk); - } - - if (status.required_source >= 0) - next_input_to_read = status.required_source; - - if (status.is_finished) - is_finished = true; -} - } diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 9122e28a913..3493ea46af1 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -67,18 +67,14 @@ private: }; /// Base class for merging transforms. -template -class IMergingTransform2 : public IProcessor +class IMergingTransformBase : public IProcessor { public: - IMergingTransform2( - Algorithm algorithm, - size_t num_inputs, - const Block & input_header, - const Block & output_header, - //size_t max_block_size, - //bool use_average_block_size, /// For adaptive granularity. Return chunks with the same avg size as inputs. - bool have_all_inputs_); + IMergingTransformBase( + size_t num_inputs, + const Block & input_header, + const Block & output_header, + bool have_all_inputs_); /// Methods to add additional input port. It is possible to do only before the first call of `prepare`. void addInput(); @@ -86,27 +82,21 @@ public: void setHaveAllInputs(); Status prepare() override; - void work() override; protected: virtual void onNewInput(); /// Is called when new input is added. Only if have_all_inputs = false. virtual void onFinish() {} /// Is called when all data is processed. - /// Profile info. - Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; - Algorithm algorithm; - -private: /// Processor state. Chunk output_chunk; - bool has_output_chunk = false; + Chunk input_chunk; bool is_finished = false; - bool is_initialized = false; bool need_data = false; size_t next_input_to_read = 0; - std::atomic have_all_inputs; + Chunks init_chunks; +private: struct InputState { explicit InputState(InputPort & port_) : port(port_) {} @@ -116,9 +106,63 @@ private: }; std::vector input_states; - Chunks init_chunks; + std::atomic have_all_inputs; + bool is_initialized = false; - Status prepareInitializeInputs(); + IProcessor::Status prepareInitializeInputs(); +}; + +template +class IMergingTransform2 : public IMergingTransformBase +{ +public: + IMergingTransform2( + Algorithm algorithm_, + size_t num_inputs, + const Block & input_header, + const Block & output_header, + bool have_all_inputs_) + : IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_) + , algorithm(std::move(algorithm_)) + { + } + + void work() override + { + if (!init_chunks.empty()) + algorithm.initialize(std::move(init_chunks)); + + if (input_chunk) + algorithm.consume(std::move(input_chunk), next_input_to_read); + + IMergingAlgorithm::Status status = algorithm.merge(); + + if (status.chunk && status.chunk.hasRows()) + output_chunk = std::move(status.chunk); + + if (status.required_source >= 0) + { + next_input_to_read = status.required_source; + need_data = true; + } + + if (status.is_finished) + is_finished = true; + } + +protected: + Algorithm algorithm; + + /// Profile info. + Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + +private: + using IMergingTransformBase::output_chunk; + using IMergingTransformBase::input_chunk; + using IMergingTransformBase::is_finished; + using IMergingTransformBase::need_data; + using IMergingTransformBase::next_input_to_read; + using IMergingTransformBase::init_chunks; }; } diff --git a/src/Processors/Merges/MergingSortedAlgorithm.cpp b/src/Processors/Merges/MergingSortedAlgorithm.cpp index 556bde169d3..3be9f3f64ef 100644 --- a/src/Processors/Merges/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/MergingSortedAlgorithm.cpp @@ -80,6 +80,11 @@ void MergingSortedAlgorithm::consume(Chunk chunk, size_t source_num) prepareChunk(chunk); source_chunks[source_num] = std::move(chunk); cursors[source_num].reset(source_chunks[source_num].getColumns(), {}); + + if (has_collation) + queue_with_collation.push(cursors[source_num]); + else + queue_without_collation.push(cursors[source_num]); } IMergingAlgorithm::Status MergingSortedAlgorithm::merge() @@ -166,9 +171,9 @@ IMergingAlgorithm::Status MergingSortedAlgorithm::insertFromChunk(size_t source_ auto num_rows = source_chunks[source_num].getNumRows(); UInt64 total_merged_rows_after_insertion = merged_data.mergedRows() + num_rows; - bool is_finished = limit && total_merged_rows_after_insertion > limit; + bool is_finished = limit && total_merged_rows_after_insertion >= limit; - if (is_finished) + if (limit && total_merged_rows_after_insertion > limit) { num_rows = total_merged_rows_after_insertion - limit; merged_data.insertFromChunk(std::move(source_chunks[source_num]), num_rows); From 868f7e9aea34b01f5d7590aa8f9f313afcf92448 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 12:16:23 +0300 Subject: [PATCH 154/743] Move Graphite params to separate file. --- src/Processors/Merges/Graphite.h | 126 ++++++++++++++++++ .../Merges/GraphiteRollupSortedTransform.h | 122 +---------------- src/Storages/MergeTree/MergeTreeData.h | 2 +- 3 files changed, 128 insertions(+), 122 deletions(-) create mode 100644 src/Processors/Merges/Graphite.h diff --git a/src/Processors/Merges/Graphite.h b/src/Processors/Merges/Graphite.h new file mode 100644 index 00000000000..03269de35d0 --- /dev/null +++ b/src/Processors/Merges/Graphite.h @@ -0,0 +1,126 @@ +#pragma once + +namespace DB +{ +/** Intended for implementation of "rollup" - aggregation (rounding) of older data + * for a table with Graphite data (Graphite is the system for time series monitoring). + * + * Table with graphite data has at least the following columns (accurate to the name): + * Path, Time, Value, Version + * + * Path - name of metric (sensor); + * Time - time of measurement; + * Value - value of measurement; + * Version - a number, that for equal pairs of Path and Time, need to leave only record with maximum version. + * + * Each row in a table correspond to one value of one sensor. + * + * Pattern should contain function, retention scheme, or both of them. The order of patterns does mean as well: + * * Aggregation OR retention patterns should be first + * * Then aggregation AND retention full patterns have to be placed + * * default pattern without regexp must be the last + * + * Rollup rules are specified in the following way: + * + * pattern + * regexp + * function + * pattern + * regexp + * age -> precision + * age -> precision + * ... + * pattern + * regexp + * function + * age -> precision + * age -> precision + * ... + * pattern + * ... + * default + * function + * age -> precision + * ... + * + * regexp - pattern for sensor name + * default - if no pattern has matched + * + * age - minimal data age (in seconds), to start rounding with specified precision. + * precision - rounding precision (in seconds) + * + * function - name of aggregate function to be applied for values, that time was rounded to same. + * + * Example: + * + * + * + * \.max$ + * max + * + * + * click_cost + * any + * + * 0 + * 5 + * + * + * 86400 + * 60 + * + * + * + * max + * + * 0 + * 60 + * + * + * 3600 + * 300 + * + * + * 86400 + * 3600 + * + * + * + */ +namespace Graphite +{ +struct Retention +{ + UInt32 age; + UInt32 precision; +}; + +using Retentions = std::vector; + +struct Pattern +{ + std::shared_ptr regexp; + std::string regexp_str; + AggregateFunctionPtr function; + Retentions retentions; /// Must be ordered by 'age' descending. + enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically +}; + +using Patterns = std::vector; +using RetentionPattern = Pattern; +using AggregationPattern = Pattern; + +struct Params +{ + String config_name; + String path_column_name; + String time_column_name; + String value_column_name; + String version_column_name; + Graphite::Patterns patterns; +}; + +using RollupRule = std::pair; +} + +} diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index 70112dd853a..6b1cda1011f 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -15,127 +16,6 @@ namespace DB { -/** Intended for implementation of "rollup" - aggregation (rounding) of older data - * for a table with Graphite data (Graphite is the system for time series monitoring). - * - * Table with graphite data has at least the following columns (accurate to the name): - * Path, Time, Value, Version - * - * Path - name of metric (sensor); - * Time - time of measurement; - * Value - value of measurement; - * Version - a number, that for equal pairs of Path and Time, need to leave only record with maximum version. - * - * Each row in a table correspond to one value of one sensor. - * - * Pattern should contain function, retention scheme, or both of them. The order of patterns does mean as well: - * * Aggregation OR retention patterns should be first - * * Then aggregation AND retention full patterns have to be placed - * * default pattern without regexp must be the last - * - * Rollup rules are specified in the following way: - * - * pattern - * regexp - * function - * pattern - * regexp - * age -> precision - * age -> precision - * ... - * pattern - * regexp - * function - * age -> precision - * age -> precision - * ... - * pattern - * ... - * default - * function - * age -> precision - * ... - * - * regexp - pattern for sensor name - * default - if no pattern has matched - * - * age - minimal data age (in seconds), to start rounding with specified precision. - * precision - rounding precision (in seconds) - * - * function - name of aggregate function to be applied for values, that time was rounded to same. - * - * Example: - * - * - * - * \.max$ - * max - * - * - * click_cost - * any - * - * 0 - * 5 - * - * - * 86400 - * 60 - * - * - * - * max - * - * 0 - * 60 - * - * - * 3600 - * 300 - * - * - * 86400 - * 3600 - * - * - * - */ -namespace Graphite -{ - struct Retention - { - UInt32 age; - UInt32 precision; - }; - - using Retentions = std::vector; - - struct Pattern - { - std::shared_ptr regexp; - std::string regexp_str; - AggregateFunctionPtr function; - Retentions retentions; /// Must be ordered by 'age' descending. - enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically - }; - - using Patterns = std::vector; - using RetentionPattern = Pattern; - using AggregationPattern = Pattern; - - struct Params - { - String config_name; - String path_column_name; - String time_column_name; - String value_column_name; - String version_column_name; - Graphite::Patterns patterns; - }; - - using RollupRule = std::pair; -} - /** Merges several sorted ports into one. * * For each group of consecutive identical values of the `path` column, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8f4ac7cc79a..30bfde17df3 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include From 27f3d2711ae25b0ade62f33fc8176e1786540ff9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 12:20:43 +0300 Subject: [PATCH 155/743] Move Graphite params to separate file. --- src/Processors/Merges/Graphite.h | 11 +++++++++-- src/Processors/Merges/GraphiteRollupSortedTransform.h | 1 - 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/Graphite.h b/src/Processors/Merges/Graphite.h index 03269de35d0..3ff297987de 100644 --- a/src/Processors/Merges/Graphite.h +++ b/src/Processors/Merges/Graphite.h @@ -1,7 +1,14 @@ #pragma once +#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). * @@ -87,8 +94,9 @@ namespace DB * * */ -namespace Graphite +namespace DB::Graphite { + struct Retention { UInt32 age; @@ -121,6 +129,5 @@ struct Params }; using RollupRule = std::pair; -} } diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index 6b1cda1011f..ee6610935f3 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -7,7 +7,6 @@ #include #include -#include #include #include #include From 16137068b20e90378cae085d08bec9eb5d51f31d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 12:22:51 +0300 Subject: [PATCH 156/743] Move Graphite params to separate file. --- src/DataStreams/TTLBlockInputStream.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 5279ebca948..28e650dd969 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { From 71c72a75d73fe0a7fc1dc199646549f48884f5e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 12:24:16 +0300 Subject: [PATCH 157/743] Move Graphite params to separate file. --- src/Storages/StorageDistributed.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2790d6e9bce..45e1aeff6f4 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -14,6 +14,8 @@ #include #include +#include + #include #include #include From 978576d0c51885ed956db410b5254f7096706d08 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 12:25:52 +0300 Subject: [PATCH 158/743] Move Graphite params to separate file. --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index c560583259c..9fca45725d9 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include From 97158f765663bc7bcc3d4564bd37e66ca1a3ef60 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 12:30:53 +0300 Subject: [PATCH 159/743] Fix build. --- src/Processors/Merges/IMergingAlgorithm.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Merges/IMergingAlgorithm.h b/src/Processors/Merges/IMergingAlgorithm.h index 51ee58cedd2..c01152eb39f 100644 --- a/src/Processors/Merges/IMergingAlgorithm.h +++ b/src/Processors/Merges/IMergingAlgorithm.h @@ -24,6 +24,7 @@ public: virtual void consume(Chunk chunk, size_t source_num) = 0; virtual Status merge() = 0; + IMergingAlgorithm(const IMergingAlgorithm &) = default; IMergingAlgorithm(IMergingAlgorithm &&) = default; virtual ~IMergingAlgorithm() = default; }; From 2aba662d2cae063a0db8c6d58ec960918e040e64 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 12:32:34 +0300 Subject: [PATCH 160/743] Fix build. --- src/Processors/Merges/IMergingAlgorithm.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Merges/IMergingAlgorithm.h b/src/Processors/Merges/IMergingAlgorithm.h index c01152eb39f..edd5b7d883c 100644 --- a/src/Processors/Merges/IMergingAlgorithm.h +++ b/src/Processors/Merges/IMergingAlgorithm.h @@ -24,7 +24,7 @@ public: virtual void consume(Chunk chunk, size_t source_num) = 0; virtual Status merge() = 0; - IMergingAlgorithm(const IMergingAlgorithm &) = default; + IMergingAlgorithm() = default; IMergingAlgorithm(IMergingAlgorithm &&) = default; virtual ~IMergingAlgorithm() = default; }; From ba50519af8bda35a7afc2a6047dde4eb40b29d36 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 10 Apr 2020 13:28:44 +0300 Subject: [PATCH 161/743] workaround for function resolver --- src/Interpreters/DictionaryReader.h | 22 +++++++++---------- .../00065_loyalty_with_storage_join.sql | 7 +++--- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/DictionaryReader.h b/src/Interpreters/DictionaryReader.h index bb13758f46c..823a3690669 100644 --- a/src/Interpreters/DictionaryReader.h +++ b/src/Interpreters/DictionaryReader.h @@ -29,21 +29,21 @@ public: ColumnNumbers arg_positions; size_t result_pos = 0; - FunctionWrapper(const IFunctionOverloadResolver & resolver, const ColumnsWithTypeAndName & arguments, Block & block, + FunctionWrapper(FunctionOverloadResolverPtr resolver, const ColumnsWithTypeAndName & arguments, Block & block, const ColumnNumbers & arg_positions_, const String & column_name, TypeIndex expected_type) : arg_positions(arg_positions_) + , result_pos(block.columns()) { - FunctionBasePtr prepare_function = resolver.build(arguments); - result_pos = block.columns(); + FunctionBasePtr prepared_function = resolver->build(arguments); ColumnWithTypeAndName result; result.name = "get_" + column_name; - result.type = prepare_function->getReturnType(); + result.type = prepared_function->getReturnType(); if (result.type->getTypeId() != expected_type) throw Exception("Type mismatch in dictionary reader for: " + column_name, ErrorCodes::TYPE_MISMATCH); block.insert(result); - function = prepare_function->prepare(block, arg_positions, result_pos); + function = prepared_function->prepare(block, arg_positions, result_pos); } void execute(Block & block, size_t rows) const @@ -60,9 +60,6 @@ public: if (src_column_names.size() != result_columns.size()) throw Exception("Columns number mismatch in dictionary reader", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); - FunctionOverloadResolverPtr dict_has(FunctionFactory::instance().get("dictHas", context)); - FunctionOverloadResolverPtr dict_get(FunctionFactory::instance().get("dictGet", context)); - ColumnWithTypeAndName dict_name; ColumnWithTypeAndName key; ColumnWithTypeAndName column_name; @@ -106,8 +103,8 @@ public: sample_block.insert(key); ColumnNumbers positions_has{0, key_position}; - function_has = std::make_unique( - *dict_has, arguments_has, sample_block, positions_has, "has", DataTypeUInt8().getTypeId()); + function_has = std::make_unique(FunctionFactory::instance().get("dictHas", context), + arguments_has, sample_block, positions_has, "has", DataTypeUInt8().getTypeId()); functions_get.reserve(result_header.columns()); for (size_t i = 0; i < result_header.columns(); ++i) @@ -116,8 +113,9 @@ public: auto & column = result_header.getByPosition(i); arguments_get[1].column = DataTypeString().createColumnConst(1, src_column_names[i]); ColumnNumbers positions_get{0, column_name_pos, key_position}; - functions_get.emplace_back(FunctionWrapper( - *dict_get, arguments_get, sample_block, positions_get, column.name, column.type->getTypeId())); + functions_get.emplace_back( + FunctionWrapper(FunctionFactory::instance().get("dictGet", context), + arguments_get, sample_block, positions_get, column.name, column.type->getTypeId())); } } diff --git a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index d3e73faa7be..2fa337b6ebc 100644 --- a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -3,14 +3,15 @@ SET any_join_distinct_right_table_keys = 1; USE test; DROP TABLE IF EXISTS join; -CREATE TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); +CREATE TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID) +SETTINGS any_join_distinct_right_table_keys = 1; INSERT INTO join SELECT UserID, toInt8(if((sum(SearchEngineID = 2) AS yandex) > (sum(SearchEngineID = 3) AS google), - yandex / (yandex + google), - -google / (yandex + google)) * 10) AS loyalty + yandex / (yandex + google), + -google / (yandex + google)) * 10) AS loyalty FROM hits WHERE (SearchEngineID = 2) OR (SearchEngineID = 3) GROUP BY UserID From 9901bf0f6e7c9dd517b0c1927569d59f4365af04 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 10 Apr 2020 14:03:08 +0300 Subject: [PATCH 162/743] fix column size in KeyGetterForDict --- src/Interpreters/HashJoin.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 16187f10fa1..1c0adb96f66 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -291,16 +291,15 @@ public: using Mapped = JoinStuff::MappedOne; using FindResult = ColumnsHashing::columns_hashing_impl::FindResultImpl; - KeyGetterForDict(const ColumnRawPtrs & key_columns_, const Sizes & key_sizes_, void *) + KeyGetterForDict(const ColumnRawPtrs & key_columns_, const Sizes &, void *) : key_columns(key_columns_) - , key_sizes(key_sizes_) {} FindResult findKey(const DictionaryReader & reader, size_t i, const Arena &) { if (!read_result) { - reader.readKeys(*key_columns[0], key_sizes[0], read_result, found, positions); + reader.readKeys(*key_columns[0], key_columns[0]->size(), read_result, found, positions); result.block = &read_result; /// TODO: check types and correct nullability } @@ -311,7 +310,6 @@ public: private: const ColumnRawPtrs & key_columns; - const Sizes & key_sizes; Block read_result; Mapped result; ColumnVector::Container found; From 7b85a36975fc76b7d52500877cb1f1ef9ae8fdc6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 15:40:16 +0300 Subject: [PATCH 163/743] Added CollapsingSortedAlgorithm. --- ...form.cpp => CollapsingSortedAlgorithm.cpp} | 81 ++++--------------- .../Merges/CollapsingSortedAlgorithm.h | 59 ++++++++++++++ .../Merges/CollapsingSortedTransform.h | 74 ++++------------- .../IMergingAlgorithmWithSharedChunks.cpp | 66 +++++++++++++++ .../IMergingAlgorithmWithSharedChunks.h | 43 ++++++++++ .../Merges/MergingSortedAlgorithm.cpp | 6 -- .../Merges/MergingSortedAlgorithm.h | 1 - 7 files changed, 201 insertions(+), 129 deletions(-) rename src/Processors/Merges/{CollapsingSortedTransform.cpp => CollapsingSortedAlgorithm.cpp} (69%) create mode 100644 src/Processors/Merges/CollapsingSortedAlgorithm.h create mode 100644 src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp create mode 100644 src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h diff --git a/src/Processors/Merges/CollapsingSortedTransform.cpp b/src/Processors/Merges/CollapsingSortedAlgorithm.cpp similarity index 69% rename from src/Processors/Merges/CollapsingSortedTransform.cpp rename to src/Processors/Merges/CollapsingSortedAlgorithm.cpp index d1099184952..a9ee070564c 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.cpp +++ b/src/Processors/Merges/CollapsingSortedAlgorithm.cpp @@ -1,4 +1,5 @@ -#include +#include + #include #include #include @@ -16,70 +17,27 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } -CollapsingSortedTransform::CollapsingSortedTransform( +CollapsingSortedAlgorithm::CollapsingSortedAlgorithm( const Block & header, size_t num_inputs, SortDescription description_, const String & sign_column, size_t max_block_size, WriteBuffer * out_row_sources_buf_, - bool use_average_block_sizes) - : IMergingTransform(num_inputs, header, header, true) + bool use_average_block_sizes, + Logger * log_) + : IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs) , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) - , description(std::move(description_)) , sign_column_number(header.getPositionByName(sign_column)) - , out_row_sources_buf(out_row_sources_buf_) - , chunk_allocator(num_inputs + max_row_refs) - , source_chunks(num_inputs) - , cursors(num_inputs) + , log(log_) { } -void CollapsingSortedTransform::initializeInputs() +void CollapsingSortedAlgorithm::reportIncorrectData() { - queue = SortingHeap(cursors); - is_queue_initialized = true; -} + if (!log) + return; -void CollapsingSortedTransform::consume(Chunk chunk, size_t input_number) -{ - updateCursor(std::move(chunk), input_number); - - if (is_queue_initialized) - queue.push(cursors[input_number]); -} - -void CollapsingSortedTransform::updateCursor(Chunk chunk, size_t source_num) -{ - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); - - chunk.setColumns(std::move(columns), num_rows); - - auto & source_chunk = source_chunks[source_num]; - - if (source_chunk) - { - source_chunk = chunk_allocator.alloc(std::move(chunk)); - cursors[source_num].reset(source_chunk->getColumns(), {}); - } - else - { - if (cursors[source_num].has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - source_chunk = chunk_allocator.alloc(std::move(chunk)); - cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); - } - - source_chunk->all_columns = cursors[source_num].all_columns; - source_chunk->sort_columns = cursors[source_num].sort_columns; -} - -void CollapsingSortedTransform::reportIncorrectData() -{ std::stringstream s; s << "Incorrect data: number of rows with sign = 1 (" << count_positive << ") differs with number of rows with sign = -1 (" << count_negative @@ -102,12 +60,12 @@ void CollapsingSortedTransform::reportIncorrectData() LOG_WARNING(log, s.rdbuf()); } -void CollapsingSortedTransform::insertRow(RowRef & row) +void CollapsingSortedAlgorithm::insertRow(RowRef & row) { merged_data.insertRow(*row.all_columns, row.row_num, row.owned_chunk->getNumRows()); } -void CollapsingSortedTransform::insertRows() +void CollapsingSortedAlgorithm::insertRows() { if (count_positive == 0 && count_negative == 0) { @@ -150,13 +108,7 @@ void CollapsingSortedTransform::insertRows() current_row_sources.size() * sizeof(RowSourcePart)); } -void CollapsingSortedTransform::work() -{ - merge(); - prepareOutputChunk(merged_data); -} - -void CollapsingSortedTransform::merge() +IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() { /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` while (queue.isValid()) @@ -174,7 +126,7 @@ void CollapsingSortedTransform::merge() /// if there are enough rows and the last one is calculated completely if (key_differs && merged_data.hasEnoughRows()) - return; + Status(merged_data.pull()); if (key_differs) { @@ -229,13 +181,12 @@ void CollapsingSortedTransform::merge() { /// We take next block from the corresponding source, if there is one. queue.removeTop(); - requestDataForInput(current.impl->order); - return; + return Status(current.impl->order); } } insertRows(); - is_finished = true; + return Status(merged_data.pull(), true); } } diff --git a/src/Processors/Merges/CollapsingSortedAlgorithm.h b/src/Processors/Merges/CollapsingSortedAlgorithm.h new file mode 100644 index 00000000000..8deacabb9d6 --- /dev/null +++ b/src/Processors/Merges/CollapsingSortedAlgorithm.h @@ -0,0 +1,59 @@ +#pragma once +#include +#include +#include + +namespace Poco +{ + class Logger; +} + +namespace DB +{ + +class CollapsingSortedAlgorithm : public IMergingAlgorithmWithSharedChunks +{ +public: + CollapsingSortedAlgorithm( + const Block & header, + size_t num_inputs, + SortDescription description_, + const String & sign_column, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_, + bool use_average_block_sizes, + Logger * log_); + + Status merge() override; + +private: + MergedData merged_data; + + const size_t sign_column_number; + + static constexpr size_t max_row_refs = 4; /// first_negative, last_positive, last, current. + RowRef first_negative_row; + RowRef last_positive_row; + RowRef last_row; + + size_t count_positive = 0; /// The number of positive rows for the current primary key. + size_t count_negative = 0; /// The number of negative rows for the current primary key. + bool last_is_positive = false; /// true if the last row for the current primary key is positive. + + /// Fields specific for VERTICAL merge algorithm. + /// Row numbers are relative to the start of current primary key. + size_t current_pos = 0; /// Current row number + size_t first_negative_pos = 0; /// Row number of first_negative + size_t last_positive_pos = 0; /// Row number of last_positive + PODArray current_row_sources; /// Sources of rows with the current primary key + + size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log. + Logger * log; + + void reportIncorrectData(); + void insertRow(RowRef & row); + void insertRows(); +}; + +} + diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index 3a3c1540c06..3dd052dfcb4 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -1,6 +1,8 @@ #pragma once #include +#include + #include #include #include @@ -24,7 +26,7 @@ namespace DB * If negative by 1 is greater than positive rows, then only the first negative row is written. * Otherwise, a logical error. */ -class CollapsingSortedTransform final : public IMergingTransform +class CollapsingSortedTransform final : public IMergingTransform2 { public: CollapsingSortedTransform( @@ -34,64 +36,22 @@ public: const String & sign_column, size_t max_block_size, WriteBuffer * out_row_sources_buf_ = nullptr, - bool use_average_block_sizes = false); + bool use_average_block_sizes = false) + : IMergingTransform2( + CollapsingSortedAlgorithm( + header, + num_inputs, + std::move(description_), + sign_column, + max_block_size, + out_row_sources_buf_, + use_average_block_sizes, + &Logger::get("CollapsingSortedTransform")), + num_inputs, header, header, true) + { + } String getName() const override { return "CollapsingSortedTransform"; } - void work() override; - -protected: - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; - -private: - Logger * log = &Logger::get("CollapsingSortedTransform"); - - MergedData merged_data; - - /// Settings - SortDescription description; - const size_t sign_column_number; - - /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) - /// If it is not nullptr then it should be populated during execution - WriteBuffer * out_row_sources_buf = nullptr; - - /// Allocator must be destroyed after all RowRefs. - detail::SharedChunkAllocator chunk_allocator; - - /// Chunks currently being merged. - using SourceChunks = std::vector; - SourceChunks source_chunks; - SortCursorImpls cursors; - - SortingHeap queue; - bool is_queue_initialized = false; - - using RowRef = detail::RowRefWithOwnedChunk; - static constexpr size_t max_row_refs = 4; /// first_negative, last_positive, last, current. - RowRef first_negative_row; - RowRef last_positive_row; - RowRef last_row; - - size_t count_positive = 0; /// The number of positive rows for the current primary key. - size_t count_negative = 0; /// The number of negative rows for the current primary key. - bool last_is_positive = false; /// true if the last row for the current primary key is positive. - - /// Fields specific for VERTICAL merge algorithm. - /// Row numbers are relative to the start of current primary key. - size_t current_pos = 0; /// Current row number - size_t first_negative_pos = 0; /// Row number of first_negative - size_t last_positive_pos = 0; /// Row number of last_positive - PODArray current_row_sources; /// Sources of rows with the current primary key - - size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log. - - void reportIncorrectData(); - void insertRow(RowRef & row); - void insertRows(); - void merge(); - void updateCursor(Chunk chunk, size_t source_num); - void setRowRef(RowRef & row, SortCursor & cursor) { row.set(cursor, source_chunks[cursor.impl->order]); } }; } diff --git a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp b/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp new file mode 100644 index 00000000000..20e1fcffef3 --- /dev/null +++ b/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp @@ -0,0 +1,66 @@ +#include + +namespace DB +{ + +IMergingAlgorithmWithSharedChunks::IMergingAlgorithmWithSharedChunks( + size_t num_inputs, + SortDescription description_, + WriteBuffer * out_row_sources_buf_, + size_t max_row_refs) + : description(std::move(description_)) + , chunk_allocator(num_inputs + max_row_refs) + , source_chunks(num_inputs) + , cursors(num_inputs) + , out_row_sources_buf(out_row_sources_buf_) +{ +} + +static void prepareChunk(Chunk & chunk) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); +} + +void IMergingAlgorithmWithSharedChunks::initialize(Chunks chunks) +{ + source_chunks.resize(chunks.size()); + + for (size_t source_num = 0; source_num < source_chunks.size(); ++source_num) + { + if (!chunks[source_num]) + continue; + + prepareChunk(chunks[source_num]); + + auto & source_chunk = source_chunks[source_num]; + + source_chunk = chunk_allocator.alloc(std::move(chunks[source_num])); + cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); + + source_chunk->all_columns = cursors[source_num].all_columns; + source_chunk->sort_columns = cursors[source_num].sort_columns; + } + + queue = SortingHeap(cursors); +} + +void IMergingAlgorithmWithSharedChunks::consume(Chunk chunk, size_t source_num) +{ + prepareChunk(chunk); + + auto & source_chunk = source_chunks[source_num]; + source_chunk = chunk_allocator.alloc(std::move(chunk)); + cursors[source_num].reset(source_chunk->getColumns(), {}); + + source_chunk->all_columns = cursors[source_num].all_columns; + source_chunk->sort_columns = cursors[source_num].sort_columns; + + queue.push(cursors[source_num]); +} + +} diff --git a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h b/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h new file mode 100644 index 00000000000..19f78629ac4 --- /dev/null +++ b/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h @@ -0,0 +1,43 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class IMergingAlgorithmWithSharedChunks : public IMergingAlgorithm +{ +public: + IMergingAlgorithmWithSharedChunks( + size_t num_inputs, + SortDescription description_, + WriteBuffer * out_row_sources_buf_, + size_t max_row_refs); + + void initialize(Chunks chunks) override; + void consume(Chunk chunk, size_t source_num) override; + +private: + SortDescription description; + + /// Allocator must be destroyed after source_chunks. + detail::SharedChunkAllocator chunk_allocator; + + /// Chunks currently being merged. + using SourceChunks = std::vector; + SourceChunks source_chunks; + SortCursorImpls cursors; + +protected: + SortingHeap queue; + + /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) + /// If it is not nullptr then it should be populated during execution + WriteBuffer * out_row_sources_buf = nullptr; + + using RowRef = detail::RowRefWithOwnedChunk; + void setRowRef(RowRef & row, SortCursor & cursor) { row.set(cursor, source_chunks[cursor.impl->order]); } +}; + +} diff --git a/src/Processors/Merges/MergingSortedAlgorithm.cpp b/src/Processors/Merges/MergingSortedAlgorithm.cpp index 3be9f3f64ef..7a74a20e9ee 100644 --- a/src/Processors/Merges/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/MergingSortedAlgorithm.cpp @@ -48,12 +48,6 @@ static void prepareChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } -void MergingSortedAlgorithm::updateCursor(size_t source_num) -{ - auto & source_chunk = source_chunks[source_num]; - cursors[source_num].reset(source_chunk.getColumns(), {}); -} - void MergingSortedAlgorithm::initialize(Chunks chunks) { source_chunks = std::move(chunks); diff --git a/src/Processors/Merges/MergingSortedAlgorithm.h b/src/Processors/Merges/MergingSortedAlgorithm.h index 5c4087ed0bd..17bcdca1ccf 100644 --- a/src/Processors/Merges/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/MergingSortedAlgorithm.h @@ -49,7 +49,6 @@ private: SortingHeap queue_without_collation; SortingHeap queue_with_collation; - void updateCursor(size_t source_num); Status insertFromChunk(size_t source_num); template From d46c228b9c2c9c6cead0923173c5fb44bced891a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 15:47:30 +0300 Subject: [PATCH 164/743] Fix build. --- src/Processors/Merges/CollapsingSortedAlgorithm.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Merges/CollapsingSortedAlgorithm.h b/src/Processors/Merges/CollapsingSortedAlgorithm.h index 8deacabb9d6..6d6f637c9af 100644 --- a/src/Processors/Merges/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/CollapsingSortedAlgorithm.h @@ -24,6 +24,8 @@ public: bool use_average_block_sizes, Logger * log_); + CollapsingSortedAlgorithm(CollapsingSortedAlgorithm &&) = default; + Status merge() override; private: From e9587b8f4cd3066eb603ac29f763798658b86c3b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 15:53:51 +0300 Subject: [PATCH 165/743] Fix build. --- src/Processors/Merges/IMergingTransform.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 3493ea46af1..163188241b9 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -117,13 +117,13 @@ class IMergingTransform2 : public IMergingTransformBase { public: IMergingTransform2( - Algorithm algorithm_, + Algorithm && algorithm_, size_t num_inputs, const Block & input_header, const Block & output_header, bool have_all_inputs_) : IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_) - , algorithm(std::move(algorithm_)) + , algorithm(std::forward(algorithm_)) { } From 51e5603044cf2300d2ea3a503f43c68bc249e8a7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 16:03:37 +0300 Subject: [PATCH 166/743] Fix build. --- .../Merges/CollapsingSortedAlgorithm.cpp | 2 ++ .../Merges/CollapsingSortedAlgorithm.h | 2 -- .../Merges/CollapsingSortedTransform.h | 27 +++++++------------ src/Processors/Merges/IMergingTransform.h | 7 ++--- .../Merges/MergingSortedAlgorithm.cpp | 5 ++++ .../Merges/MergingSortedAlgorithm.h | 2 -- .../Merges/MergingSortedTransform.cpp | 16 +++++------ 7 files changed, 28 insertions(+), 33 deletions(-) diff --git a/src/Processors/Merges/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/CollapsingSortedAlgorithm.cpp index a9ee070564c..b1e06cd4541 100644 --- a/src/Processors/Merges/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/CollapsingSortedAlgorithm.cpp @@ -5,6 +5,8 @@ #include #include +#include + /// Maximum number of messages about incorrect data in the log. #define MAX_ERROR_MESSAGES 10 diff --git a/src/Processors/Merges/CollapsingSortedAlgorithm.h b/src/Processors/Merges/CollapsingSortedAlgorithm.h index 6d6f637c9af..8deacabb9d6 100644 --- a/src/Processors/Merges/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/CollapsingSortedAlgorithm.h @@ -24,8 +24,6 @@ public: bool use_average_block_sizes, Logger * log_); - CollapsingSortedAlgorithm(CollapsingSortedAlgorithm &&) = default; - Status merge() override; private: diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index 3dd052dfcb4..5a9ed16ea2a 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -3,14 +3,6 @@ #include #include -#include -#include -#include -#include -#include - -#include - namespace DB { @@ -38,16 +30,15 @@ public: WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) : IMergingTransform2( - CollapsingSortedAlgorithm( - header, - num_inputs, - std::move(description_), - sign_column, - max_block_size, - out_row_sources_buf_, - use_average_block_sizes, - &Logger::get("CollapsingSortedTransform")), - num_inputs, header, header, true) + num_inputs, header, header, true, + header, + num_inputs, + std::move(description_), + sign_column, + max_block_size, + out_row_sources_buf_, + use_average_block_sizes, + &Logger::get("CollapsingSortedTransform")) { } diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 163188241b9..260f57806a7 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -116,14 +116,15 @@ template class IMergingTransform2 : public IMergingTransformBase { public: + template IMergingTransform2( - Algorithm && algorithm_, size_t num_inputs, const Block & input_header, const Block & output_header, - bool have_all_inputs_) + bool have_all_inputs_, + Args && ... args) : IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_) - , algorithm(std::forward(algorithm_)) + , algorithm(std::forward(args) ...) { } diff --git a/src/Processors/Merges/MergingSortedAlgorithm.cpp b/src/Processors/Merges/MergingSortedAlgorithm.cpp index 7a74a20e9ee..383d8fe90de 100644 --- a/src/Processors/Merges/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/MergingSortedAlgorithm.cpp @@ -5,6 +5,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + MergingSortedAlgorithm::MergingSortedAlgorithm( const Block & header, size_t num_inputs, diff --git a/src/Processors/Merges/MergingSortedAlgorithm.h b/src/Processors/Merges/MergingSortedAlgorithm.h index 17bcdca1ccf..734caec3424 100644 --- a/src/Processors/Merges/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/MergingSortedAlgorithm.h @@ -19,8 +19,6 @@ public: WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes); - MergingSortedAlgorithm(MergingSortedAlgorithm && other) = default; - void addInput(); void initialize(Chunks chunks) override; diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 2657077c143..68af48062ba 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -8,11 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - MergingSortedTransform::MergingSortedTransform( const Block & header, size_t num_inputs, @@ -24,9 +19,14 @@ MergingSortedTransform::MergingSortedTransform( bool use_average_block_sizes, bool have_all_inputs_) : IMergingTransform2( - MergingSortedAlgorithm(header, num_inputs, std::move(description_), max_block_size, - limit_, out_row_sources_buf_, use_average_block_sizes), - num_inputs, header, header, have_all_inputs_) + num_inputs, header, header, have_all_inputs_, + header, + num_inputs, + std::move(description_), + max_block_size, + limit_, + out_row_sources_buf_, + use_average_block_sizes) , quiet(quiet_) { } From 6cc1389356a77fb7ed0dffa3ea00e8b101698c31 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 10 Apr 2020 10:38:58 -0300 Subject: [PATCH 167/743] Update merge_tree_settings.md --- docs/ru/operations/settings/merge_tree_settings.md | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index afd88694def..9707eee3964 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -1,17 +1,15 @@ # Настройки MergeTree таблиц {#merge-tree-settings} -Значения по умолчанию (для всех таблиц) задаются в config.xml в секции merge_tree. +Значения по умолчанию (для всех таблиц) можно посмотреть в таблице `system.merge_tree_settings`, их можно переопределить в config.xml в секции merge_tree, или задать в секции `SETTINGS` у каждой таблицы. -Пример: +Пример переопределения в `config.xml`: ```text 5 ``` -Эти значения можно задать (перекрыть) у таблиц в секции `SETTINGS` у команды `CREATE TABLE`. - -Пример: +Пример для определения в `SETTINGS` у конкретной таблицы: ```sql CREATE TABLE foo ( From 123c654db384c3676558dbd396c1425714553755 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 16:41:00 +0300 Subject: [PATCH 168/743] Fix build. --- src/Processors/Merges/CollapsingSortedAlgorithm.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Merges/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/CollapsingSortedAlgorithm.cpp index b1e06cd4541..3d0dab37c4e 100644 --- a/src/Processors/Merges/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/CollapsingSortedAlgorithm.cpp @@ -128,7 +128,7 @@ IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() /// if there are enough rows and the last one is calculated completely if (key_differs && merged_data.hasEnoughRows()) - Status(merged_data.pull()); + return Status(merged_data.pull()); if (key_differs) { From ad0c994b28e98a887c923a3dd51ebfe3aa39952c Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 10 Apr 2020 10:48:19 -0300 Subject: [PATCH 169/743] Update merge_tree_settings.md --- docs/ru/operations/settings/merge_tree_settings.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index 9707eee3964..628104dda2b 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -1,6 +1,6 @@ # Настройки MergeTree таблиц {#merge-tree-settings} -Значения по умолчанию (для всех таблиц) можно посмотреть в таблице `system.merge_tree_settings`, их можно переопределить в config.xml в секции merge_tree, или задать в секции `SETTINGS` у каждой таблицы. +Значения настроек merge-tree для (для всех MergeTree таблиц) можно посмотреть в таблице `system.merge_tree_settings`, их можно переопределить в `config.xml` в секции `merge_tree`, или задать в секции `SETTINGS` у каждой таблицы. Пример переопределения в `config.xml`: ```text @@ -20,9 +20,7 @@ ORDER BY tuple() SETTINGS max_suspicious_broken_parts = 500; ``` -Или изменить с помощью команды `ALTER TABLE ... MODIFY SETTING`. - -Пример: +Пример изменения настроек у таблицы командой `ALTER TABLE ... MODIFY SETTING`: ```sql ALTER TABLE foo MODIFY SETTING max_suspicious_broken_parts = 100; From f32ae36f223d16f62e7bb9c3d9284dfdf3b0638b Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 10 Apr 2020 10:48:53 -0300 Subject: [PATCH 170/743] Update merge_tree_settings.md --- docs/ru/operations/settings/merge_tree_settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/merge_tree_settings.md b/docs/ru/operations/settings/merge_tree_settings.md index 628104dda2b..5297e359547 100644 --- a/docs/ru/operations/settings/merge_tree_settings.md +++ b/docs/ru/operations/settings/merge_tree_settings.md @@ -1,6 +1,6 @@ # Настройки MergeTree таблиц {#merge-tree-settings} -Значения настроек merge-tree для (для всех MergeTree таблиц) можно посмотреть в таблице `system.merge_tree_settings`, их можно переопределить в `config.xml` в секции `merge_tree`, или задать в секции `SETTINGS` у каждой таблицы. +Значения настроек merge-tree (для всех MergeTree таблиц) можно посмотреть в таблице `system.merge_tree_settings`, их можно переопределить в `config.xml` в секции `merge_tree`, или задать в секции `SETTINGS` у каждой таблицы. Пример переопределения в `config.xml`: ```text @@ -20,7 +20,7 @@ ORDER BY tuple() SETTINGS max_suspicious_broken_parts = 500; ``` -Пример изменения настроек у таблицы командой `ALTER TABLE ... MODIFY SETTING`: +Пример изменения настроек у конкретной таблицы командой `ALTER TABLE ... MODIFY SETTING`: ```sql ALTER TABLE foo MODIFY SETTING max_suspicious_broken_parts = 100; From 994f95ddb0704d94ea190ac1d05588c87ba33a03 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 17:03:36 +0300 Subject: [PATCH 171/743] Added ReplacingSortedAlgorithm. --- ...sform.cpp => ReplacingSortedAlgorithm.cpp} | 85 ++++--------------- .../Merges/ReplacingSortedAlgorithm.h | 43 ++++++++++ .../Merges/ReplacingSortedTransform.h | 66 +++----------- 3 files changed, 72 insertions(+), 122 deletions(-) rename src/Processors/Merges/{ReplacingSortedTransform.cpp => ReplacingSortedAlgorithm.cpp} (51%) create mode 100644 src/Processors/Merges/ReplacingSortedAlgorithm.h diff --git a/src/Processors/Merges/ReplacingSortedTransform.cpp b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp similarity index 51% rename from src/Processors/Merges/ReplacingSortedTransform.cpp rename to src/Processors/Merges/ReplacingSortedAlgorithm.cpp index 1b29b4842ed..d0b0947aa70 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.cpp +++ b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB @@ -6,71 +6,23 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; +extern const int LOGICAL_ERROR; } -ReplacingSortedTransform::ReplacingSortedTransform( - const Block & header, size_t num_inputs, - SortDescription description_, const String & version_column, - size_t max_block_size, - WriteBuffer * out_row_sources_buf_, - bool use_average_block_sizes) - : IMergingTransform(num_inputs, header, header, true) - , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) - , description(std::move(description_)) - , out_row_sources_buf(out_row_sources_buf_) - , chunk_allocator(num_inputs + max_row_refs) - , source_chunks(num_inputs) - , cursors(num_inputs) +ReplacingSortedAlgorithm::ReplacingSortedAlgorithm( + const Block & header, size_t num_inputs, + SortDescription description_, const String & version_column, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_, + bool use_average_block_sizes) + : IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs) + , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) { if (!version_column.empty()) version_column_number = header.getPositionByName(version_column); } -void ReplacingSortedTransform::initializeInputs() -{ - queue = SortingHeap(cursors); - is_queue_initialized = true; -} - -void ReplacingSortedTransform::consume(Chunk chunk, size_t input_number) -{ - updateCursor(std::move(chunk), input_number); - - if (is_queue_initialized) - queue.push(cursors[input_number]); -} - -void ReplacingSortedTransform::updateCursor(Chunk chunk, size_t source_num) -{ - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); - - chunk.setColumns(std::move(columns), num_rows); - - auto & source_chunk = source_chunks[source_num]; - - if (source_chunk) - { - source_chunk = chunk_allocator.alloc(std::move(chunk)); - cursors[source_num].reset(source_chunk->getColumns(), {}); - } - else - { - if (cursors[source_num].has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - source_chunk = chunk_allocator.alloc(std::move(chunk)); - cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); - } - - source_chunk->all_columns = cursors[source_num].all_columns; - source_chunk->sort_columns = cursors[source_num].sort_columns; -} - -void ReplacingSortedTransform::insertRow() +void ReplacingSortedAlgorithm::insertRow() { if (out_row_sources_buf) { @@ -86,13 +38,7 @@ void ReplacingSortedTransform::insertRow() selected_row.clear(); } -void ReplacingSortedTransform::work() -{ - merge(); - prepareOutputChunk(merged_data); -} - -void ReplacingSortedTransform::merge() +IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() { /// Take the rows in needed order and put them into `merged_columns` until rows no more than `max_block_size` while (queue.isValid()) @@ -109,7 +55,7 @@ void ReplacingSortedTransform::merge() /// if there are enough rows and the last one is calculated completely if (key_differs && merged_data.hasEnoughRows()) - return; + return Status(merged_data.pull()); if (key_differs) { @@ -143,8 +89,7 @@ void ReplacingSortedTransform::merge() { /// We get the next block from the corresponding source, if there is one. queue.removeTop(); - requestDataForInput(current.impl->order); - return; + return Status(current.impl->order); } } @@ -152,7 +97,7 @@ void ReplacingSortedTransform::merge() if (!selected_row.empty()) insertRow(); - is_finished = true; + return Status(merged_data.pull(), true); } } diff --git a/src/Processors/Merges/ReplacingSortedAlgorithm.h b/src/Processors/Merges/ReplacingSortedAlgorithm.h new file mode 100644 index 00000000000..9f3362e2e08 --- /dev/null +++ b/src/Processors/Merges/ReplacingSortedAlgorithm.h @@ -0,0 +1,43 @@ +#pragma once +#include +#include +#include + +namespace Poco +{ +class Logger; +} + +namespace DB +{ + +class ReplacingSortedAlgorithm : public IMergingAlgorithmWithSharedChunks +{ +public: + ReplacingSortedAlgorithm( + const Block & header, size_t num_inputs, + SortDescription description_, const String & version_column, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_ = nullptr, + bool use_average_block_sizes = false); + + Status merge() override; + +private: + MergedData merged_data; + + ssize_t version_column_number = -1; + + using RowRef = detail::RowRefWithOwnedChunk; + static constexpr size_t max_row_refs = 3; /// last, current, selected. + RowRef last_row; + RowRef selected_row; /// Last row with maximum version for current primary key. + size_t max_pos = 0; /// The position (into current_row_sources) of the row with the highest version. + + /// Sources of rows with the current primary key. + PODArray current_row_sources; + + void insertRow(); +}; + +} diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index 194e81ab6db..a6c36cedb71 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -1,13 +1,7 @@ #pragma once #include -#include -#include -#include -#include -#include - -#include +#include namespace DB @@ -17,7 +11,7 @@ namespace DB * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), * keeps row with max `version` value. */ -class ReplacingSortedTransform final : public IMergingTransform +class ReplacingSortedTransform final : public IMergingTransform2 { public: ReplacingSortedTransform( @@ -25,52 +19,20 @@ public: SortDescription description_, const String & version_column, size_t max_block_size, WriteBuffer * out_row_sources_buf_ = nullptr, - bool use_average_block_sizes = false); + bool use_average_block_sizes = false) + : IMergingTransform2( + num_inputs, header, header, true, + header, + num_inputs, + std::move(description_), + version_column, + max_block_size, + out_row_sources_buf_, + use_average_block_sizes) + { + } String getName() const override { return "ReplacingSorted"; } - void work() override; - -protected: - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; - -private: - Logger * log = &Logger::get("ReplacingSortedTransform"); - - MergedData merged_data; - - SortDescription description; - ssize_t version_column_number = -1; - - /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) - /// If it is not nullptr then it should be populated during execution - WriteBuffer * out_row_sources_buf = nullptr; - - /// Allocator must be destroyed after all RowRefs. - detail::SharedChunkAllocator chunk_allocator; - - /// Chunks currently being merged. - using SourceChunks = std::vector; - SourceChunks source_chunks; - SortCursorImpls cursors; - - SortingHeap queue; - bool is_queue_initialized = false; - - using RowRef = detail::RowRefWithOwnedChunk; - static constexpr size_t max_row_refs = 3; /// last, current, selected. - RowRef last_row; - /// RowRef next_key; /// Primary key of next row. - RowRef selected_row; /// Last row with maximum version for current primary key. - size_t max_pos = 0; /// The position (into current_row_sources) of the row with the highest version. - - /// Sources of rows with the current primary key. - PODArray current_row_sources; - - void insertRow(); - void merge(); - void updateCursor(Chunk chunk, size_t source_num); - void setRowRef(RowRef & row, SortCursor & cursor) { row.set(cursor, source_chunks[cursor.impl->order]); } }; } From f35a8bf3517d868dc2076d1b005b20af5d9768cb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 18:41:22 +0300 Subject: [PATCH 172/743] Fix style. --- src/Processors/Merges/CollapsingSortedAlgorithm.cpp | 1 - src/Processors/Merges/ReplacingSortedAlgorithm.cpp | 5 ----- 2 files changed, 6 deletions(-) diff --git a/src/Processors/Merges/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/CollapsingSortedAlgorithm.cpp index 3d0dab37c4e..a0273a470da 100644 --- a/src/Processors/Merges/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/CollapsingSortedAlgorithm.cpp @@ -15,7 +15,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int INCORRECT_DATA; } diff --git a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp index d0b0947aa70..66792e2490a 100644 --- a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp @@ -4,11 +4,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int LOGICAL_ERROR; -} - ReplacingSortedAlgorithm::ReplacingSortedAlgorithm( const Block & header, size_t num_inputs, SortDescription description_, const String & version_column, From 758500893e39cbd870adb0136a86dad1270a572a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Fri, 10 Apr 2020 19:09:44 +0300 Subject: [PATCH 173/743] fixed test --- tests/integration/test_dictionary_custom_settings/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index a9a7c19d4af..97874879525 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -50,7 +50,7 @@ def test_work(start_cluster): caught_exception = '' try: - instance.query("CREATE TABLE source (id UInt64, first String, second String, third String) ENGINE=File(CSVWithNames);") + instance.query("CREATE TABLE source (id UInt64, first String, second String, third String) ENGINE=TinyLog;") instance.query("INSERT INTO default.source VALUES (1, 'aaa', 'bbb', 'cccc'), (2, 'ddd', 'eee', 'fff')") instance.query("SELECT dictGetString('test_clickhouse', 'second', toUInt64(1))") except Exception as e: From 864e0a407fefbce65969f4f52e6a8475cda328ba Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 19:15:51 +0300 Subject: [PATCH 174/743] Added GraphiteRollupSortedAlgorithm. --- ....cpp => GraphiteRollupSortedAlgorithm.cpp} | 100 +++---------- .../Merges/GraphiteRollupSortedAlgorithm.h | 114 ++++++++++++++ .../Merges/GraphiteRollupSortedTransform.h | 141 ++---------------- .../IMergingAlgorithmWithSharedChunks.cpp | 2 +- .../IMergingAlgorithmWithSharedChunks.h | 7 +- 5 files changed, 154 insertions(+), 210 deletions(-) rename src/Processors/Merges/{GraphiteRollupSortedTransform.cpp => GraphiteRollupSortedAlgorithm.cpp} (80%) create mode 100644 src/Processors/Merges/GraphiteRollupSortedAlgorithm.h diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.cpp b/src/Processors/Merges/GraphiteRollupSortedAlgorithm.cpp similarity index 80% rename from src/Processors/Merges/GraphiteRollupSortedTransform.cpp rename to src/Processors/Merges/GraphiteRollupSortedAlgorithm.cpp index 564b697139b..20015dc508a 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.cpp +++ b/src/Processors/Merges/GraphiteRollupSortedAlgorithm.cpp @@ -1,17 +1,15 @@ -#include +#include +#include +#include +#include namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -static GraphiteRollupSortedTransform::ColumnsDefinition defineColumns( +static GraphiteRollupSortedAlgorithm::ColumnsDefinition defineColumns( const Block & header, const Graphite::Params & params) { - GraphiteRollupSortedTransform::ColumnsDefinition def; + GraphiteRollupSortedAlgorithm::ColumnsDefinition def; def.path_column_num = header.getPositionByName(params.path_column_name); def.time_column_num = header.getPositionByName(params.time_column_name); @@ -26,16 +24,12 @@ static GraphiteRollupSortedTransform::ColumnsDefinition defineColumns( return def; } -GraphiteRollupSortedTransform::GraphiteRollupSortedTransform( +GraphiteRollupSortedAlgorithm::GraphiteRollupSortedAlgorithm( const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size, Graphite::Params params_, time_t time_of_merge_) - : IMergingTransform(num_inputs, header, header, true) + : IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), nullptr, max_row_refs) , merged_data(header.cloneEmptyColumns(), false, max_block_size) - , description(std::move(description_)) - , chunk_allocator(num_inputs + max_row_refs) - , source_chunks(num_inputs) - , cursors(num_inputs) , params(std::move(params_)), time_of_merge(time_of_merge_) { size_t max_size_of_aggregate_state = 0; @@ -54,7 +48,7 @@ GraphiteRollupSortedTransform::GraphiteRollupSortedTransform( columns_definition = defineColumns(header, params); } -Graphite::RollupRule GraphiteRollupSortedTransform::selectPatternForPath(StringRef path) const +Graphite::RollupRule GraphiteRollupSortedAlgorithm::selectPatternForPath(StringRef path) const { const Graphite::Pattern * first_match = &undef_pattern; @@ -110,7 +104,7 @@ Graphite::RollupRule GraphiteRollupSortedTransform::selectPatternForPath(StringR return {nullptr, nullptr}; } -UInt32 GraphiteRollupSortedTransform::selectPrecision(const Graphite::Retentions & retentions, time_t time) const +UInt32 GraphiteRollupSortedAlgorithm::selectPrecision(const Graphite::Retentions & retentions, time_t time) const { static_assert(is_signed_v, "time_t must be signed type"); @@ -149,56 +143,7 @@ static time_t roundTimeToPrecision(const DateLUTImpl & date_lut, time_t time, UI } } -void GraphiteRollupSortedTransform::initializeInputs() -{ - queue = SortingHeap(cursors); - is_queue_initialized = true; -} - -void GraphiteRollupSortedTransform::consume(Chunk chunk, size_t input_number) -{ - updateCursor(std::move(chunk), input_number); - - if (is_queue_initialized) - queue.push(cursors[input_number]); -} - -void GraphiteRollupSortedTransform::updateCursor(Chunk chunk, size_t source_num) -{ - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); - - chunk.setColumns(std::move(columns), num_rows); - - auto & source_chunk = source_chunks[source_num]; - - if (source_chunk) - { - source_chunk = chunk_allocator.alloc(std::move(chunk)); - cursors[source_num].reset(source_chunk->getColumns(), {}); - } - else - { - if (cursors[source_num].has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - source_chunk = chunk_allocator.alloc(std::move(chunk)); - cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); - } - - source_chunk->all_columns = cursors[source_num].all_columns; - source_chunk->sort_columns = cursors[source_num].sort_columns; -} - -void GraphiteRollupSortedTransform::work() -{ - merge(); - prepareOutputChunk(merged_data); -} - -void GraphiteRollupSortedTransform::merge() +IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge() { const DateLUTImpl & date_lut = DateLUT::instance(); @@ -257,7 +202,7 @@ void GraphiteRollupSortedTransform::merge() /// next call to merge() the same next_cursor will be processed once more and /// the next output row will be created from it. if (merged_data.hasEnoughRows()) - return; + return Status(merged_data.pull()); } /// At this point previous row has been fully processed, so we can advance the loop @@ -295,8 +240,7 @@ void GraphiteRollupSortedTransform::merge() { /// We get the next block from the appropriate source, if there is one. queue.removeTop(); - requestDataForInput(current.impl->order); - return; + return Status(current.impl->order); } } @@ -307,25 +251,25 @@ void GraphiteRollupSortedTransform::merge() finishCurrentGroup(); } - is_finished = true; + return Status(merged_data.pull(), true); } -void GraphiteRollupSortedTransform::startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule) +void GraphiteRollupSortedAlgorithm::startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule) { merged_data.startNextGroup(cursor->all_columns, cursor->pos, next_rule, columns_definition); } -void GraphiteRollupSortedTransform::finishCurrentGroup() +void GraphiteRollupSortedAlgorithm::finishCurrentGroup() { merged_data.insertRow(current_time_rounded, current_subgroup_newest_row, columns_definition); } -void GraphiteRollupSortedTransform::accumulateRow(RowRef & row) +void GraphiteRollupSortedAlgorithm::accumulateRow(RowRef & row) { merged_data.accumulateRow(row, columns_definition); } -void GraphiteRollupSortedTransform::GraphiteRollupMergedData::startNextGroup( +void GraphiteRollupSortedAlgorithm::GraphiteRollupMergedData::startNextGroup( const ColumnRawPtrs & raw_columns, size_t row, Graphite::RollupRule next_rule, ColumnsDefinition & def) { @@ -345,8 +289,8 @@ void GraphiteRollupSortedTransform::GraphiteRollupMergedData::startNextGroup( was_group_started = true; } -void GraphiteRollupSortedTransform::GraphiteRollupMergedData::insertRow( - time_t time, RowRef & row, ColumnsDefinition & def) +void GraphiteRollupSortedAlgorithm::GraphiteRollupMergedData::insertRow( + time_t time, RowRef & row, ColumnsDefinition & def) { /// Insert calculated values of the columns `time`, `value`, `version`. columns[def.time_column_num]->insert(time); @@ -371,7 +315,7 @@ void GraphiteRollupSortedTransform::GraphiteRollupMergedData::insertRow( was_group_started = false; } -void GraphiteRollupSortedTransform::GraphiteRollupMergedData::accumulateRow(RowRef & row, ColumnsDefinition & def) +void GraphiteRollupSortedAlgorithm::GraphiteRollupMergedData::accumulateRow(RowRef & row, ColumnsDefinition & def) { const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule); if (aggregate_state_created) @@ -381,7 +325,7 @@ void GraphiteRollupSortedTransform::GraphiteRollupMergedData::accumulateRow(RowR } } -GraphiteRollupSortedTransform::GraphiteRollupMergedData::~GraphiteRollupMergedData() +GraphiteRollupSortedAlgorithm::GraphiteRollupMergedData::~GraphiteRollupMergedData() { if (aggregate_state_created) std::get<1>(current_rule)->function->destroy(place_for_aggregate_state.data()); diff --git a/src/Processors/Merges/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/GraphiteRollupSortedAlgorithm.h new file mode 100644 index 00000000000..effbc3fd266 --- /dev/null +++ b/src/Processors/Merges/GraphiteRollupSortedAlgorithm.h @@ -0,0 +1,114 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +class GraphiteRollupSortedAlgorithm : public IMergingAlgorithmWithSharedChunks +{ +public: + GraphiteRollupSortedAlgorithm( + const Block & header, size_t num_inputs, + SortDescription description_, size_t max_block_size, + Graphite::Params params_, time_t time_of_merge_); + + Status merge() override; + + struct ColumnsDefinition + { + size_t path_column_num; + size_t time_column_num; + size_t value_column_num; + size_t version_column_num; + + /// All columns other than 'time', 'value', 'version'. They are unmodified during rollup. + ColumnNumbers unmodified_column_numbers; + }; + + using RowRef = detail::RowRefWithOwnedChunk; + + /// Specialization for SummingSortedTransform. + class GraphiteRollupMergedData : public MergedData + { + public: + using MergedData::MergedData; + ~GraphiteRollupMergedData(); + + void startNextGroup(const ColumnRawPtrs & raw_columns, size_t row, + Graphite::RollupRule next_rule, ColumnsDefinition & def); + void insertRow(time_t time, RowRef & row, ColumnsDefinition & def); + void accumulateRow(RowRef & row, ColumnsDefinition & def); + bool wasGroupStarted() const { return was_group_started; } + + const Graphite::RollupRule & currentRule() const { return current_rule; } + void allocMemForAggregates(size_t size, size_t alignment) { place_for_aggregate_state.reset(size, alignment); } + + private: + Graphite::RollupRule current_rule = {nullptr, nullptr}; + AlignedBuffer place_for_aggregate_state; + bool aggregate_state_created = false; /// Invariant: if true then current_rule is not NULL. + bool was_group_started = false; + }; + +private: + GraphiteRollupMergedData merged_data; + + const Graphite::Params params; + ColumnsDefinition columns_definition; + + time_t time_of_merge; + + /// No data has been read. + bool is_first = true; + + /* | path | time | rounded_time | version | value | unmodified | + * ----------------------------------------------------------------------------------- + * | A | 11 | 10 | 1 | 1 | a | | + * | A | 11 | 10 | 3 | 2 | b |> subgroup(A, 11) | + * | A | 11 | 10 | 2 | 3 | c | |> group(A, 10) + * ----------------------------------------------------------------------------------|> + * | A | 12 | 10 | 0 | 4 | d | |> Outputs (A, 10, avg(2, 5), a) + * | A | 12 | 10 | 1 | 5 | e |> subgroup(A, 12) | + * ----------------------------------------------------------------------------------- + * | A | 21 | 20 | 1 | 6 | f | + * | B | 11 | 10 | 1 | 7 | g | + * ... + */ + + /// Path name of current bucket + StringRef current_group_path; + + static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row. + /// Last row with maximum version for current primary key (time bucket). + RowRef current_subgroup_newest_row; + + /// Time of last read row + 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. + void startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule); + + /// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows. + void finishCurrentGroup(); + + /// Update the state of the aggregate function with the new `value`. + void accumulateRow(RowRef & row); +}; + +} diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index ee6610935f3..0d8493e1eb1 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -1,17 +1,7 @@ #pragma once #include -#include -#include -#include - -#include -#include -#include -#include -#include - -#include +#include namespace DB { @@ -26,130 +16,25 @@ namespace DB * merge `value` values using the specified aggregate functions, * as well as keeping the maximum value of the `version` column. */ -class GraphiteRollupSortedTransform : public IMergingTransform +class GraphiteRollupSortedTransform : public IMergingTransform2 { public: GraphiteRollupSortedTransform( const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size, - Graphite::Params params_, time_t time_of_merge_); + Graphite::Params params_, time_t time_of_merge_) + : IMergingTransform2( + num_inputs, header, header, true, + header, + num_inputs, + std::move(description_), + max_block_size, + std::move(params_), + time_of_merge_) + { + } String getName() const override { return "GraphiteRollupSortedTransform"; } - void work() override; - - struct ColumnsDefinition - { - size_t path_column_num; - size_t time_column_num; - size_t value_column_num; - size_t version_column_num; - - /// All columns other than 'time', 'value', 'version'. They are unmodified during rollup. - ColumnNumbers unmodified_column_numbers; - }; - - using RowRef = detail::RowRefWithOwnedChunk; - - /// Specialization for SummingSortedTransform. - class GraphiteRollupMergedData : public MergedData - { - public: - using MergedData::MergedData; - ~GraphiteRollupMergedData(); - - void startNextGroup(const ColumnRawPtrs & raw_columns, size_t row, - Graphite::RollupRule next_rule, ColumnsDefinition & def); - void insertRow(time_t time, RowRef & row, ColumnsDefinition & def); - void accumulateRow(RowRef & row, ColumnsDefinition & def); - bool wasGroupStarted() const { return was_group_started; } - - const Graphite::RollupRule & currentRule() const { return current_rule; } - void allocMemForAggregates(size_t size, size_t alignment) { place_for_aggregate_state.reset(size, alignment); } - - private: - Graphite::RollupRule current_rule = {nullptr, nullptr}; - AlignedBuffer place_for_aggregate_state; - bool aggregate_state_created = false; /// Invariant: if true then current_rule is not NULL. - bool was_group_started = false; - }; - -protected: - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; - -private: - Logger * log = &Logger::get("GraphiteRollupSortedBlockInputStream"); - - GraphiteRollupMergedData merged_data; - SortDescription description; - - /// Allocator must be destroyed after all RowRefs. - detail::SharedChunkAllocator chunk_allocator; - - /// Chunks currently being merged. - using SourceChunks = std::vector; - SourceChunks source_chunks; - SortCursorImpls cursors; - - SortingHeap queue; - bool is_queue_initialized = false; - - const Graphite::Params params; - ColumnsDefinition columns_definition; - - time_t time_of_merge; - - /// No data has been read. - bool is_first = true; - - /* | path | time | rounded_time | version | value | unmodified | - * ----------------------------------------------------------------------------------- - * | A | 11 | 10 | 1 | 1 | a | | - * | A | 11 | 10 | 3 | 2 | b |> subgroup(A, 11) | - * | A | 11 | 10 | 2 | 3 | c | |> group(A, 10) - * ----------------------------------------------------------------------------------|> - * | A | 12 | 10 | 0 | 4 | d | |> Outputs (A, 10, avg(2, 5), a) - * | A | 12 | 10 | 1 | 5 | e |> subgroup(A, 12) | - * ----------------------------------------------------------------------------------- - * | A | 21 | 20 | 1 | 6 | f | - * | B | 11 | 10 | 1 | 7 | g | - * ... - */ - - /// Path name of current bucket - StringRef current_group_path; - - static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row. - /// Last row with maximum version for current primary key (time bucket). - RowRef current_subgroup_newest_row; - - /// Time of last read row - 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. - void startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule); - - /// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows. - void finishCurrentGroup(); - - /// Update the state of the aggregate function with the new `value`. - void accumulateRow(RowRef & row); - - void merge(); - void updateCursor(Chunk chunk, size_t source_num); }; } diff --git a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp b/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp index 20e1fcffef3..e234b29abbe 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp +++ b/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp @@ -10,8 +10,8 @@ IMergingAlgorithmWithSharedChunks::IMergingAlgorithmWithSharedChunks( size_t max_row_refs) : description(std::move(description_)) , chunk_allocator(num_inputs + max_row_refs) - , source_chunks(num_inputs) , cursors(num_inputs) + , source_chunks(num_inputs) , out_row_sources_buf(out_row_sources_buf_) { } diff --git a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h b/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h index 19f78629ac4..dfce4fd3825 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h +++ b/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h @@ -24,12 +24,13 @@ private: /// Allocator must be destroyed after source_chunks. detail::SharedChunkAllocator chunk_allocator; - /// Chunks currently being merged. - using SourceChunks = std::vector; - SourceChunks source_chunks; SortCursorImpls cursors; protected: + /// Chunks currently being merged. + using SourceChunks = std::vector; + SourceChunks source_chunks; + SortingHeap queue; /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) From 2fa40ad554f0128cd21d3892aa9488db8f5b72e6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 19:18:53 +0300 Subject: [PATCH 175/743] Fix build. --- src/Processors/Merges/AggregatingSortedTransform.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index da26527d435..279ac9c7597 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -5,6 +5,7 @@ #include #include +#include #include #include #include From ba840b5a59e296414bf1483e4330016b6133a2a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 19:44:47 +0300 Subject: [PATCH 176/743] Added VersionedCollapsingAlgorithm. --- ...m.cpp => VersionedCollapsingAlgorithm.cpp} | 84 +++---------------- .../Merges/VersionedCollapsingAlgorithm.h | 40 +++++++++ .../Merges/VersionedCollapsingTransform.h | 68 ++++----------- 3 files changed, 66 insertions(+), 126 deletions(-) rename src/Processors/Merges/{VersionedCollapsingTransform.cpp => VersionedCollapsingAlgorithm.cpp} (64%) create mode 100644 src/Processors/Merges/VersionedCollapsingAlgorithm.h diff --git a/src/Processors/Merges/VersionedCollapsingTransform.cpp b/src/Processors/Merges/VersionedCollapsingAlgorithm.cpp similarity index 64% rename from src/Processors/Merges/VersionedCollapsingTransform.cpp rename to src/Processors/Merges/VersionedCollapsingAlgorithm.cpp index cb0e085c8d1..50cce60b484 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.cpp +++ b/src/Processors/Merges/VersionedCollapsingAlgorithm.cpp @@ -1,85 +1,27 @@ -#include -#include +#include #include +#include namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - static const size_t MAX_ROWS_IN_MULTIVERSION_QUEUE = 8192; -VersionedCollapsingTransform::VersionedCollapsingTransform( +VersionedCollapsingAlgorithm::VersionedCollapsingAlgorithm( const Block & header, size_t num_inputs, SortDescription description_, const String & sign_column_, size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) - : IMergingTransform(num_inputs, header, header, true) + : IMergingAlgorithmWithSharedChunks( + num_inputs, std::move(description_), out_row_sources_buf_, MAX_ROWS_IN_MULTIVERSION_QUEUE) , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) - , description(std::move(description_)) - , out_row_sources_buf(out_row_sources_buf_) , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 1) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer - , chunk_allocator(num_inputs + max_rows_in_queue + 1) /// +1 just in case (for current_row) - , source_chunks(num_inputs) - , cursors(num_inputs) , current_keys(max_rows_in_queue) { sign_column_number = header.getPositionByName(sign_column_); } -void VersionedCollapsingTransform::initializeInputs() -{ - queue = SortingHeap(cursors); - is_queue_initialized = true; -} - -void VersionedCollapsingTransform::consume(Chunk chunk, size_t input_number) -{ - updateCursor(std::move(chunk), input_number); - - if (is_queue_initialized) - queue.push(cursors[input_number]); -} - -void VersionedCollapsingTransform::updateCursor(Chunk chunk, size_t source_num) -{ - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); - - chunk.setColumns(std::move(columns), num_rows); - - auto & source_chunk = source_chunks[source_num]; - - if (source_chunk) - { - source_chunk = chunk_allocator.alloc(std::move(chunk)); - cursors[source_num].reset(source_chunk->getColumns(), {}); - } - else - { - if (cursors[source_num].has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - source_chunk = chunk_allocator.alloc(std::move(chunk)); - cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); - } - - source_chunk->all_columns = cursors[source_num].all_columns; - source_chunk->sort_columns = cursors[source_num].sort_columns; -} - -void VersionedCollapsingTransform::work() -{ - merge(); - prepareOutputChunk(merged_data); -} - inline ALWAYS_INLINE static void writeRowSourcePart(WriteBuffer & buffer, RowSourcePart row_source) { if constexpr (sizeof(RowSourcePart) == 1) @@ -88,7 +30,7 @@ inline ALWAYS_INLINE static void writeRowSourcePart(WriteBuffer & buffer, RowSou buffer.write(reinterpret_cast(&row_source), sizeof(RowSourcePart)); } -void VersionedCollapsingTransform::insertGap(size_t gap_size) +void VersionedCollapsingAlgorithm::insertGap(size_t gap_size) { if (out_row_sources_buf) { @@ -100,7 +42,7 @@ void VersionedCollapsingTransform::insertGap(size_t gap_size) } } -void VersionedCollapsingTransform::insertRow(size_t skip_rows, const RowRef & row) +void VersionedCollapsingAlgorithm::insertRow(size_t skip_rows, const RowRef & row) { merged_data.insertRow(*row.all_columns, row.row_num, row.owned_chunk->getNumRows()); @@ -114,7 +56,7 @@ void VersionedCollapsingTransform::insertRow(size_t skip_rows, const RowRef & ro } } -void VersionedCollapsingTransform::merge() +IMergingAlgorithm::Status VersionedCollapsingAlgorithm::merge() { /// Take rows in correct order and put them into `merged_columns` until the rows no more than `max_block_size` while (queue.isValid()) @@ -153,7 +95,7 @@ void VersionedCollapsingTransform::merge() /// It's ok to return here, because we didn't affect queue. if (merged_data.hasEnoughRows()) - return; + return Status(merged_data.pull()); } if (current_keys.empty()) @@ -183,8 +125,7 @@ void VersionedCollapsingTransform::merge() { /// We take next block from the corresponding source, if there is one. queue.removeTop(); - requestDataForInput(current.impl->order); - return; + return Status(current.impl->order); } } @@ -197,13 +138,12 @@ void VersionedCollapsingTransform::merge() current_keys.popFront(); if (merged_data.hasEnoughRows()) - return; + return Status(merged_data.pull()); } /// Write information about last collapsed rows. insertGap(current_keys.frontGap()); - is_finished = true; + return Status(merged_data.pull(), true); } - } diff --git a/src/Processors/Merges/VersionedCollapsingAlgorithm.h b/src/Processors/Merges/VersionedCollapsingAlgorithm.h new file mode 100644 index 00000000000..78719d4e3dd --- /dev/null +++ b/src/Processors/Merges/VersionedCollapsingAlgorithm.h @@ -0,0 +1,40 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +class VersionedCollapsingAlgorithm : public IMergingAlgorithmWithSharedChunks +{ +public: + /// Don't need version column. It's in primary key. + VersionedCollapsingAlgorithm( + const Block & header, size_t num_inputs, + SortDescription description_, const String & sign_column_, + size_t max_block_size, + WriteBuffer * out_row_sources_buf_ = nullptr, + bool use_average_block_sizes = false); + + Status merge() override; + +private: + MergedData merged_data; + + size_t sign_column_number = 0; + + const size_t max_rows_in_queue; + + /// Rows with the same primary key and sign. + FixedSizeDequeWithGaps current_keys; + Int8 sign_in_queue = 0; + + std::queue current_row_sources; /// Sources of rows with the current primary key + + void insertGap(size_t gap_size); + void insertRow(size_t skip_rows, const RowRef & row); +}; + +} diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index 875377a544b..31b5673ee4d 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -1,15 +1,7 @@ #pragma once #include -#include -#include -#include -#include -#include -#include - -#include -#include +#include namespace DB @@ -20,7 +12,7 @@ namespace DB * (the columns by which the data is sorted, including specially specified version column), * merges any pair of consecutive rows with opposite sign. */ -class VersionedCollapsingTransform final : public IMergingTransform +class VersionedCollapsingTransform final : public IMergingTransform2 { public: /// Don't need version column. It's in primary key. @@ -29,52 +21,20 @@ public: SortDescription description_, const String & sign_column_, size_t max_block_size, WriteBuffer * out_row_sources_buf_ = nullptr, - bool use_average_block_sizes = false); + bool use_average_block_sizes = false) + : IMergingTransform2( + num_inputs, header, header, true, + header, + num_inputs, + std::move(description_), + sign_column_, + max_block_size, + out_row_sources_buf_, + use_average_block_sizes) + { + } String getName() const override { return "VersionedCollapsingTransform"; } - void work() override; - -protected: - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; - -private: - Logger * log = &Logger::get("VersionedCollapsingTransform"); - - MergedData merged_data; - - SortDescription description; - size_t sign_column_number = 0; - - /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step) - /// If it is not nullptr then it should be populated during execution - WriteBuffer * out_row_sources_buf = nullptr; - - using RowRef = detail::RowRefWithOwnedChunk; - const size_t max_rows_in_queue; - - /// Allocator must be destroyed after all RowRefs. - detail::SharedChunkAllocator chunk_allocator; - - /// Chunks currently being merged. - using SourceChunks = std::vector; - SourceChunks source_chunks; - SortCursorImpls cursors; - - SortingHeap queue; - bool is_queue_initialized = false; - - /// Rows with the same primary key and sign. - FixedSizeDequeWithGaps current_keys; - Int8 sign_in_queue = 0; - - std::queue current_row_sources; /// Sources of rows with the current primary key - - void insertGap(size_t gap_size); - void insertRow(size_t skip_rows, const RowRef & row); - void merge(); - void updateCursor(Chunk chunk, size_t source_num); - void setRowRef(RowRef & row, SortCursor & cursor) { row.set(cursor, source_chunks[cursor.impl->order]); } }; } From 44a3e216d13db351e57a5e6e15a9af009adab14e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 20:16:38 +0300 Subject: [PATCH 177/743] Try simplify Replacing. --- .../Merges/ReplacingSortedAlgorithm.cpp | 17 +++++++---------- .../Merges/ReplacingSortedAlgorithm.h | 3 +-- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp index 66792e2490a..2245c1d6e7c 100644 --- a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp @@ -40,23 +40,20 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() { SortCursor current = queue.current(); - if (last_row.empty()) - setRowRef(last_row, current); - RowRef current_row; setRowRef(current_row, current); - bool key_differs = !current_row.hasEqualSortColumnsWith(last_row); - - /// if there are enough rows and the last one is calculated completely - if (key_differs && merged_data.hasEnoughRows()) - return Status(merged_data.pull()); - + bool key_differs = selected_row.empty() || !current_row.hasEqualSortColumnsWith(selected_row); if (key_differs) { + selected_row.clear(); + + /// if there are enough rows and the last one is calculated completely + if (merged_data.hasEnoughRows()) + return Status(merged_data.pull()); + /// Write the data for the previous primary key. insertRow(); - last_row.swap(current_row); } /// Initially, skip all rows. Unskip last on insert. diff --git a/src/Processors/Merges/ReplacingSortedAlgorithm.h b/src/Processors/Merges/ReplacingSortedAlgorithm.h index 9f3362e2e08..4dbdf5d7ad8 100644 --- a/src/Processors/Merges/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/ReplacingSortedAlgorithm.h @@ -29,8 +29,7 @@ private: ssize_t version_column_number = -1; using RowRef = detail::RowRefWithOwnedChunk; - static constexpr size_t max_row_refs = 3; /// last, current, selected. - RowRef last_row; + static constexpr size_t max_row_refs = 2; /// last, current. RowRef selected_row; /// Last row with maximum version for current primary key. size_t max_pos = 0; /// The position (into current_row_sources) of the row with the highest version. From ca8c78bfa2c02de5f507f5733ad5f23bfc206ba0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 20:19:44 +0300 Subject: [PATCH 178/743] Try simplify Replacing. --- src/Processors/Merges/ReplacingSortedAlgorithm.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp index 2245c1d6e7c..03cbc936ec2 100644 --- a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp @@ -46,14 +46,13 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() bool key_differs = selected_row.empty() || !current_row.hasEqualSortColumnsWith(selected_row); if (key_differs) { - selected_row.clear(); - /// if there are enough rows and the last one is calculated completely if (merged_data.hasEnoughRows()) return Status(merged_data.pull()); /// Write the data for the previous primary key. insertRow(); + selected_row.clear(); } /// Initially, skip all rows. Unskip last on insert. From 198f02d4cfae0c2a97259a99875685e950b7ff51 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Apr 2020 20:22:15 +0300 Subject: [PATCH 179/743] Try simplify Replacing. --- src/Processors/Merges/ReplacingSortedAlgorithm.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp index 03cbc936ec2..bdebf2aba66 100644 --- a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/ReplacingSortedAlgorithm.cpp @@ -51,7 +51,9 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() return Status(merged_data.pull()); /// Write the data for the previous primary key. - insertRow(); + if (!selected_row.empty()) + insertRow(); + selected_row.clear(); } From 68960b59efa21fd7992629f5ee8ba644f483e252 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 2 Apr 2020 02:42:21 +0300 Subject: [PATCH 180/743] Documentation for SimpleAggregateFunction Someone may not want using something that is not documented (what a crazy caution). Follow-up for: #4629 --- .../mergetree_family/aggregatingmergetree.md | 5 ++- .../simpleaggregatefunction.md | 36 +++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md diff --git a/docs/en/engines/table_engines/mergetree_family/aggregatingmergetree.md b/docs/en/engines/table_engines/mergetree_family/aggregatingmergetree.md index 2103efe98dc..9e310d313b9 100644 --- a/docs/en/engines/table_engines/mergetree_family/aggregatingmergetree.md +++ b/docs/en/engines/table_engines/mergetree_family/aggregatingmergetree.md @@ -9,7 +9,10 @@ The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree), alte You can use `AggregatingMergeTree` tables for incremental data aggregation, including for aggregated materialized views. -The engine processes all columns with [AggregateFunction](../../../sql_reference/data_types/aggregatefunction.md) type. +The engine processes all columns with the following types: + +- [AggregateFunction](../../../sql_reference/data_types/aggregatefunction.md) +- [SimpleAggregateFunction](../../../sql_reference/data_types/simpleaggregatefunction.md) It is appropriate to use `AggregatingMergeTree` if it reduces the number of rows by orders. diff --git a/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md b/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md new file mode 100644 index 00000000000..4fd4c37254b --- /dev/null +++ b/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md @@ -0,0 +1,36 @@ +# SimpleAggregateFunction(name, types\_of\_arguments…) {#data-type-simpleaggregatefunction} + +Unlike [`AggregateFunction`](../aggregatefunction.md), which stores not the value of the aggregate function but it's state: + +- `SimpleAggregateFunction` data type stores current value of aggregation and does not store aggregation state as [`AggregateFunction`](../aggregatefunction.md). +. +It supports simple stateless aggregate functions, including: + + - [`any`](../../query_language/agg_functions/reference.md#agg_function-any) + - [`anyLast`](../../query_language/agg_functions/reference.md#anylastx) + - [`min`](../../query_language/agg_functions/reference.md#agg_function-min) + - [`max`](../../query_language/agg_functions/reference.md#agg_function-max) + - [`sum`](../../query_language/agg_functions/reference.md#agg_function-sum) + - [`groupBitAnd`](../../query_language/agg_functions/reference.md#groupbitand) + - [`groupBitOr`](../../query_language/agg_functions/reference.md#groupbitor) + - [`groupBitXor`](../../query_language/agg_functions/reference.md#groupbitxor) + +- Type of the `SimpleAggregateFunction(func, Type)` is `Type` itself, so you do not need to apply functions with `-Merge`/`-State` suffixes. +- `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function. + +**Parameters** + +- Name of the aggregate function. +- Types of the aggregate function arguments. + +**Example** + +``` sql +CREATE TABLE t +( + column1 SimpleAggregateFunction(sum, UInt64), + column2 SimpleAggregateFunction(any, String) +) ENGINE = ... +``` + +[Original article](https://clickhouse.tech/docs/en/data_types/nested_data_structures/simpleaggregatefunction/) From a8d72d01c3c70855480e87d910434b4486a47e12 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 11 Apr 2020 00:29:54 +0300 Subject: [PATCH 181/743] mvp fix --- src/Interpreters/InterserverIOHandler.h | 8 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +- .../MergeTree/MergeTreeDataMergerMutator.h | 8 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 34 +++ .../MergeTree/ReplicatedMergeTreeQueue.h | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 163 ++++++++++- src/Storages/StorageReplicatedMergeTree.h | 8 + .../test_polymorphic_parts/test.py | 2 + .../configs/conf.d/clusters.xml | 37 --- .../configs/conf.d/ddl.xml | 5 - .../configs/users.d/settings.xml | 12 + .../test_quorum_inserts/configs/users.xml | 27 -- tests/integration/test_quorum_inserts/test.py | 266 +++++++++++------- 13 files changed, 393 insertions(+), 185 deletions(-) delete mode 100644 tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml delete mode 100644 tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml create mode 100644 tests/integration/test_quorum_inserts/configs/users.d/settings.xml delete mode 100644 tests/integration/test_quorum_inserts/configs/users.xml diff --git a/src/Interpreters/InterserverIOHandler.h b/src/Interpreters/InterserverIOHandler.h index 0e85f30c886..0ffccb6c33c 100644 --- a/src/Interpreters/InterserverIOHandler.h +++ b/src/Interpreters/InterserverIOHandler.h @@ -34,7 +34,7 @@ class InterserverIOEndpoint public: virtual std::string getId(const std::string & path) const = 0; virtual void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response) = 0; - virtual ~InterserverIOEndpoint() {} + virtual ~InterserverIOEndpoint() = default; /// You need to stop the data transfer if blocker is activated. ActionBlocker blocker; @@ -53,8 +53,6 @@ public: void addEndpoint(const String & name, InterserverIOEndpointPtr endpoint) { std::lock_guard lock(mutex); - LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), "anime addEndpoint() " << name); - LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), StackTrace().toString()); bool inserted = endpoint_map.try_emplace(name, std::move(endpoint)).second; if (!inserted) throw Exception("Duplicate interserver IO endpoint: " + name, ErrorCodes::DUPLICATE_INTERSERVER_IO_ENDPOINT); @@ -63,8 +61,6 @@ public: bool removeEndpointIfExists(const String & name) { std::lock_guard lock(mutex); - LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), "anime removeEndpointIfExists() " << name); - LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), StackTrace().toString()); return endpoint_map.erase(name); } @@ -72,8 +68,6 @@ public: try { std::lock_guard lock(mutex); - LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), "anime getEndpoint() " << name); - LOG_FATAL(&Poco::Logger::get("InterserverIOHandler"), StackTrace().toString()); return endpoint_map.at(name); } catch (...) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 2b732d879b0..f892302086d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -204,7 +204,8 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, - String * out_disable_reason) + String * out_disable_reason, + const AllowedSingleMergePredicate & single_merge) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); const auto data_settings = data.getSettings(); @@ -225,6 +226,9 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( bool has_part_with_expired_ttl = false; for (const MergeTreeData::DataPartPtr & part : data_parts) { + if (!single_merge(part, nullptr)) + continue; + const String & partition_id = part->info.partition_id; if (!prev_partition_id || partition_id != *prev_partition_id || (prev_part && !can_merge_callback(*prev_part, part, nullptr))) { diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index b24b56a4780..0efdb403bf0 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -49,7 +49,10 @@ struct FutureMergedMutatedPart class MergeTreeDataMergerMutator { public: - using AllowedMergingPredicate = std::function; + using AllowedMergingPredicate = std::function; + using AllowedSingleMergePredicate = std::function; +// template +// using AllowedMergingPredicate = std::function; public: MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size); @@ -81,7 +84,8 @@ public: bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge, - String * out_disable_reason = nullptr); + String * out_disable_reason = nullptr, + const AllowedSingleMergePredicate & single_merge = [](const MergeTreeData::DataPartPtr &, String *) -> bool { return true; }); /** Select all the parts in the specified partition for merge, if possible. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 796664d889c..3e9f476e5ec 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1781,6 +1781,40 @@ bool ReplicatedMergeTreeMergePredicate::operator()( return true; } +bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const +{ + LOG_FATAL(&Poco::Logger::get("ReplicatedMergeTreeMergePredicate::operator()"), "begin"); + + if (part->name == inprogress_quorum_part) + { + LOG_FATAL(&Poco::Logger::get("ReplicatedMergeTreeMergePredicate"), "operator()"); + if (out_reason) + *out_reason = "Quorum insert for part " + part->name + " is currently in progress"; + return false; + } + + if (prev_virtual_parts.getContainingPart(part->info).empty()) + { + if (out_reason) + *out_reason = "Entry for part " + part->name + " hasn't been read from the replication log yet"; + return false; + } + + std::lock_guard lock(queue.state_mutex); + + /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer + /// and it is guaranteed that it will contain all merges assigned before this object is constructed. + String containing_part = queue.virtual_parts.getContainingPart(part->info); + if (containing_part != part->name) + { + if (out_reason) + *out_reason = "Part " + part->name >+ " has already been assigned a merge into " + containing_part; + return false; + } + + return true; +} + std::optional> ReplicatedMergeTreeMergePredicate::getDesiredMutationVersion(const MergeTreeData::DataPartPtr & part) const { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index fcb3dfb4b86..e31c0eb7ab1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -412,6 +412,8 @@ public: const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String * out_reason = nullptr) const; + bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; + /// Return nonempty optional of desired mutation version and alter version. /// If we have no alter (modify/drop) mutations in mutations queue, than we return biggest possible /// mutation version (and -1 as alter version). In other case, we return biggest mutation version with diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7107328e4ff..ff4d5df03b9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2052,6 +2052,8 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() { + LOG_FATAL(&Poco::Logger::get("queueTask()"), "begin"); + /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) { @@ -2189,8 +2191,14 @@ void StorageReplicatedMergeTree::mergeSelectingTask() FutureMergedMutatedPart future_merged_part; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred)) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, nullptr, + [&merge_pred](const MergeTreeData::DataPartPtr & part, String * explain) -> bool { return merge_pred.canMergeSinglePart(part, explain); })) { + /// We have to exclude parts, that are currently being writted with quorum. (check .../quorum/status node) + /// Also we have to exclude last parts written with quorum (check .../quorum/last_part node) +// if (global_context.getSettingsRef().insert_quorum) +// excludeSomePartsFromMerge(future_merged_part); + success = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, force_ttl); } @@ -2575,6 +2583,7 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart( */ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) { + LOG_FATAL(&Poco::Logger::get("updateQuorum"), "BEGIN!"); auto zookeeper = getZooKeeper(); /// Information on which replicas a part has been added, if the quorum has not yet been reached. @@ -2667,6 +2676,140 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } +void StorageReplicatedMergeTree::deletePartFromPendingQuorum(const String & part_name) +{ + auto zookeeper = getZooKeeper(); + /// Information on which replicas a part has been added, if the quorum has not yet been reached. + const String quorum_status_path = zookeeper_path + "/quorum/status"; + + /// Delete "status" node if required. + + String value; + Coordination::Stat stat; + + /// If there is no node, then all quorum INSERTs have already reached the quorum, and nothing is needed. + while (zookeeper->tryGet(quorum_status_path, value, &stat)) + { + ReplicatedMergeTreeQuorumEntry quorum_entry; + quorum_entry.fromString(value); + + if (quorum_entry.part_name != part_name) + { + /// There is no information about interested part in this node. + break; + } + + /// Since that we are sure that interested part is being involved in insert with quorum. + /// Our goal is to delete "status" node and information from "last_part" node. + + auto code = zookeeper->tryRemove(quorum_status_path, stat.version); + + if (code == Coordination::ZOK) + { + break; + } + else if (code == Coordination::ZNONODE) + { + /// The quorum has already been achieved. + break; + } + else if (code == Coordination::ZBADVERSION) + { + /// Node was updated meanwhile. We must re-read it and repeat all the actions. + continue; + } + else + throw Coordination::Exception(code, quorum_status_path); + } +} + +void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id, const String & part_name) +{ + auto zookeeper = getZooKeeper(); + + /// The name of the previous part for which the quorum was reached. + const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + + /// Delete information from "last_part" node. + + while (true) + { + Coordination::Requests ops; + Coordination::Responses responses; + + Coordination::Stat added_parts_stat; + String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); + + ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(format_version); + + if (!old_added_parts.empty()) + parts_with_quorum.fromString(old_added_parts); + + /// Delete information about particular partition. + + /// Since c++20. + if (!parts_with_quorum.added_parts.contains(partition_id)) + { + /// There is no information about interested part. + break; + } + + /// De Morgan's law + if (part_name == "" || parts_with_quorum.added_parts[partition_id] == part_name) + parts_with_quorum.added_parts.erase(partition_id); + else + break; + + String new_added_parts = parts_with_quorum.toString(); + + auto code = zookeeper->trySet(quorum_last_part_path, new_added_parts, added_parts_stat.version); + + if (code == Coordination::ZOK) + { + break; + } + else if (code == Coordination::ZNONODE) + { + /// Node is deleted. It is impossible, but it is Ok. + break; + } + else if (code == Coordination::ZBADVERSION) + { + /// Node was updated meanwhile. We must re-read it and repeat all the actions. + continue; + } + else + throw Coordination::Exception(code, quorum_last_part_path); + } +} + + +void StorageReplicatedMergeTree::excludeSomePartsFromMerge(FutureMergedMutatedPart & future_part) +{ + LOG_FATAL(&Poco::Logger::get("excludeSomePartsFromMerge"), "BEGIN!"); + + auto zookeeper = getZooKeeper(); + /// Information on which replicas a part has been added, if the quorum has not yet been reached. + const String quorum_status_path = zookeeper_path + "/quorum/status"; + + String value; + Coordination::Stat stat; + + if (zookeeper->tryGet(quorum_status_path, value, &stat)) { + ReplicatedMergeTreeQuorumEntry quorum_entry; + quorum_entry.fromString(value); + + MergeTreeData::DataPartsVector & parts_to_merge = future_part.parts; + + parts_to_merge.erase( + std::remove_if( + parts_to_merge.begin(), parts_to_merge.end(), + [&quorum_entry](const MergeTreeData::DataPartPtr & part_to_merge) { return part_to_merge->name == quorum_entry.part_name; }), + parts_to_merge.end()); + } +} + + bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & source_replica_path, bool to_detached, size_t quorum) { const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); @@ -2871,6 +3014,7 @@ void StorageReplicatedMergeTree::startup() void StorageReplicatedMergeTree::shutdown() { + LOG_FATAL(&Poco::Logger::get("shutdown"), "SHUTDOWN!"); clearOldPartsFromFilesystem(true); /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. fetcher.blocker.cancelForever(); @@ -3510,6 +3654,9 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPt String partition_id = getPartitionIDFromQuery(partition, query_context); + if (query_context.getSettingsRef().insert_quorum) + cleanLastPartNode(partition_id); + LogEntry entry; if (dropPartsInPartition(*zookeeper, partition_id, entry, detach)) { @@ -5177,10 +5324,17 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti return merger_mutator.ttl_merges_blocker.cancel(); if (action_type == ActionLocks::PartsFetch) - return fetcher.blocker.cancel(); + { + return fetcher.blocker.cancel(); + } + if (action_type == ActionLocks::PartsSend) - return data_parts_exchange_endpoint ? data_parts_exchange_endpoint->blocker.cancel() : ActionLock(); + { + LOG_FATAL(&Poco::Logger::get("ActionLock"), "Cancel PartsSend"); + return data_parts_exchange_endpoint ? data_parts_exchange_endpoint->blocker.cancel() : ActionLock(); + } + if (action_type == ActionLocks::ReplicationQueue) return queue.actions_blocker.cancel(); @@ -5198,6 +5352,9 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI /// Let's fetch new log entries firstly queue.pullLogsToQueue(getZooKeeper()); + /// This is significant, because the execution of this task could be delayed at BackgroundPool. + /// And we force it to be executed. + queue_task_handle->wake(); Poco::Event target_size_event; auto callback = [&target_size_event, queue_size] (size_t new_queue_size) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 01dd32614f9..bf07b592e2f 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -464,6 +464,14 @@ private: /// With the quorum being tracked, add a replica to the quorum for the part. void updateQuorum(const String & part_name); + /// Delete particular part name not to track it in future. + void deletePartFromPendingQuorum(const String & part_name); + + + void cleanLastPartNode(const String & partition_id, const String & part_name = ""); + + void excludeSomePartsFromMerge(FutureMergedMutatedPart & future_part); + /// Creates new block number if block with such block_id does not exist std::optional allocateBlockNumber( const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index f7256de9d9a..fdf9ac2834b 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -90,6 +90,8 @@ def test_polymorphic_parts_basics(start_cluster, first_node, second_node): first_node.query("SYSTEM STOP MERGES") second_node.query("SYSTEM STOP MERGES") + print(first_node.query("SELECT * FROM system.settings where name='insert_quorum' format Vertical")) + for size in [300, 300, 600]: insert_random_data('polymorphic_table', first_node, size) second_node.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20) diff --git a/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml b/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml deleted file mode 100644 index adf6ad80247..00000000000 --- a/tests/integration/test_quorum_inserts/configs/conf.d/clusters.xml +++ /dev/null @@ -1,37 +0,0 @@ - - - - - - true - - s0_0_0 - 9000 - - - s0_0_1 - 9000 - - - - - - - true - - s0_0_0 - 9000 - - - s0_0_1 - 9000 - - - s0_0_2 - 9000 - - - - - - diff --git a/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml b/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml deleted file mode 100644 index abad0dee450..00000000000 --- a/tests/integration/test_quorum_inserts/configs/conf.d/ddl.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - /clickhouse/task_queue/ddl - - \ No newline at end of file diff --git a/tests/integration/test_quorum_inserts/configs/users.d/settings.xml b/tests/integration/test_quorum_inserts/configs/users.d/settings.xml new file mode 100644 index 00000000000..0dbee66ff44 --- /dev/null +++ b/tests/integration/test_quorum_inserts/configs/users.d/settings.xml @@ -0,0 +1,12 @@ + + + + + 10000000000 + 0 + + 2 + 1 + + + \ No newline at end of file diff --git a/tests/integration/test_quorum_inserts/configs/users.xml b/tests/integration/test_quorum_inserts/configs/users.xml deleted file mode 100644 index c5114c10cde..00000000000 --- a/tests/integration/test_quorum_inserts/configs/users.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - - 1 - 2 - 1 - 5000 - - - - - - - - ::/0 - - default - default - - - - - - - - diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 1f6c1b9b852..64190c6d8aa 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -1,173 +1,233 @@ -import os -import sys import time import pytest +from helpers.test_tools import TSV from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) +zero = cluster.add_instance("zero", + config_dir="configs", + macros={"cluster": "anime", "shard": "0", "replica": "zero"}, + with_zookeeper=True) + +first = cluster.add_instance("first", + config_dir="configs", + macros={"cluster": "anime", "shard": "0", "replica": "first"}, + with_zookeeper=True) + +second = cluster.add_instance("second", + config_dir="configs", + macros={"cluster": "anime", "shard": "0", "replica": "second"}, + with_zookeeper=True) + +def execute_on_all_cluster(query_): + for node in [zero, first, second]: + node.query(query_) + @pytest.fixture(scope="module") def started_cluster(): global cluster try: - clusters_schema = { - "0" : {"0" : ["0", "1", "2"]} - } - - for cluster_name, shards in clusters_schema.iteritems(): - for shard_name, replicas in shards.iteritems(): - for replica_name in replicas: - name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) - cluster.add_instance(name, - config_dir="configs", - macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, - with_zookeeper=True) - cluster.start() yield cluster finally: cluster.shutdown() -def test_drop_replica_and_achieve_quorum(started_cluster): - zero = cluster.instances['s0_0_0'] - first = cluster.instances['s0_0_1'] - second = cluster.instances['s0_0_2'] - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") - zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_three_replicas") +def test_simple_add_replica(started_cluster): + execute_on_all_cluster("DROP TABLE IF EXISTS test_simple") - create_query = "CREATE TABLE bug.test_drop_replica_and_achieve_quorum " \ + create_query = "CREATE TABLE test_simple " \ "(a Int8, d Date) " \ - "Engine = ReplicatedMergeTree('/clickhouse/tables/test_drop_replica_and_achieve_quorum', '{}') " \ + "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ + "PARTITION BY d ORDER BY a" + + zero.query(create_query) + first.query(create_query) + + first.query("SYSTEM STOP FETCHES test_simple") + + zero.query("INSERT INTO test_simple VALUES (1, '2011-01-01')", settings={'insert_quorum' : 1}) + + assert '1\t2011-01-01\n' == zero.query("SELECT * from test_simple") + assert '' == first.query("SELECT * from test_simple") + + first.query("SYSTEM START FETCHES test_simple") + + zero.query("SYSTEM SYNC REPLICA test_simple", timeout=20) + + assert '1\t2011-01-01\n' == zero.query("SELECT * from test_simple") + assert '1\t2011-01-01\n' == first.query("SELECT * from test_simple") + + second.query(create_query) + second.query("SYSTEM SYNC REPLICA test_simple", timeout=20) + + assert '1\t2011-01-01\n' == zero.query("SELECT * from test_simple") + assert '1\t2011-01-01\n' == first.query("SELECT * from test_simple") + assert '1\t2011-01-01\n' == second.query("SELECT * from test_simple") + + execute_on_all_cluster("DROP TABLE IF EXISTS test_simple") + + + +def test_drop_replica_and_achieve_quorum(started_cluster): + execute_on_all_cluster("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum") + + create_query = "CREATE TABLE test_drop_replica_and_achieve_quorum " \ + "(a Int8, d Date) " \ + "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ "PARTITION BY d ORDER BY a" print("Create Replicated table with two replicas") - zero.query(create_query.format(0)) - first.query(create_query.format(1)) + zero.query(create_query) + first.query(create_query) print("Stop fetches on one replica. Since that, it will be isolated.") - first.query("SYSTEM STOP FETCHES bug.test_drop_replica_and_achieve_quorum") + first.query("SYSTEM STOP FETCHES test_drop_replica_and_achieve_quorum") print("Insert to other replica. This query will fail.") - quorum_timeout = zero.query_and_get_error("INSERT INTO bug.test_drop_replica_and_achieve_quorum(a,d) VALUES (1, '2011-01-01')") + quorum_timeout = zero.query_and_get_error("INSERT INTO test_drop_replica_and_achieve_quorum(a,d) VALUES (1, '2011-01-01')", + settings={'insert_quorum_timeout' : 5000}) assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." - assert "1\t2011-01-01\n" == zero.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum", - settings={'select_sequential_consistency' : 0}) + assert TSV("1\t2011-01-01\n") == TSV(zero.query("SELECT * FROM test_drop_replica_and_achieve_quorum", + settings={'select_sequential_consistency' : 0})) - print("Add third replica") - second.query(create_query.format(2)) - - zero.query("SYSTEM RESTART REPLICA bug.test_drop_replica_and_achieve_quorum") + assert TSV("") == TSV(zero.query("SELECT * FROM test_drop_replica_and_achieve_quorum", + settings={'select_sequential_consistency' : 1})) + #TODO:(Mikhaylov) begin; maybe delete this lines. I want clickhouse to fetch parts and update quorum. print("START FETCHES first replica") - first.query("SYSTEM START FETCHES bug.test_drop_replica_and_achieve_quorum") - - time.sleep(5) - - print(zero.query("SELECT * from system.replicas format Vertical")) - - - print("---------") - print(zero.query("SELECT * from system.replication_queue format Vertical")) - print("---------") - - - print(first.query("SELECT * from system.replicas format Vertical")) - print("---------") - print(first.query("SELECT * from system.replication_queue format Vertical")) - print("---------") - print(second.query("SELECT * from system.replicas format Vertical")) - print("---------") - print(first.query("SELECT * from system.replication_queue format Vertical")) - + first.query("SYSTEM START FETCHES test_drop_replica_and_achieve_quorum") print("SYNC first replica") - first.query("SYSTEM SYNC REPLICA bug.test_drop_replica_and_achieve_quorum") + first.query("SYSTEM SYNC REPLICA test_drop_replica_and_achieve_quorum", timeout=20) + #TODO:(Mikhaylov) end + + print("Add second replica") + second.query(create_query) print("SYNC second replica") - second.query("SYSTEM SYNC REPLICA bug.test_drop_replica_and_achieve_quorum") + second.query("SYSTEM SYNC REPLICA test_drop_replica_and_achieve_quorum", timeout=20) print("Quorum for previous insert achieved.") - assert "1\t2011-01-01\n" == second.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum", - settings={'select_sequential_consistency' : 1}) + assert TSV("1\t2011-01-01\n") == TSV(second.query("SELECT * FROM test_drop_replica_and_achieve_quorum", + settings={'select_sequential_consistency' : 1})) print("Now we can insert some other data.") - zero.query("INSERT INTO bug.test_drop_replica_and_achieve_quorum(a,d) VALUES (2, '2012-02-02')") + zero.query("INSERT INTO test_drop_replica_and_achieve_quorum(a,d) VALUES (2, '2012-02-02')") - assert "1\t2011-01-01\n2 2012-02-02" == zero.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum") - assert "1\t2011-01-01\n2 2012-02-02" == second.query("SELECT * FROM bug.test_drop_replica_and_achieve_quorum") + assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV(zero.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a")) + assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV(first.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a")) + assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV(second.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a")) - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + execute_on_all_cluster("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum") -def test_insert_quorum_with_drop_partition(started_cluster): - zero = cluster.instances['s0_0_0'] - first = cluster.instances['s0_0_1'] - second = cluster.instances['s0_0_2'] +@pytest.mark.parametrize( + ('add_new_data'), + [ + False, + True + ] +) - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") - zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_three_replicas") +def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): + execute_on_all_cluster("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition") - zero.query("CREATE TABLE bug.quorum_insert_with_drop_partition ON CLUSTER one_shard_three_replicas " - "(a Int8, d Date) " - "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " - "PARTITION BY d ORDER BY a ") + create_query = "CREATE TABLE test_quorum_insert_with_drop_partition " \ + "(a Int8, d Date) " \ + "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ + "PARTITION BY d ORDER BY a " - print("Stop fetches for bug.quorum_insert_with_drop_partition at first replica.") - first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_drop_partition") + print("Create Replicated table with two replicas") + zero.query(create_query) + first.query(create_query) + second.query(create_query) + + print("Stop fetches for test_quorum_insert_with_drop_partition at first replica.") + first.query("SYSTEM STOP FETCHES test_quorum_insert_with_drop_partition") print("Insert with quorum. (zero and second)") - zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_drop_partition(a,d) VALUES(1, '2011-01-01')") + zero.query("INSERT INTO test_quorum_insert_with_drop_partition(a,d) VALUES(1, '2011-01-01')") print("Drop partition.") - zero.query_and_get_error("ALTER TABLE bug.quorum_insert_with_drop_partition DROP PARTITION '2011-01-01'") + zero.query("ALTER TABLE test_quorum_insert_with_drop_partition DROP PARTITION '2011-01-01'") - print("Insert to deleted partition") - zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_drop_partition(a,d) VALUES(2, '2011-01-01')") + if (add_new_data): + print("Insert to deleted partition") + zero.query("INSERT INTO test_quorum_insert_with_drop_partition(a,d) VALUES(2, '2011-01-01')") - print("Sync other replica from quorum.") - second.query("SYSTEM SYNC REPLICA bug.quorum_insert_with_drop_partition") + print("Resume fetches for test_quorum_insert_with_drop_partition at first replica.") + first.query("SYSTEM START FETCHES test_quorum_insert_with_drop_partition") + + print("Sync first replica with others.") + first.query("SYSTEM SYNC REPLICA test_quorum_insert_with_drop_partition") + + assert "20110101" not in first.query("SELECT * FROM system.zookeeper " \ + "where path='/clickhouse/tables/0/test_quorum_insert_with_drop_partition/quorum/last_part' " \ + "format Vertical") print("Select from updated partition.") - assert "2 2011-01-01\n" == zero.query("SELECT * FROM bug.quorum_insert_with_drop_partition") - assert "2 2011-01-01\n" == second.query("SELECT * FROM bug.quorum_insert_with_drop_partition") + if (add_new_data): + assert TSV("2\t2011-01-01\n") == TSV(zero.query("SELECT * FROM test_quorum_insert_with_drop_partition")) + assert TSV("2\t2011-01-01\n") == TSV(second.query("SELECT * FROM test_quorum_insert_with_drop_partition")) + else: + assert TSV("") == TSV(zero.query("SELECT * FROM test_quorum_insert_with_drop_partition")) + assert TSV("") == TSV(second.query("SELECT * FROM test_quorum_insert_with_drop_partition")) - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_three_replicas") + execute_on_all_cluster("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition") def test_insert_quorum_with_ttl(started_cluster): - zero = cluster.instances['s0_0_0'] - first = cluster.instances['s0_0_1'] + execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_ttl") - zero.query("DROP DATABASE IF EXISTS bug ON CLUSTER one_shard_two_replicas") - zero.query("CREATE DATABASE IF NOT EXISTS bug ON CLUSTER one_shard_two_replicas") + create_query = "CREATE TABLE test_insert_quorum_with_ttl " \ + "(a Int8, d Date) " \ + "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " \ + "PARTITION BY d ORDER BY a " \ + "TTL d + INTERVAL 5 second " \ + "SETTINGS merge_with_ttl_timeout=2 " - zero.query("CREATE TABLE bug.quorum_insert_with_ttl ON CLUSTER one_shard_two_replicas " - "(a Int8, d Date) " - "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " - "PARTITION BY d ORDER BY a " - "TTL d + INTERVAL 5 second " - "SETTINGS merge_with_ttl_timeout=2 ") + print("Create Replicated table with two replicas") + zero.query(create_query) + first.query(create_query) - print("Stop fetches for bug.quorum_insert_with_ttl at first replica.") - first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_ttl") + print("Stop fetches for test_insert_quorum_with_ttl at first replica.") + first.query("SYSTEM STOP FETCHES test_insert_quorum_with_ttl") print("Insert should fail since it can not reach the quorum.") - quorum_timeout = zero.query_and_get_error("INSERT INTO bug.quorum_insert_with_ttl(a,d) VALUES(6, now())") + quorum_timeout = zero.query_and_get_error("INSERT INTO test_insert_quorum_with_ttl(a,d) VALUES(1, '2011-01-01')", + settings={'insert_quorum_timeout' : 5000}) assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." - print("Wait 10 seconds and the data should be dropped by TTL.") - time.sleep(10) - count = zero.query("SELECT count() FROM bug.quorum_insert_with_ttl WHERE a=6") - assert count == "0\n", "Data have to be dropped by TTL" + print(zero.query("SELECT * FROM system.parts format Vertical")) - print("Resume fetches for bug.quorum_test_with_ttl at first replica.") - first.query("SYSTEM STOP FETCHES bug.quorum_insert_with_ttl") - time.sleep(5) + print("Wait 10 seconds and TTL merge have to be executed. But it won't delete data.") + time.sleep(10) + assert TSV("1\t2011-01-01\n") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency' : 0})) + + print("Resume fetches for test_insert_quorum_with_ttl at first replica.") + first.query("SYSTEM START FETCHES test_insert_quorum_with_ttl") + + print("Sync first replica.") + first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_ttl") + + + print(first.query("SELECT * from system.replicas format Vertical")) + print(first.query("SELECT * from system.zookeeper where path='/clickhouse/tables/test_insert_quorum_with_ttl/quorum' format Vertical")) + + zero.query("INSERT INTO test_insert_quorum_with_ttl(a,d) VALUES(1, '2011-01-01')", + settings={'insert_quorum_timeout' : 5000}) + + + assert TSV("1\t2011-01-01\n") == TSV(first.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency' : 0})) + assert TSV("1\t2011-01-01\n") == TSV(first.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency' : 1})) print("Inserts should resume.") - zero.query("INSERT INTO bug.quorum_insert_with_ttl(a) VALUES(6)") + zero.query("INSERT INTO test_insert_quorum_with_ttl(a, d) VALUES(2, '2012-02-02')") + + execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_ttl") From f0edb65e3338c31851db84275e143626b17e07a8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 11 Apr 2020 02:02:15 +0300 Subject: [PATCH 182/743] add test --- src/Common/rename.cpp | 4 +- src/Databases/DatabaseAtomic.cpp | 27 ++++++-- src/Databases/DatabaseAtomic.h | 2 + src/Interpreters/DatabaseCatalog.cpp | 11 +++- src/Interpreters/InterpreterDropQuery.cpp | 5 ++ src/Interpreters/InterpreterSystemQuery.cpp | 2 +- .../01107_atomic_db_detach_attach.reference | 2 + .../01107_atomic_db_detach_attach.sh | 9 ++- .../0_stateless/01109_exchange_tables.sql | 8 ++- .../01114_database_atomic.reference | 16 +++++ .../0_stateless/01114_database_atomic.sh | 63 +++++++++++++++++++ 11 files changed, 135 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/01114_database_atomic.reference create mode 100755 tests/queries/0_stateless/01114_database_atomic.sh diff --git a/src/Common/rename.cpp b/src/Common/rename.cpp index 76e24db3954..d81476b5f7d 100644 --- a/src/Common/rename.cpp +++ b/src/Common/rename.cpp @@ -29,7 +29,7 @@ static bool supportsRenameat2Impl() if (uname(&sysinfo)) return false; char * point = nullptr; - long v_major = strtol(sysinfo.release, &point, 10); + auto v_major = strtol(sysinfo.release, &point, 10); errno = 0; if (errno || *point != '.' || v_major < 3) @@ -38,7 +38,7 @@ static bool supportsRenameat2Impl() return true; errno = 0; - long v_minor = strtol(point + 1, nullptr, 10); + auto v_minor = strtol(point + 1, nullptr, 10); return !errno && 15 <= v_minor; #else return false; diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 66183f450fb..847aab84308 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes extern const int TABLE_ALREADY_EXISTS; extern const int FILE_DOESNT_EXIST; extern const int CANNOT_ASSIGN_ALTER; + extern const int DATABASE_NOT_EMPTY; } class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator @@ -117,23 +118,23 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n String old_metadata_path = getObjectMetadataPath(table_name); String new_metadata_path = to_database.getObjectMetadataPath(to_table_name); - auto detach = [this](DatabaseAtomic & db, const String & table_name_) + auto detach = [](DatabaseAtomic & db, const String & table_name_) { auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second; db.tables.erase(table_name_); db.table_name_to_path.erase(table_name_); - tryRemoveSymlink(table_name_); + db.tryRemoveSymlink(table_name_); return table_data_path_saved; }; - auto attach = [this](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_) + auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_) { db.tables.emplace(table_name_, table_); db.table_name_to_path.emplace(table_name_, table_data_path_); - tryCreateSymlink(table_name_, table_data_path_); + db.tryCreateSymlink(table_name_, table_data_path_); }; - auto assertCanMoveMatView = [&](const StoragePtr & table_) + auto assertCanMoveMatView = [inside_database](const StoragePtr & table_) { if (inside_database) return; @@ -256,6 +257,22 @@ DatabaseAtomic::DetachedTables DatabaseAtomic::cleenupDetachedTables() return not_in_use; } +void DatabaseAtomic::assertCanBeDetached(bool cleenup) +{ + if (cleenup) + { + DetachedTables not_in_use; + { + std::lock_guard lock(mutex); + not_in_use = cleenupDetachedTables(); + } + } + std::lock_guard lock(mutex); + if (!detached_tables.empty()) + throw Exception("Database " + backQuoteIfNeed(database_name) + " cannot be detached, " + "because some tables are still in use. Retry later.", ErrorCodes::DATABASE_NOT_EMPTY); +} + DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const IDatabase::FilterByNameFunction & filter_by_table_name) { auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(filter_by_table_name); diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 73a4c0166a1..714976d18b9 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -38,6 +38,8 @@ public: void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; + void assertCanBeDetached(bool cleenup); + private: void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 69b57e97c6e..8ab5e22225a 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -262,8 +262,15 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d assertDatabaseExistsUnlocked(database_name); db = databases.find(database_name)->second; - if (check_empty && !db->empty(*global_context)) - throw Exception("New table appeared in database being dropped or detached. Try again.", ErrorCodes::DATABASE_NOT_EMPTY); + if (check_empty) + { + if (!db->empty(*global_context)) + throw Exception("New table appeared in database being dropped or detached. Try again.", + ErrorCodes::DATABASE_NOT_EMPTY); + auto database_atomic = typeid_cast(db.get()); + if (!drop && database_atomic) + database_atomic->assertCanBeDetached(false); + } databases.erase(database_name); } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ed0b4e45b69..a2908c781cd 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -234,6 +235,10 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS executeToDictionary(database_name, current_dictionary, kind, false, false, false); } + auto database_atomic = typeid_cast(database.get()); + if (!drop && database_atomic) + database_atomic->assertCanBeDetached(true); + /// DETACH or DROP database itself DatabaseCatalog::instance().detachDatabase(database_name, drop); } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 2ae73146bee..06ec1c73bbe 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -367,7 +367,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) { if (dynamic_cast(iterator->table().get())) - replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name(), iterator->uuid()}); + replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()}); } } diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.reference b/tests/queries/0_stateless/01107_atomic_db_detach_attach.reference index a08773dab53..bf34eef313a 100644 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.reference +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.reference @@ -1,2 +1,4 @@ OK +OK +5 10 5 10 diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index e0b46c4e7b7..87976cec96a 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -7,14 +7,19 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" & +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & sleep 1 $CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" $CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" 2>&1 | grep -F "Code: 219" > /dev/null && echo "OK" -sleep 5 +wait $CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" +$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" +$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107 ENGINE=Atomic" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(100)" & $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" diff --git a/tests/queries/0_stateless/01109_exchange_tables.sql b/tests/queries/0_stateless/01109_exchange_tables.sql index 5114aa7e347..7fbb36e8ce9 100644 --- a/tests/queries/0_stateless/01109_exchange_tables.sql +++ b/tests/queries/0_stateless/01109_exchange_tables.sql @@ -3,11 +3,15 @@ CREATE DATABASE test_01109 ENGINE=Atomic; USE test_01109; -CREATE TABLE t1 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables']))); -CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t1), * FROM (SELECT arrayJoin(['hello', 'world'])); +CREATE TABLE t0 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables']))); +CREATE TABLE t1 ENGINE=Log() AS SELECT * FROM system.tables AS t JOIN system.databases AS d ON t.database=d.name; +CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t0), * FROM (SELECT arrayJoin(['hello', 'world'])); EXCHANGE TABLES t1 AND t3; -- { serverError 60 } EXCHANGE TABLES t4 AND t2; -- { serverError 60 } +RENAME TABLE t0 TO t1; -- { serverError 57 } +DROP TABLE t1; +RENAME TABLE t0 TO t1; SELECT * FROM t1; SELECT * FROM t2; diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference new file mode 100644 index 00000000000..3cef8bd8bc1 --- /dev/null +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -0,0 +1,16 @@ +CREATE DATABASE test_01114_1\nENGINE = Atomic +CREATE DATABASE test_01114_2\nENGINE = Atomic +CREATE DATABASE test_01114_3\nENGINE = Ordinary +test_01114_1 Atomic store metadata test_01114_1 +test_01114_2 Atomic store metadata test_01114_2 +test_01114_3 Ordinary test_01114_3 metadata test_01114_3 +20 +100 +CREATE TABLE test_01114_2.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +mt 00001114-0000-4000-8000-000000000001 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000001\' (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 +20 +CREATE TABLE test_01114_1.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01114_2.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +5 +20 190 +30 435 diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh new file mode 100755 index 00000000000..26feda6b5b8 --- /dev/null +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -0,0 +1,63 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_1" +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2" + + +$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --default_database_engine=Atomic --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_2" +$CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01114_3" + +$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_1" +$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_2" +$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_3" +$CLICKHOUSE_CLIENT -q "SELECT name, engine, splitByChar('/', data_path)[-2], splitByChar('/', metadata_path)[-3], splitByChar('/', metadata_path)[-2] FROM system.databases WHERE name LIKE 'test_01114_%'" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_1.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_1.mt_tmp SELECT * FROM numbers(100)" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_3.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_3.mt SELECT * FROM numbers(20)" + +$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt_tmp TO test_01114_3.mt_tmp" # move from Atomic to Ordinary +$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_3.mt TO test_01114_1.mt" # move from Ordinary to Atomic +$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" +$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_3.mt_tmp" + +$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_3" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '00001114-0000-4000-8000-000000000001' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" +$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_2.mt" +$CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" + + +$CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1) AS col FROM test_01114_1.mt)" & # 20s, result: 20, 190 +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1) FROM numbers(30)" & # 30s +sleep 1 # SELECT and INSERT should start before the following RENAMEs + +$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp" +$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt_tmp TO test_01114_2.mt_tmp" +$CLICKHOUSE_CLIENT -q "EXCHANGE TABLES test_01114_2.mt AND test_01114_2.mt_tmp" +$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_2.mt_tmp TO test_01114_1.mt" +$CLICKHOUSE_CLIENT -q "EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt" + +# Check that nothing changed +$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" +$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_1.mt" +$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_2.mt" + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_01114_1.mt" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_1.mt (s String) ENGINE=Log()" +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5)" +$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" # result: 5 + +$CLICKHOUSE_CLIENT -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s +sleep 1 +$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" + +wait # for INSERT + +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01114_2.mt" # result: 30, 435 +$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_2" From 8b6c8367201ff0cc55caf37db038aab0d9c26cc5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 11 Apr 2020 18:38:41 +0300 Subject: [PATCH 183/743] fix --- src/Databases/DatabaseAtomic.cpp | 8 ++++---- src/Databases/DatabaseOnDisk.cpp | 1 - .../queries/0_stateless/01107_atomic_db_detach_attach.sh | 1 + tests/queries/0_stateless/01114_database_atomic.sh | 1 + 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 847aab84308..2620ce94b6d 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -15,9 +15,9 @@ namespace ErrorCodes { extern const int UNKNOWN_TABLE; extern const int TABLE_ALREADY_EXISTS; - extern const int FILE_DOESNT_EXIST; extern const int CANNOT_ASSIGN_ALTER; extern const int DATABASE_NOT_EMPTY; + extern const int NOT_IMPLEMENTED; } class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator @@ -134,7 +134,7 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n db.tryCreateSymlink(table_name_, table_data_path_); }; - auto assertCanMoveMatView = [inside_database](const StoragePtr & table_) + auto assert_can_move_mat_view = [inside_database](const StoragePtr & table_) { if (inside_database) return; @@ -165,12 +165,12 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n } StoragePtr table = getTableUnlocked(table_name); - assertCanMoveMatView(table); + assert_can_move_mat_view(table); StoragePtr other_table; if (exchange) { other_table = other_db.getTableUnlocked(to_table_name); - assertCanMoveMatView(other_table); + assert_can_move_mat_view(other_table); } if (exchange) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index a85adcf5887..0877bcf2b8d 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -36,7 +36,6 @@ namespace ErrorCodes extern const int INCORRECT_FILE_NAME; extern const int SYNTAX_ERROR; extern const int TABLE_ALREADY_EXISTS; - extern const int UNKNOWN_TABLE; extern const int DICTIONARY_ALREADY_EXISTS; extern const int EMPTY_LIST_OF_COLUMNS_PASSED; } diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index 87976cec96a..3720a8907b5 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -22,4 +22,5 @@ $CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(100)" & +sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 26feda6b5b8..6754a79beda 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -5,6 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_1" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2" +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_3" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" From 82ef20d6dce2ddac4a40c956a43d0dbed5f76bcd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 12 Apr 2020 17:17:34 +0300 Subject: [PATCH 184/743] Fix build. --- src/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cb8172993a5..6549105318e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -311,7 +311,7 @@ add_object_library(clickhouse_processors_formats Processors/Formats) add_object_library(clickhouse_processors_formats_impl Processors/Formats/Impl) add_object_library(clickhouse_processors_transforms Processors/Transforms) add_object_library(clickhouse_processors_sources Processors/Sources) -add_object_library(clickhouse_processors_sources Processors/Merges) +add_object_library(clickhouse_processors_merges Processors/Merges) if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) From 20e6593f96114e130b7f5addd2e68c29e2939ff5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 12 Apr 2020 21:52:59 +0300 Subject: [PATCH 185/743] disable by default --- src/Core/Settings.h | 4 +-- src/Interpreters/DatabaseCatalog.cpp | 29 +++++++++++-------- src/Interpreters/InterpreterCreateQuery.cpp | 7 ++--- .../InterpreterShowCreateQuery.cpp | 6 ---- src/Storages/StorageMaterializedView.cpp | 1 + .../queries/0_stateless/00116_storage_set.sql | 2 +- .../00180_attach_materialized_view.sql | 3 +- ...per_deduplication_and_unexpected_parts.sql | 2 +- .../0_stateless/00311_array_primary_key.sql | 2 +- .../00423_storage_log_single_thread.sql | 6 ++-- .../01107_atomic_db_detach_attach.sh | 4 +-- .../0_stateless/01109_exchange_tables.sql | 1 + .../01114_database_atomic.reference | 8 ++--- .../0_stateless/01114_database_atomic.sh | 11 +++---- ...1225_show_create_table_from_dictionary.sql | 2 +- .../00065_loyalty_with_storage_join.sql | 2 +- 16 files changed, 45 insertions(+), 45 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 534d53164ad..2172adec83c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -393,8 +393,8 @@ 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(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(SettingDefaultDatabaseEngine, default_database_engine, /*DefaultDatabaseEngine::Ordinary*/ DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \ - M(SettingBool, allow_experimental_database_atomic, /*false*/ true, "Allow to create database with Engine=Atomic.", 0) \ + M(SettingDefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ + M(SettingBool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 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) \ M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(SettingUInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8ab5e22225a..99c7201993e 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -10,8 +10,6 @@ #include #include #include -#include -#include #include #include @@ -501,18 +499,25 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() continue; /// Process .sql files with metadata of tables which were marked as dropped - String full_path = path + it.name(); - - Strings name_parts; - boost::split(name_parts, it.name(), boost::is_any_of(".")); // NOLINT: LLVM Bug 41141 - if (name_parts.size() != 4) /// Unexpected file - continue; - StorageID dropped_id = StorageID::createEmpty(); - dropped_id.database_name = unescapeForFileName(name_parts[0]); - dropped_id.table_name = unescapeForFileName(name_parts[1]); - dropped_id.uuid = parse(name_parts[2]); + size_t dot_pos = it.name().find('.'); + if (dot_pos == std::string::npos) + continue; + dropped_id.database_name = unescapeForFileName(it.name().substr(0, dot_pos)); + size_t prev_dot_pos = dot_pos; + dot_pos = it.name().find('.', prev_dot_pos + 1); + if (dot_pos == std::string::npos) + continue; + dropped_id.table_name = unescapeForFileName(it.name().substr(prev_dot_pos + 1, dot_pos - prev_dot_pos - 1)); + + prev_dot_pos = dot_pos; + dot_pos = it.name().find('.', prev_dot_pos + 1); + if (dot_pos == std::string::npos) + continue; + dropped_id.uuid = parse(it.name().substr(prev_dot_pos + 1, dot_pos - prev_dot_pos - 1)); + + String full_path = path + it.name(); dropped_metadata.emplace(std::move(full_path), std::move(dropped_id)); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6df8d7d9298..a1a4654a1d6 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -102,10 +102,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) engine->name = old_style_database ? "Ordinary" : "Atomic"; storage->set(storage->engine, engine); create.set(create.storage, storage); - - if (!context.getSettingsRef().allow_experimental_database_atomic) - throw Exception("Atomic is an experimental database engine. Enable allow_experimental_database_atomic to use it.", - ErrorCodes::UNKNOWN_DATABASE_ENGINE); } else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) { @@ -115,6 +111,9 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); } + if (create.storage->engine->name == "Atomic" && !context.getSettingsRef().allow_experimental_database_atomic && !internal) + throw Exception("Atomic is an experimental database engine. Enable allow_experimental_database_atomic to use it.", + ErrorCodes::UNKNOWN_DATABASE_ENGINE); String database_name_escaped = escapeForFileName(database_name); String path = context.getPath(); diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 11d73159775..7f26666a1ae 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -14,8 +14,6 @@ #include #include -#include - namespace DB { @@ -73,10 +71,6 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (!create_query && show_query && show_query->temporary) throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY); - //FIXME temporary print create query without UUID for tests (remove it) - auto & create = create_query->as(); - create.uuid = UUIDHelpers::Nil; - std::stringstream stream; formatAST(*create_query, stream, false, false); String res = stream.str(); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index ab1c1865411..cf633e913e6 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -221,6 +221,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, drop_query->database = target_table_id.database_name; drop_query->table = target_table_id.table_name; drop_query->kind = kind; + drop_query->no_delay = true; ASTPtr ast_drop_query = drop_query; InterpreterDropQuery drop_interpreter(ast_drop_query, global_context); drop_interpreter.execute(); diff --git a/tests/queries/0_stateless/00116_storage_set.sql b/tests/queries/0_stateless/00116_storage_set.sql index 2b573c87f4a..aa93a0620d0 100644 --- a/tests/queries/0_stateless/00116_storage_set.sql +++ b/tests/queries/0_stateless/00116_storage_set.sql @@ -19,7 +19,7 @@ INSERT INTO set2 VALUES ('abc'), ('World'); SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; DETACH TABLE set2; -ATTACH TABLE set2; -- (x String) ENGINE = Set; +ATTACH TABLE set2 (x String) ENGINE = Set; SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; diff --git a/tests/queries/0_stateless/00180_attach_materialized_view.sql b/tests/queries/0_stateless/00180_attach_materialized_view.sql index 0e20b15a438..089e4926bcf 100644 --- a/tests/queries/0_stateless/00180_attach_materialized_view.sql +++ b/tests/queries/0_stateless/00180_attach_materialized_view.sql @@ -6,8 +6,7 @@ CREATE TABLE t_00180 (x UInt8) ENGINE = Null; CREATE MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; DETACH TABLE mv_00180; ---ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; -ATTACH TABLE mv_00180; +ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; DROP TABLE t_00180; DROP TABLE mv_00180; diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index 1abe7d54610..6b29d0a8cd3 100644 --- a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1); SELECT * FROM deduplication; DETACH TABLE deduplication; -ATTACH TABLE deduplication;-- (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); +ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); SELECT * FROM deduplication; diff --git a/tests/queries/0_stateless/00311_array_primary_key.sql b/tests/queries/0_stateless/00311_array_primary_key.sql index 0598583c411..0ea368609da 100644 --- a/tests/queries/0_stateless/00311_array_primary_key.sql +++ b/tests/queries/0_stateless/00311_array_primary_key.sql @@ -11,7 +11,7 @@ INSERT INTO array_pk VALUES ([5, 6], 'ghi', 6); SELECT * FROM array_pk ORDER BY n; DETACH TABLE array_pk; -ATTACH TABLE array_pk; -- (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +ATTACH TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); SELECT * FROM array_pk ORDER BY n; diff --git a/tests/queries/0_stateless/00423_storage_log_single_thread.sql b/tests/queries/0_stateless/00423_storage_log_single_thread.sql index 512118dd55b..7d5e14c9ee5 100644 --- a/tests/queries/0_stateless/00423_storage_log_single_thread.sql +++ b/tests/queries/0_stateless/00423_storage_log_single_thread.sql @@ -5,7 +5,7 @@ SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log; -- (s String) ENGINE = Log; +ATTACH TABLE log (s String) ENGINE = Log; SELECT * FROM log; SELECT * FROM log LIMIT 1; @@ -15,13 +15,13 @@ INSERT INTO log VALUES ('Hello'), ('World'); SELECT * FROM log LIMIT 1; DETACH TABLE log; -ATTACH TABLE log; -- (s String) ENGINE = Log; +ATTACH TABLE log (s String) ENGINE = Log; SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log; -- (s String) ENGINE = Log; +ATTACH TABLE log (s String) ENGINE = Log; SELECT * FROM log; SELECT * FROM log LIMIT 1; diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index 3720a8907b5..6ff36318d30 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & @@ -18,7 +18,7 @@ wait $CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" $CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" -$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "ATTACH DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(100)" & diff --git a/tests/queries/0_stateless/01109_exchange_tables.sql b/tests/queries/0_stateless/01109_exchange_tables.sql index 7fbb36e8ce9..7125bfea851 100644 --- a/tests/queries/0_stateless/01109_exchange_tables.sql +++ b/tests/queries/0_stateless/01109_exchange_tables.sql @@ -1,4 +1,5 @@ DROP DATABASE IF EXISTS test_01109; +SET allow_experimental_database_atomic=1; CREATE DATABASE test_01109 ENGINE=Atomic; USE test_01109; diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index 3cef8bd8bc1..a6c9fc3c4f4 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -6,11 +6,11 @@ test_01114_2 Atomic store metadata test_01114_2 test_01114_3 Ordinary test_01114_3 metadata test_01114_3 20 100 -CREATE TABLE test_01114_2.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -mt 00001114-0000-4000-8000-000000000001 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000001\' (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 +CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\' (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 20 -CREATE TABLE test_01114_1.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01114_2.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01114_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 5 20 190 30 435 diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 6754a79beda..9cd624ad4b0 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -8,7 +8,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_3" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" $CLICKHOUSE_CLIENT --default_database_engine=Atomic --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_2" $CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01114_3" @@ -29,13 +29,13 @@ $CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_3.mt_tmp" $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_3" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '00001114-0000-4000-8000-000000000001' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '00001114-0000-4000-8000-000000000002' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" $CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_2.mt" $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" -$CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1) AS col FROM test_01114_1.mt)" & # 20s, result: 20, 190 -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1) FROM numbers(30)" & # 30s +$CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 30s, result: 20, 190 +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s sleep 1 # SELECT and INSERT should start before the following RENAMEs $CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp" @@ -46,7 +46,8 @@ $CLICKHOUSE_CLIENT -q "EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt" # Check that nothing changed $CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" -$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_1.mt" +uuid_mt1=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database='test_01114_1' AND name='mt'"` +$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_1.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g" $CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_2.mt" $CLICKHOUSE_CLIENT -q "DROP TABLE test_01114_1.mt" diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index 7550d5292d0..6c1ae2b907b 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -15,7 +15,7 @@ LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.dict` FORMAT TSVRaw; -SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 36; } +SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 60; } DROP DATABASE dict_db_01225; DROP DATABASE dict_db_01225_dictionary; diff --git a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index b03013c11fe..d3e73faa7be 100644 --- a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -24,7 +24,7 @@ GROUP BY loyalty ORDER BY loyalty ASC; DETACH TABLE join; -ATTACH TABLE join;-- (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); +ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); SELECT loyalty, From b9931863eff3b528109b89d94a555ed81575cc07 Mon Sep 17 00:00:00 2001 From: Avogar Date: Mon, 13 Apr 2020 00:01:17 +0300 Subject: [PATCH 186/743] Fix FixedString packing --- src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index 7c5e2c5b522..cef7b001505 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -85,7 +85,6 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr packer.pack_uint64(assert_cast(column).getElement(row_num)); return; } - case TypeIndex::FixedString: [[fallthrough]]; case TypeIndex::String: { const StringRef & string = assert_cast(column).getDataAt(row_num); @@ -93,6 +92,13 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr packer.pack_str_body(string.data, string.size); return; } + case TypeIndex::FixedString: + { + const StringRef & string = assert_cast(column).getDataAt(row_num); + packer.pack_str(string.size); + packer.pack_str_body(string.data, string.size); + return; + } case TypeIndex::Array: { auto nested_type = assert_cast(*data_type).getNestedType(); From d49dc5c008cbd3802dd35066f4607e5f3e21fde9 Mon Sep 17 00:00:00 2001 From: Avogar Date: Mon, 13 Apr 2020 00:16:27 +0300 Subject: [PATCH 187/743] Add test --- tests/queries/0_stateless/01098_msgpack_format.reference | 3 +++ tests/queries/0_stateless/01098_msgpack_format.sh | 8 ++++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/01098_msgpack_format.reference b/tests/queries/0_stateless/01098_msgpack_format.reference index 8059526a38f..ad116a5ba91 100644 --- a/tests/queries/0_stateless/01098_msgpack_format.reference +++ b/tests/queries/0_stateless/01098_msgpack_format.reference @@ -8,3 +8,6 @@ [[1,2,3],[1001,2002],[3167]] [[['one'],['two']],[['three']],[['four'],['five']]] [0,1,2,3,42,253,254,255] [255,254,253,42,3,2,1,0] +2020-01-01 +2020-01-02 +2020-01-02 diff --git a/tests/queries/0_stateless/01098_msgpack_format.sh b/tests/queries/0_stateless/01098_msgpack_format.sh index afebd6de3dc..233399570bb 100755 --- a/tests/queries/0_stateless/01098_msgpack_format.sh +++ b/tests/queries/0_stateless/01098_msgpack_format.sh @@ -52,3 +52,11 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM msgpack"; $CLICKHOUSE_CLIENT --query="DROP TABLE msgpack"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE msgpack (date FixedString(10)) ENGINE = Memory"; + +$CLICKHOUSE_CLIENT --query="INSERT INTO msgpack VALUES ('2020-01-01'), ('2020-01-02'), ('2020-01-02')"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM msgpack"; + +$CLICKHOUSE_CLIENT --query="DROP TABLE msgpack"; + From 9860ffee5189189b7285dc5641c92c35bae49591 Mon Sep 17 00:00:00 2001 From: Avogar Date: Mon, 13 Apr 2020 00:59:28 +0300 Subject: [PATCH 188/743] Add MsgPack performance test --- tests/performance/parse_engine_file.xml | 1 + tests/performance/select_format.xml | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/performance/parse_engine_file.xml b/tests/performance/parse_engine_file.xml index fb10fa97915..c96f4e537ff 100644 --- a/tests/performance/parse_engine_file.xml +++ b/tests/performance/parse_engine_file.xml @@ -34,6 +34,7 @@ RowBinary Native Avro + MsgPack diff --git a/tests/performance/select_format.xml b/tests/performance/select_format.xml index b8df874304f..e47d981c4d7 100644 --- a/tests/performance/select_format.xml +++ b/tests/performance/select_format.xml @@ -44,6 +44,7 @@ ODBCDriver2 MySQLWire Avro + MsgPack From 46c0b65c9542d41c80349c9c33b8bc6ddcad075e Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 13 Apr 2020 12:20:38 +0300 Subject: [PATCH 189/743] Minor fix in MergeTree over S3 tests. --- tests/integration/test_merge_tree_s3/test.py | 2 +- tests/integration/test_replicated_merge_tree_s3/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 4beb33604be..50cf532e9a4 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -67,7 +67,7 @@ def create_table(cluster, table_name, additional_settings=None): PARTITION BY dt ORDER BY (dt, id) SETTINGS - storage_policy = 's3', + storage_policy='s3', old_parts_lifetime=0, index_granularity=512 """.format(table_name) diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 8689e7ccf5d..d6b6015a388 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -67,7 +67,7 @@ def create_table(cluster): ) ENGINE=ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/s3', '{instance}') PARTITION BY dt ORDER BY (dt, id) - SETTINGS storage_policy = 's3' + SETTINGS storage_policy='s3' """ for node in cluster.instances.values(): From 03dad5ccf5258b9e2fe5f0bb58da6ace17e97617 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 14:45:59 +0300 Subject: [PATCH 190/743] Added AggregatingSortedAlgorithm. --- ...orm.cpp => AggregatingSortedAlgorithm.cpp} | 112 +++-------- .../Merges/AggregatingSortedAlgorithm.h | 176 ++++++++++++++++++ .../Merges/AggregatingSortedTransform.h | 164 +--------------- src/Processors/Merges/IMergingAlgorithm.h | 3 +- .../IMergingAlgorithmWithDelayedChunk.cpp | 50 +++++ .../IMergingAlgorithmWithDelayedChunk.h | 41 ++++ 6 files changed, 307 insertions(+), 239 deletions(-) rename src/Processors/Merges/{AggregatingSortedTransform.cpp => AggregatingSortedAlgorithm.cpp} (68%) create mode 100644 src/Processors/Merges/AggregatingSortedAlgorithm.h create mode 100644 src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp create mode 100644 src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h diff --git a/src/Processors/Merges/AggregatingSortedTransform.cpp b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp similarity index 68% rename from src/Processors/Merges/AggregatingSortedTransform.cpp rename to src/Processors/Merges/AggregatingSortedAlgorithm.cpp index fe86d01b390..a185e7cee67 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.cpp +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -8,17 +8,12 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace { - AggregatingSortedTransform::ColumnsDefinition defineColumns( + AggregatingSortedAlgorithm::ColumnsDefinition defineColumns( const Block & header, const SortDescription & description) { - AggregatingSortedTransform::ColumnsDefinition def = {}; + AggregatingSortedAlgorithm::ColumnsDefinition def = {}; size_t num_columns = header.columns(); /// Fill in the column numbers that need to be aggregated. @@ -53,7 +48,7 @@ namespace type = nullptr; // simple aggregate function - AggregatingSortedTransform::SimpleAggregateDescription desc(simple_aggr->getFunction(), i, type); + AggregatingSortedAlgorithm::SimpleAggregateDescription desc(simple_aggr->getFunction(), i, type); if (desc.function->allocatesMemoryInArena()) def.allocates_memory_in_arena = true; @@ -69,7 +64,7 @@ namespace return def; } - MutableColumns getMergedColumns(const Block & header, const AggregatingSortedTransform::ColumnsDefinition & def) + MutableColumns getMergedColumns(const Block & header, const AggregatingSortedAlgorithm::ColumnsDefinition & def) { MutableColumns columns; columns.resize(header.columns()); @@ -88,34 +83,17 @@ namespace } } -AggregatingSortedTransform::AggregatingSortedTransform( - const Block & header, size_t num_inputs, +AggregatingSortedAlgorithm::AggregatingSortedAlgorithm( + const Block & header_, size_t num_inputs, SortDescription description_, size_t max_block_size) - : IMergingTransform(num_inputs, header, header, true) + : IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_)) + , header(header_) , columns_definition(defineColumns(header, description_)) - , merged_data(getMergedColumns(header, columns_definition), false, max_block_size) - , description(std::move(description_)) - , source_chunks(num_inputs) - , cursors(num_inputs) + , merged_data(getMergedColumns(header, columns_definition), max_block_size, columns_definition) { - merged_data.initAggregateDescription(columns_definition); } -void AggregatingSortedTransform::initializeInputs() -{ - queue = SortingHeap(cursors); - is_queue_initialized = true; -} - -void AggregatingSortedTransform::consume(Chunk chunk, size_t input_number) -{ - updateCursor(std::move(chunk), input_number); - - if (is_queue_initialized) - queue.push(cursors[input_number]); -} - -void AggregatingSortedTransform::updateCursor(Chunk chunk, size_t source_num) +void AggregatingSortedAlgorithm::prepareChunk(Chunk & chunk) const { auto num_rows = chunk.getNumRows(); auto columns = chunk.detachColumns(); @@ -128,56 +106,24 @@ void AggregatingSortedTransform::updateCursor(Chunk chunk, size_t source_num) columns[desc.column_number] = recursiveRemoveLowCardinality(columns[desc.column_number]); chunk.setColumns(std::move(columns), num_rows); - - auto & source_chunk = source_chunks[source_num]; - - if (source_chunk) - { - /// Extend lifetime of last chunk. - last_chunk = std::move(source_chunk); - last_chunk_sort_columns = std::move(cursors[source_num].sort_columns); - - source_chunk = std::move(chunk); - cursors[source_num].reset(source_chunk.getColumns(), {}); - } - else - { - if (cursors[source_num].has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - source_chunk = std::move(chunk); - cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num); - } } -void AggregatingSortedTransform::work() +void AggregatingSortedAlgorithm::initialize(Chunks chunks) { - merge(); - prepareOutputChunk(merged_data); + for (auto & chunk : chunks) + if (chunk) + prepareChunk(chunk); - if (has_output_chunk) - { - size_t num_rows = output_chunk.getNumRows(); - auto columns = output_chunk.detachColumns(); - auto & header = getOutputs().back().getHeader(); - - for (auto & desc : columns_definition.columns_to_simple_aggregate) - { - if (desc.inner_type) - { - auto & from_type = desc.inner_type; - auto & to_type = header.getByPosition(desc.column_number).type; - columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type); - } - } - - output_chunk.setColumns(std::move(columns), num_rows); - - merged_data.initAggregateDescription(columns_definition); - } + initializeQueue(std::move(chunks)); } -void AggregatingSortedTransform::merge() +void AggregatingSortedAlgorithm::consume(Chunk chunk, size_t source_num) +{ + prepareChunk(chunk); + updateCursor(std::move(chunk), source_num); +} + +IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge() { /// We take the rows in the correct order and put them in `merged_block`, while the rows are no more than `max_block_size` while (queue.isValid()) @@ -213,7 +159,7 @@ void AggregatingSortedTransform::merge() if (merged_data.hasEnoughRows()) { last_key.reset(); - return; + Status(merged_data.pull(columns_definition, header)); } /// We will write the data for the group. We copy the values of ordinary columns. @@ -242,8 +188,7 @@ void AggregatingSortedTransform::merge() { /// We get the next block from the corresponding source, if there is one. queue.removeTop(); - requestDataForInput(current.impl->order); - return; + return Status(current.impl->order); } } @@ -255,10 +200,10 @@ void AggregatingSortedTransform::merge() } last_chunk_sort_columns.clear(); - is_finished = true; + return Status(merged_data.pull(columns_definition, header), true); } -void AggregatingSortedTransform::addRow(SortCursor & cursor) +void AggregatingSortedAlgorithm::addRow(SortCursor & cursor) { for (auto & desc : columns_definition.columns_to_aggregate) desc.column->insertMergeFrom(*cursor->all_columns[desc.column_number], cursor->pos); @@ -270,7 +215,7 @@ void AggregatingSortedTransform::addRow(SortCursor & cursor) } } -void AggregatingSortedTransform::insertSimpleAggregationResult() +void AggregatingSortedAlgorithm::insertSimpleAggregationResult() { for (auto & desc : columns_definition.columns_to_simple_aggregate) { @@ -279,4 +224,5 @@ void AggregatingSortedTransform::insertSimpleAggregationResult() } } + } diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/AggregatingSortedAlgorithm.h new file mode 100644 index 00000000000..95a12c49618 --- /dev/null +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.h @@ -0,0 +1,176 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class ColumnAggregateFunction; + +class AggregatingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChunk +{ +public: + AggregatingSortedAlgorithm( + const Block & header, size_t num_inputs, + SortDescription description_, size_t max_block_size); + + void initialize(Chunks chunks) override; + void consume(Chunk chunk, size_t source_num) override; + Status merge() override; + + struct SimpleAggregateDescription; + + struct ColumnsDefinition + { + struct AggregateDescription + { + ColumnAggregateFunction * column = nullptr; + const size_t column_number = 0; + + AggregateDescription() = default; + explicit AggregateDescription(size_t col_number) : column_number(col_number) {} + }; + + /// Columns with which numbers should not be aggregated. + ColumnNumbers column_numbers_not_to_aggregate; + std::vector columns_to_aggregate; + std::vector columns_to_simple_aggregate; + + /// Does SimpleAggregateFunction allocates memory in arena? + bool allocates_memory_in_arena = false; + }; + +private: + /// Specialization for AggregatingSortedAlgorithm. + struct AggregatingMergedData : public MergedData + { + public: + AggregatingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def) + : MergedData(std::move(columns_), false, max_block_size_) + { + initAggregateDescription(def); + } + + void initializeRow(const ColumnRawPtrs & raw_columns, size_t row, const ColumnNumbers & column_numbers) + { + for (auto column_number : column_numbers) + columns[column_number]->insertFrom(*raw_columns[column_number], row); + + is_group_started = true; + } + + bool isGroupStarted() const { return is_group_started; } + + void insertRow() + { + is_group_started = false; + ++total_merged_rows; + ++merged_rows; + /// TODO: sum_blocks_granularity += block_size; + } + + Chunk pull(ColumnsDefinition & def, const Block & header_) + { + auto chunk = pull(); + + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + for (auto & desc : def.columns_to_simple_aggregate) + { + if (desc.inner_type) + { + auto & from_type = desc.inner_type; + auto & to_type = header_.getByPosition(desc.column_number).type; + columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type); + } + } + + chunk.setColumns(std::move(columns), num_rows); + initAggregateDescription(def); + } + + private: + bool is_group_started = false; + + /// Initialize aggregate descriptions with columns. + void initAggregateDescription(ColumnsDefinition & def) + { + for (auto & desc : def.columns_to_simple_aggregate) + desc.column = columns[desc.column_number].get(); + + for (auto & desc : def.columns_to_aggregate) + desc.column = typeid_cast(columns[desc.column_number].get()); + } + + using MergedData::pull; + }; + + Block header; + + ColumnsDefinition columns_definition; + AggregatingMergedData merged_data; + + /// Memory pool for SimpleAggregateFunction + /// (only when allocates_memory_in_arena == true). + std::unique_ptr arena; + + void prepareChunk(Chunk & chunk) const; + void addRow(SortCursor & cursor); + void insertSimpleAggregationResult(); + +public: + /// Stores information for aggregation of SimpleAggregateFunction columns + struct SimpleAggregateDescription + { + /// An aggregate function 'anyLast', 'sum'... + AggregateFunctionPtr function; + IAggregateFunction::AddFunc add_function = nullptr; + + size_t column_number = 0; + IColumn * column = nullptr; + const DataTypePtr inner_type; + + AlignedBuffer state; + bool created = false; + + SimpleAggregateDescription(AggregateFunctionPtr function_, const size_t column_number_, DataTypePtr type) + : function(std::move(function_)), column_number(column_number_), inner_type(std::move(type)) + { + add_function = function->getAddressOfAddFunction(); + state.reset(function->sizeOfData(), function->alignOfData()); + } + + void createState() + { + if (created) + return; + function->create(state.data()); + created = true; + } + + void destroyState() + { + if (!created) + return; + function->destroy(state.data()); + created = false; + } + + /// Explicitly destroy aggregation state if the stream is terminated + ~SimpleAggregateDescription() + { + destroyState(); + } + + SimpleAggregateDescription() = default; + SimpleAggregateDescription(SimpleAggregateDescription &&) = default; + SimpleAggregateDescription(const SimpleAggregateDescription &) = delete; + }; +}; + +} diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index 279ac9c7597..89e22173a13 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -1,15 +1,7 @@ #pragma once #include -#include -#include - -#include -#include -#include -#include -#include -#include +#include namespace DB { @@ -22,158 +14,22 @@ class ColumnAggregateFunction; * corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type, * when merged, the first value is selected. */ -class AggregatingSortedTransform : public IMergingTransform +class AggregatingSortedTransform : public IMergingTransform2 { public: AggregatingSortedTransform( const Block & header, size_t num_inputs, - SortDescription description_, size_t max_block_size); - - struct SimpleAggregateDescription; - - struct ColumnsDefinition + SortDescription description_, size_t max_block_size) + : IMergingTransform2( + num_inputs, header, header, true, + header, + num_inputs, + std::move(description_), + max_block_size) { - struct AggregateDescription - { - ColumnAggregateFunction * column = nullptr; - const size_t column_number = 0; - - AggregateDescription() = default; - explicit AggregateDescription(size_t col_number) : column_number(col_number) {} - }; - - /// Columns with which numbers should not be aggregated. - ColumnNumbers column_numbers_not_to_aggregate; - std::vector columns_to_aggregate; - std::vector columns_to_simple_aggregate; - - /// Does SimpleAggregateFunction allocates memory in arena? - bool allocates_memory_in_arena = false; - }; + } String getName() const override { return "AggregatingSortedTransform"; } - void work() override; - -protected: - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; - -private: - - /// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns. - struct AggregatingMergedData : public MergedData - { - public: - using MergedData::MergedData; - - void initializeRow(const ColumnRawPtrs & raw_columns, size_t row, const ColumnNumbers & column_numbers) - { - for (auto column_number : column_numbers) - columns[column_number]->insertFrom(*raw_columns[column_number], row); - - is_group_started = true; - } - - bool isGroupStarted() const { return is_group_started; } - - void insertRow() - { - is_group_started = false; - ++total_merged_rows; - ++merged_rows; - /// TODO: sum_blocks_granularity += block_size; - } - - /// Initialize aggregate descriptions with columns. - void initAggregateDescription(ColumnsDefinition & def) - { - for (auto & desc : def.columns_to_simple_aggregate) - desc.column = columns[desc.column_number].get(); - - for (auto & desc : def.columns_to_aggregate) - desc.column = typeid_cast(columns[desc.column_number].get()); - } - private: - bool is_group_started = false; - }; - - ColumnsDefinition columns_definition; - AggregatingMergedData merged_data; - - SortDescription description; - - /// Chunks currently being merged. - std::vector source_chunks; - SortCursorImpls cursors; - - /// In merging algorithm, we need to compare current sort key with the last one. - /// So, sorting columns for last row needed to be stored. - /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). - Chunk last_chunk; - ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. - - detail::RowRef last_key; - - SortingHeap queue; - bool is_queue_initialized = false; - - /// Memory pool for SimpleAggregateFunction - /// (only when allocates_memory_in_arena == true). - std::unique_ptr arena; - - void merge(); - void updateCursor(Chunk chunk, size_t source_num); - void addRow(SortCursor & cursor); - void insertSimpleAggregationResult(); - -public: - /// Stores information for aggregation of SimpleAggregateFunction columns - struct SimpleAggregateDescription - { - /// An aggregate function 'anyLast', 'sum'... - AggregateFunctionPtr function; - IAggregateFunction::AddFunc add_function = nullptr; - - size_t column_number = 0; - IColumn * column = nullptr; - const DataTypePtr inner_type; - - AlignedBuffer state; - bool created = false; - - SimpleAggregateDescription(AggregateFunctionPtr function_, const size_t column_number_, DataTypePtr type) - : function(std::move(function_)), column_number(column_number_), inner_type(std::move(type)) - { - add_function = function->getAddressOfAddFunction(); - state.reset(function->sizeOfData(), function->alignOfData()); - } - - void createState() - { - if (created) - return; - function->create(state.data()); - created = true; - } - - void destroyState() - { - if (!created) - return; - function->destroy(state.data()); - created = false; - } - - /// Explicitly destroy aggregation state if the stream is terminated - ~SimpleAggregateDescription() - { - destroyState(); - } - - SimpleAggregateDescription() = default; - SimpleAggregateDescription(SimpleAggregateDescription &&) = default; - SimpleAggregateDescription(const SimpleAggregateDescription &) = delete; - }; }; } diff --git a/src/Processors/Merges/IMergingAlgorithm.h b/src/Processors/Merges/IMergingAlgorithm.h index edd5b7d883c..263acee4c2d 100644 --- a/src/Processors/Merges/IMergingAlgorithm.h +++ b/src/Processors/Merges/IMergingAlgorithm.h @@ -25,12 +25,11 @@ public: virtual Status merge() = 0; IMergingAlgorithm() = default; - IMergingAlgorithm(IMergingAlgorithm &&) = default; virtual ~IMergingAlgorithm() = default; }; // TODO: use when compile with clang which could support it // template -// concept MergingAlgorithm = std::is_base_of::value && std::is_move_constructible::value; +// concept MergingAlgorithm = std::is_base_of::value; } diff --git a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp new file mode 100644 index 00000000000..5b6c00750ba --- /dev/null +++ b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp @@ -0,0 +1,50 @@ +// +// Created by nik-kochetov on 4/13/20. +// + +#include + + +namespace DB +{ + +IMergingAlgorithmWithDelayedChunk::IMergingAlgorithmWithDelayedChunk( + size_t num_inputs, + SortDescription description_) + : description(std::move(description_)) + , source_chunks(num_inputs) +{ +} + + +void IMergingAlgorithmWithDelayedChunk::initializeQueue(Chunks chunks) +{ + source_chunks.resize(chunks.size()); + + for (size_t source_num = 0; source_num < source_chunks.size(); ++source_num) + { + if (!chunks[source_num]) + continue; + + source_chunks[source_num] = std::move(chunks[source_num]); + cursors[source_num] = SortCursorImpl(source_chunks[source_num].getColumns(), description, source_num); + } + + queue = SortingHeap(cursors); +} + +void IMergingAlgorithmWithDelayedChunk::updateCursor(Chunk chunk, size_t source_num) +{ + auto & source_chunk = source_chunks[source_num]; + + /// Extend lifetime of last chunk. + last_chunk = std::move(source_chunk); + last_chunk_sort_columns = std::move(cursors[source_num].sort_columns); + + source_chunk = std::move(chunk); + cursors[source_num].reset(source_chunk.getColumns(), {}); + + queue.push(cursors[source_num]); +} + +} diff --git a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h new file mode 100644 index 00000000000..2a53f22f5aa --- /dev/null +++ b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class IMergingAlgorithmWithDelayedChunk : public IMergingAlgorithm +{ +public: + IMergingAlgorithmWithDelayedChunk( + size_t num_inputs, + SortDescription description_); + +protected: + SortingHeap queue; + + /// Previous row. May refer to last_chunk_sort_columns or row from source_chunks. + detail::RowRef last_key; + + ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. + + void initializeQueue(Chunks chunks); + void updateCursor(Chunk chunk, size_t source_num); + +private: + SortDescription description; + + /// Chunks currently being merged. + std::vector source_chunks; + SortCursorImpls cursors; + + /// In merging algorithm, we need to compare current sort key with the last one. + /// So, sorting columns for last row needed to be stored. + /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). + Chunk last_chunk; +}; + +} From a6bf7a14c8eb3bc04a40ddb711cf400a90914091 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 15:02:59 +0300 Subject: [PATCH 191/743] Fix build. --- src/Processors/Merges/AggregatingSortedAlgorithm.h | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/AggregatingSortedAlgorithm.h index 95a12c49618..3327776b2f4 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.h @@ -5,12 +5,11 @@ #include #include #include +#include namespace DB { -class ColumnAggregateFunction; - class AggregatingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChunk { public: @@ -78,7 +77,7 @@ private: auto chunk = pull(); size_t num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); + auto columns_ = chunk.detachColumns(); for (auto & desc : def.columns_to_simple_aggregate) { @@ -86,12 +85,14 @@ private: { auto & from_type = desc.inner_type; auto & to_type = header_.getByPosition(desc.column_number).type; - columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type); + columns_[desc.column_number] = recursiveTypeConversion(columns_[desc.column_number], from_type, to_type); } } - chunk.setColumns(std::move(columns), num_rows); + chunk.setColumns(std::move(columns_), num_rows); initAggregateDescription(def); + + return chunk; } private: From d5c055ab03d9c608201f0736d4a46bf0edb1fcdb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Apr 2020 15:13:13 +0300 Subject: [PATCH 192/743] remove logs --- src/Storages/StorageReplicatedMergeTree.cpp | 11 ----------- tests/integration/test_polymorphic_parts/test.py | 2 -- 2 files changed, 13 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ff4d5df03b9..7548edad48a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2052,8 +2052,6 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() { - LOG_FATAL(&Poco::Logger::get("queueTask()"), "begin"); - /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) { @@ -2583,7 +2581,6 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart( */ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) { - LOG_FATAL(&Poco::Logger::get("updateQuorum"), "BEGIN!"); auto zookeeper = getZooKeeper(); /// Information on which replicas a part has been added, if the quorum has not yet been reached. @@ -3014,7 +3011,6 @@ void StorageReplicatedMergeTree::startup() void StorageReplicatedMergeTree::shutdown() { - LOG_FATAL(&Poco::Logger::get("shutdown"), "SHUTDOWN!"); clearOldPartsFromFilesystem(true); /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. fetcher.blocker.cancelForever(); @@ -5324,17 +5320,10 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti return merger_mutator.ttl_merges_blocker.cancel(); if (action_type == ActionLocks::PartsFetch) - { return fetcher.blocker.cancel(); - } - if (action_type == ActionLocks::PartsSend) - { - LOG_FATAL(&Poco::Logger::get("ActionLock"), "Cancel PartsSend"); return data_parts_exchange_endpoint ? data_parts_exchange_endpoint->blocker.cancel() : ActionLock(); - } - if (action_type == ActionLocks::ReplicationQueue) return queue.actions_blocker.cancel(); diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index fdf9ac2834b..f7256de9d9a 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -90,8 +90,6 @@ def test_polymorphic_parts_basics(start_cluster, first_node, second_node): first_node.query("SYSTEM STOP MERGES") second_node.query("SYSTEM STOP MERGES") - print(first_node.query("SELECT * FROM system.settings where name='insert_quorum' format Vertical")) - for size in [300, 300, 600]: insert_random_data('polymorphic_table', first_node, size) second_node.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20) From 12ca9f5912cb5f8cd95036d96675cd3c2635afa9 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Mon, 13 Apr 2020 15:32:45 +0300 Subject: [PATCH 193/743] disable alwaysReturnsEmptySet() for HashJoin over dictionary --- src/Interpreters/HashJoin.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 48e7e9e9c9a..2a9314f1c80 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -188,7 +188,7 @@ public: /// Sum size in bytes of all buffers, used for JOIN maps and for all memory pools. size_t getTotalByteCount() const final; - bool alwaysReturnsEmptySet() const final { return isInnerOrRight(getKind()) && data->empty; } + bool alwaysReturnsEmptySet() const final { return isInnerOrRight(getKind()) && data->empty && !overDictionary(); } ASTTableJoin::Kind getKind() const { return kind; } ASTTableJoin::Strictness getStrictness() const { return strictness; } From a2b5fd4b3d3b6cd229367b7e38c30740a3cc3b4f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 16:18:46 +0300 Subject: [PATCH 194/743] Fix build. --- src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp index 5b6c00750ba..4364dd97f76 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp +++ b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp @@ -19,14 +19,14 @@ IMergingAlgorithmWithDelayedChunk::IMergingAlgorithmWithDelayedChunk( void IMergingAlgorithmWithDelayedChunk::initializeQueue(Chunks chunks) { - source_chunks.resize(chunks.size()); + source_chunks = std::move(chunks); for (size_t source_num = 0; source_num < source_chunks.size(); ++source_num) { - if (!chunks[source_num]) + if (!source_chunks[source_num]) continue; - source_chunks[source_num] = std::move(chunks[source_num]); + source_chunks[source_num] = std::move(source_chunks[source_num]); cursors[source_num] = SortCursorImpl(source_chunks[source_num].getColumns(), description, source_num); } From 5efdf1203ee1d7acda6218337cacd5567cf4b77e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 16:22:32 +0300 Subject: [PATCH 195/743] Fix build. --- src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp index 4364dd97f76..54d957014ea 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp +++ b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp @@ -13,10 +13,10 @@ IMergingAlgorithmWithDelayedChunk::IMergingAlgorithmWithDelayedChunk( SortDescription description_) : description(std::move(description_)) , source_chunks(num_inputs) + , cursors(num_inputs) { } - void IMergingAlgorithmWithDelayedChunk::initializeQueue(Chunks chunks) { source_chunks = std::move(chunks); From 0544c1f25be1056c59516578c5aef2cecc08d828 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 16:26:50 +0300 Subject: [PATCH 196/743] Fix build. --- src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp index 54d957014ea..16eac826f16 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp +++ b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp @@ -26,7 +26,6 @@ void IMergingAlgorithmWithDelayedChunk::initializeQueue(Chunks chunks) if (!source_chunks[source_num]) continue; - source_chunks[source_num] = std::move(source_chunks[source_num]); cursors[source_num] = SortCursorImpl(source_chunks[source_num].getColumns(), description, source_num); } From 9ce0607de7ad07cdf4dda93ac14b4d2561628223 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 16:40:53 +0300 Subject: [PATCH 197/743] Remove header from AggregatingSortedAlgorithm. --- .../Merges/AggregatingSortedAlgorithm.cpp | 11 +++++----- .../Merges/AggregatingSortedAlgorithm.h | 22 +++++++++++-------- 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp index a185e7cee67..d27ef315155 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp @@ -48,7 +48,7 @@ namespace type = nullptr; // simple aggregate function - AggregatingSortedAlgorithm::SimpleAggregateDescription desc(simple_aggr->getFunction(), i, type); + AggregatingSortedAlgorithm::SimpleAggregateDescription desc(simple_aggr->getFunction(), i, type, column.type); if (desc.function->allocatesMemoryInArena()) def.allocates_memory_in_arena = true; @@ -84,10 +84,9 @@ namespace } AggregatingSortedAlgorithm::AggregatingSortedAlgorithm( - const Block & header_, size_t num_inputs, + const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size) : IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_)) - , header(header_) , columns_definition(defineColumns(header, description_)) , merged_data(getMergedColumns(header, columns_definition), max_block_size, columns_definition) { @@ -102,7 +101,7 @@ void AggregatingSortedAlgorithm::prepareChunk(Chunk & chunk) const column = column->convertToFullColumnIfConst(); for (auto & desc : columns_definition.columns_to_simple_aggregate) - if (desc.inner_type) + if (desc.nested_type) columns[desc.column_number] = recursiveRemoveLowCardinality(columns[desc.column_number]); chunk.setColumns(std::move(columns), num_rows); @@ -159,7 +158,7 @@ IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge() if (merged_data.hasEnoughRows()) { last_key.reset(); - Status(merged_data.pull(columns_definition, header)); + Status(merged_data.pull(columns_definition)); } /// We will write the data for the group. We copy the values of ordinary columns. @@ -200,7 +199,7 @@ IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge() } last_chunk_sort_columns.clear(); - return Status(merged_data.pull(columns_definition, header), true); + return Status(merged_data.pull(columns_definition), true); } void AggregatingSortedAlgorithm::addRow(SortCursor & cursor) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/AggregatingSortedAlgorithm.h index 3327776b2f4..993973c1bc7 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.h @@ -72,7 +72,7 @@ private: /// TODO: sum_blocks_granularity += block_size; } - Chunk pull(ColumnsDefinition & def, const Block & header_) + Chunk pull(ColumnsDefinition & def) { auto chunk = pull(); @@ -81,10 +81,10 @@ private: for (auto & desc : def.columns_to_simple_aggregate) { - if (desc.inner_type) + if (desc.nested_type) { - auto & from_type = desc.inner_type; - auto & to_type = header_.getByPosition(desc.column_number).type; + auto & from_type = desc.nested_type; + auto & to_type = desc.real_type; columns_[desc.column_number] = recursiveTypeConversion(columns_[desc.column_number], from_type, to_type); } } @@ -111,8 +111,6 @@ private: using MergedData::pull; }; - Block header; - ColumnsDefinition columns_definition; AggregatingMergedData merged_data; @@ -134,13 +132,19 @@ public: size_t column_number = 0; IColumn * column = nullptr; - const DataTypePtr inner_type; + + /// For LowCardinality, convert is converted to nested type. nested_type is nullptr if no conversion needed. + const DataTypePtr nested_type; /// Nested type for LowCardinality, if it is. + const DataTypePtr real_type; /// Type in header. AlignedBuffer state; bool created = false; - SimpleAggregateDescription(AggregateFunctionPtr function_, const size_t column_number_, DataTypePtr type) - : function(std::move(function_)), column_number(column_number_), inner_type(std::move(type)) + SimpleAggregateDescription( + AggregateFunctionPtr function_, const size_t column_number_, + DataTypePtr nested_type_, DataTypePtr real_type_) + : function(std::move(function_)), column_number(column_number_) + , nested_type(std::move(nested_type_)), real_type(std::move(real_type_)) { add_function = function->getAddressOfAddFunction(); state.reset(function->sizeOfData(), function->alignOfData()); From 377e16c00c2d5ce2ebc8173ac25a18ed4c7401e2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 17:42:58 +0300 Subject: [PATCH 198/743] Refactor AggregatingSortedAlgorithm. --- .../Merges/AggregatingSortedAlgorithm.cpp | 375 +++++++++++------- .../Merges/AggregatingSortedAlgorithm.h | 147 ++----- 2 files changed, 264 insertions(+), 258 deletions(-) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp index d27ef315155..5e2fe037d3d 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -8,81 +7,251 @@ namespace DB { -namespace +/// Stores information for aggregation of AggregateFunction columns +struct AggregatingSortedAlgorithm::AggregateDescription { - AggregatingSortedAlgorithm::ColumnsDefinition defineColumns( - const Block & header, const SortDescription & description) + ColumnAggregateFunction * column = nullptr; + const size_t column_number = 0; + + AggregateDescription() = default; + explicit AggregateDescription(size_t col_number) : column_number(col_number) {} +}; + +/// Stores information for aggregation of SimpleAggregateFunction columns +struct AggregatingSortedAlgorithm::SimpleAggregateDescription +{ + /// An aggregate function 'anyLast', 'sum'... + AggregateFunctionPtr function; + IAggregateFunction::AddFunc add_function = nullptr; + + size_t column_number = 0; + IColumn * column = nullptr; + + /// For LowCardinality, convert is converted to nested type. nested_type is nullptr if no conversion needed. + const DataTypePtr nested_type; /// Nested type for LowCardinality, if it is. + const DataTypePtr real_type; /// Type in header. + + AlignedBuffer state; + bool created = false; + + SimpleAggregateDescription( + AggregateFunctionPtr function_, const size_t column_number_, + DataTypePtr nested_type_, DataTypePtr real_type_) + : function(std::move(function_)), column_number(column_number_) + , nested_type(std::move(nested_type_)), real_type(std::move(real_type_)) { - AggregatingSortedAlgorithm::ColumnsDefinition def = {}; - size_t num_columns = header.columns(); - - /// Fill in the column numbers that need to be aggregated. - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnWithTypeAndName & column = header.safeGetByPosition(i); - - /// We leave only states of aggregate functions. - if (!dynamic_cast(column.type.get()) - && !dynamic_cast(column.type->getCustomName())) - { - def.column_numbers_not_to_aggregate.push_back(i); - continue; - } - - /// Included into PK? - auto it = description.begin(); - for (; it != description.end(); ++it) - if (it->column_name == column.name || (it->column_name.empty() && it->column_number == i)) - break; - - if (it != description.end()) - { - def.column_numbers_not_to_aggregate.push_back(i); - continue; - } - - if (auto simple_aggr = dynamic_cast(column.type->getCustomName())) - { - auto type = recursiveRemoveLowCardinality(column.type); - if (type.get() == column.type.get()) - type = nullptr; - - // simple aggregate function - AggregatingSortedAlgorithm::SimpleAggregateDescription desc(simple_aggr->getFunction(), i, type, column.type); - if (desc.function->allocatesMemoryInArena()) - def.allocates_memory_in_arena = true; - - def.columns_to_simple_aggregate.emplace_back(std::move(desc)); - } - else - { - // standard aggregate function - def.columns_to_aggregate.emplace_back(i); - } - } - - return def; + add_function = function->getAddressOfAddFunction(); + state.reset(function->sizeOfData(), function->alignOfData()); } - MutableColumns getMergedColumns(const Block & header, const AggregatingSortedAlgorithm::ColumnsDefinition & def) + void createState() { - MutableColumns columns; - columns.resize(header.columns()); + if (created) + return; + function->create(state.data()); + created = true; + } - for (auto & desc : def.columns_to_simple_aggregate) + void destroyState() + { + if (!created) + return; + function->destroy(state.data()); + created = false; + } + + /// Explicitly destroy aggregation state if the stream is terminated + ~SimpleAggregateDescription() + { + destroyState(); + } + + SimpleAggregateDescription() = default; + SimpleAggregateDescription(SimpleAggregateDescription &&) = default; + SimpleAggregateDescription(const SimpleAggregateDescription &) = delete; +}; + +static AggregatingSortedAlgorithm::ColumnsDefinition defineColumns( + const Block & header, const SortDescription & description) +{ + AggregatingSortedAlgorithm::ColumnsDefinition def = {}; + size_t num_columns = header.columns(); + + /// Fill in the column numbers that need to be aggregated. + for (size_t i = 0; i < num_columns; ++i) + { + const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + + /// We leave only states of aggregate functions. + if (!dynamic_cast(column.type.get()) + && !dynamic_cast(column.type->getCustomName())) { - auto & type = header.getByPosition(desc.column_number).type; - columns[desc.column_number] = recursiveRemoveLowCardinality(type)->createColumn(); + def.column_numbers_not_to_aggregate.push_back(i); + continue; } - for (size_t i = 0; i < columns.size(); ++i) - if (!columns[i]) - columns[i] = header.getByPosition(i).type->createColumn(); + /// Included into PK? + auto it = description.begin(); + for (; it != description.end(); ++it) + if (it->column_name == column.name || (it->column_name.empty() && it->column_number == i)) + break; - return columns; + if (it != description.end()) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + if (auto simple = dynamic_cast(column.type->getCustomName())) + { + auto type = recursiveRemoveLowCardinality(column.type); + if (type.get() == column.type.get()) + type = nullptr; + + // simple aggregate function + AggregatingSortedAlgorithm::SimpleAggregateDescription desc(simple->getFunction(), i, type, column.type); + if (desc.function->allocatesMemoryInArena()) + def.allocates_memory_in_arena = true; + + def.columns_to_simple_aggregate.emplace_back(std::move(desc)); + } + else + { + // standard aggregate function + def.columns_to_aggregate.emplace_back(i); + } + } + + return def; +} + +static MutableColumns getMergedColumns(const Block & header, const AggregatingSortedAlgorithm::ColumnsDefinition & def) +{ + MutableColumns columns; + columns.resize(header.columns()); + + for (auto & desc : def.columns_to_simple_aggregate) + { + auto & type = header.getByPosition(desc.column_number).type; + columns[desc.column_number] = recursiveRemoveLowCardinality(type)->createColumn(); + } + + for (size_t i = 0; i < columns.size(); ++i) + if (!columns[i]) + columns[i] = header.getByPosition(i).type->createColumn(); + + return columns; +} + +static void prepareChunk(Chunk & chunk, const AggregatingSortedAlgorithm::ColumnsDefinition & def) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + for (auto & desc : def.columns_to_simple_aggregate) + if (desc.nested_type) + columns[desc.column_number] = recursiveRemoveLowCardinality(columns[desc.column_number]); + + chunk.setColumns(std::move(columns), num_rows); +} + + +AggregatingSortedAlgorithm::AggregatingMergedData::AggregatingMergedData( + MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_) + : MergedData(std::move(columns_), false, max_block_size_), def(def_) +{ + initAggregateDescription(); +} + +void AggregatingSortedAlgorithm::AggregatingMergedData::startGroup(const ColumnRawPtrs & raw_columns, size_t row) +{ + /// We will write the data for the group. We copy the values of ordinary columns. + for (auto column_number : def.column_numbers_not_to_aggregate) + columns[column_number]->insertFrom(*raw_columns[column_number], row); + + /// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function. + for (auto & column_to_aggregate : def.columns_to_aggregate) + column_to_aggregate.column->insertDefault(); + + /// Reset simple aggregation states for next row + for (auto & desc : def.columns_to_simple_aggregate) + desc.createState(); + + if (def.allocates_memory_in_arena) + arena = std::make_unique(); + + is_group_started = true; +} + +void AggregatingSortedAlgorithm::AggregatingMergedData::finishGroup() +{ + /// Write the simple aggregation result for the current group. + for (auto & desc : def.columns_to_simple_aggregate) + { + desc.function->insertResultInto(desc.state.data(), *desc.column); + desc.destroyState(); + } + + is_group_started = false; + ++total_merged_rows; + ++merged_rows; + /// TODO: sum_blocks_granularity += block_size; +} + +void AggregatingSortedAlgorithm::AggregatingMergedData::addRow(SortCursor & cursor) +{ + if (!is_group_started) + throw Exception("Can't add a row to the group because it was not started.", ErrorCodes::LOGICAL_ERROR); + + for (auto & desc : def.columns_to_aggregate) + desc.column->insertMergeFrom(*cursor->all_columns[desc.column_number], cursor->pos); + + for (auto & desc : def.columns_to_simple_aggregate) + { + auto & col = cursor->all_columns[desc.column_number]; + desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get()); } } +Chunk AggregatingSortedAlgorithm::AggregatingMergedData::pull() +{ + if (is_group_started) + throw Exception("Can't pull chunk because group was not finished.", ErrorCodes::LOGICAL_ERROR); + + auto chunk = MergedData::pull(); + + size_t num_rows = chunk.getNumRows(); + auto columns_ = chunk.detachColumns(); + + for (auto & desc : def.columns_to_simple_aggregate) + { + if (desc.nested_type) + { + auto & from_type = desc.nested_type; + auto & to_type = desc.real_type; + columns_[desc.column_number] = recursiveTypeConversion(columns_[desc.column_number], from_type, to_type); + } + } + + chunk.setColumns(std::move(columns_), num_rows); + initAggregateDescription(); + + return chunk; +} + +void AggregatingSortedAlgorithm::AggregatingMergedData::initAggregateDescription() +{ + for (auto & desc : def.columns_to_simple_aggregate) + desc.column = columns[desc.column_number].get(); + + for (auto & desc : def.columns_to_aggregate) + desc.column = typeid_cast(columns[desc.column_number].get()); +} + + AggregatingSortedAlgorithm::AggregatingSortedAlgorithm( const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size) @@ -92,33 +261,18 @@ AggregatingSortedAlgorithm::AggregatingSortedAlgorithm( { } -void AggregatingSortedAlgorithm::prepareChunk(Chunk & chunk) const -{ - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); - - for (auto & desc : columns_definition.columns_to_simple_aggregate) - if (desc.nested_type) - columns[desc.column_number] = recursiveRemoveLowCardinality(columns[desc.column_number]); - - chunk.setColumns(std::move(columns), num_rows); -} - void AggregatingSortedAlgorithm::initialize(Chunks chunks) { for (auto & chunk : chunks) if (chunk) - prepareChunk(chunk); + prepareChunk(chunk, columns_definition); initializeQueue(std::move(chunks)); } void AggregatingSortedAlgorithm::consume(Chunk chunk, size_t source_num) { - prepareChunk(chunk); + prepareChunk(chunk, columns_definition); updateCursor(std::move(chunk), source_num); } @@ -128,18 +282,13 @@ IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge() while (queue.isValid()) { bool key_differs; - bool has_previous_group = !last_key.empty(); - SortCursor current = queue.current(); { detail::RowRef current_key; current_key.set(current); - if (!has_previous_group) /// The first key encountered. - key_differs = true; - else - key_differs = !last_key.hasEqualSortColumnsWith(current_key); + key_differs = last_key.empty() || !last_key.hasEqualSortColumnsWith(current_key); last_key = current_key; last_chunk_sort_columns.clear(); @@ -147,37 +296,20 @@ IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge() if (key_differs) { - /// Write the simple aggregation result for the previous group. if (merged_data.isGroupStarted()) - { - insertSimpleAggregationResult(); - merged_data.insertRow(); - } + merged_data.finishGroup(); /// if there are enough rows accumulated and the last one is calculated completely if (merged_data.hasEnoughRows()) { last_key.reset(); - Status(merged_data.pull(columns_definition)); + Status(merged_data.pull()); } - /// We will write the data for the group. We copy the values of ordinary columns. - merged_data.initializeRow(current->all_columns, current->pos, - columns_definition.column_numbers_not_to_aggregate); - - /// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function. - for (auto & column_to_aggregate : columns_definition.columns_to_aggregate) - column_to_aggregate.column->insertDefault(); - - /// Reset simple aggregation states for next row - for (auto & desc : columns_definition.columns_to_simple_aggregate) - desc.createState(); - - if (columns_definition.allocates_memory_in_arena) - arena = std::make_unique(); + merged_data.startGroup(current->all_columns, current->pos); } - addRow(current); + merged_data.addRow(current); if (!current->isLast()) { @@ -193,35 +325,10 @@ IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge() /// Write the simple aggregation result for the previous group. if (merged_data.isGroupStarted()) - { - insertSimpleAggregationResult(); - merged_data.insertRow(); - } + merged_data.finishGroup(); last_chunk_sort_columns.clear(); - return Status(merged_data.pull(columns_definition), true); + return Status(merged_data.pull(), true); } -void AggregatingSortedAlgorithm::addRow(SortCursor & cursor) -{ - for (auto & desc : columns_definition.columns_to_aggregate) - desc.column->insertMergeFrom(*cursor->all_columns[desc.column_number], cursor->pos); - - for (auto & desc : columns_definition.columns_to_simple_aggregate) - { - auto & col = cursor->all_columns[desc.column_number]; - desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get()); - } -} - -void AggregatingSortedAlgorithm::insertSimpleAggregationResult() -{ - for (auto & desc : columns_definition.columns_to_simple_aggregate) - { - desc.function->insertResultInto(desc.state.data(), *desc.column); - desc.destroyState(); - } -} - - } diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/AggregatingSortedAlgorithm.h index 993973c1bc7..f4c6c258297 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -22,18 +23,14 @@ public: Status merge() override; struct SimpleAggregateDescription; + struct AggregateDescription; + /// This structure define columns into one of three types: + /// * columns which are not aggregate functions and not needed to be aggregated + /// * usual aggregate functions, which stores states into ColumnAggregateFunction + /// * simple aggregate functions, which store states into ordinary columns struct ColumnsDefinition { - struct AggregateDescription - { - ColumnAggregateFunction * column = nullptr; - const size_t column_number = 0; - - AggregateDescription() = default; - explicit AggregateDescription(size_t col_number) : column_number(col_number) {} - }; - /// Columns with which numbers should not be aggregated. ColumnNumbers column_numbers_not_to_aggregate; std::vector columns_to_aggregate; @@ -47,135 +44,37 @@ private: /// Specialization for AggregatingSortedAlgorithm. struct AggregatingMergedData : public MergedData { + private: + using MergedData::pull; + using MergedData::insertRow; + public: - AggregatingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def) - : MergedData(std::move(columns_), false, max_block_size_) - { - initAggregateDescription(def); - } + AggregatingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_); - void initializeRow(const ColumnRawPtrs & raw_columns, size_t row, const ColumnNumbers & column_numbers) - { - for (auto column_number : column_numbers) - columns[column_number]->insertFrom(*raw_columns[column_number], row); - - is_group_started = true; - } + void startGroup(const ColumnRawPtrs & raw_columns, size_t row); + void finishGroup(); bool isGroupStarted() const { return is_group_started; } + void addRow(SortCursor & cursor); - void insertRow() - { - is_group_started = false; - ++total_merged_rows; - ++merged_rows; - /// TODO: sum_blocks_granularity += block_size; - } - - Chunk pull(ColumnsDefinition & def) - { - auto chunk = pull(); - - size_t num_rows = chunk.getNumRows(); - auto columns_ = chunk.detachColumns(); - - for (auto & desc : def.columns_to_simple_aggregate) - { - if (desc.nested_type) - { - auto & from_type = desc.nested_type; - auto & to_type = desc.real_type; - columns_[desc.column_number] = recursiveTypeConversion(columns_[desc.column_number], from_type, to_type); - } - } - - chunk.setColumns(std::move(columns_), num_rows); - initAggregateDescription(def); - - return chunk; - } + Chunk pull(); private: + ColumnsDefinition & def; + + /// Memory pool for SimpleAggregateFunction + /// (only when allocates_memory_in_arena == true). + std::unique_ptr arena; + bool is_group_started = false; /// Initialize aggregate descriptions with columns. - void initAggregateDescription(ColumnsDefinition & def) - { - for (auto & desc : def.columns_to_simple_aggregate) - desc.column = columns[desc.column_number].get(); - - for (auto & desc : def.columns_to_aggregate) - desc.column = typeid_cast(columns[desc.column_number].get()); - } - - using MergedData::pull; + void initAggregateDescription(); }; + /// Order between members is important because merged_data has reference to columns_definition. ColumnsDefinition columns_definition; AggregatingMergedData merged_data; - - /// Memory pool for SimpleAggregateFunction - /// (only when allocates_memory_in_arena == true). - std::unique_ptr arena; - - void prepareChunk(Chunk & chunk) const; - void addRow(SortCursor & cursor); - void insertSimpleAggregationResult(); - -public: - /// Stores information for aggregation of SimpleAggregateFunction columns - struct SimpleAggregateDescription - { - /// An aggregate function 'anyLast', 'sum'... - AggregateFunctionPtr function; - IAggregateFunction::AddFunc add_function = nullptr; - - size_t column_number = 0; - IColumn * column = nullptr; - - /// For LowCardinality, convert is converted to nested type. nested_type is nullptr if no conversion needed. - const DataTypePtr nested_type; /// Nested type for LowCardinality, if it is. - const DataTypePtr real_type; /// Type in header. - - AlignedBuffer state; - bool created = false; - - SimpleAggregateDescription( - AggregateFunctionPtr function_, const size_t column_number_, - DataTypePtr nested_type_, DataTypePtr real_type_) - : function(std::move(function_)), column_number(column_number_) - , nested_type(std::move(nested_type_)), real_type(std::move(real_type_)) - { - add_function = function->getAddressOfAddFunction(); - state.reset(function->sizeOfData(), function->alignOfData()); - } - - void createState() - { - if (created) - return; - function->create(state.data()); - created = true; - } - - void destroyState() - { - if (!created) - return; - function->destroy(state.data()); - created = false; - } - - /// Explicitly destroy aggregation state if the stream is terminated - ~SimpleAggregateDescription() - { - destroyState(); - } - - SimpleAggregateDescription() = default; - SimpleAggregateDescription(SimpleAggregateDescription &&) = default; - SimpleAggregateDescription(const SimpleAggregateDescription &) = delete; - }; }; } From d3adcc1cb0fa589f4e9166c907dfcdd845c23dc1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 17:53:04 +0300 Subject: [PATCH 199/743] Fix build. --- src/Processors/Merges/AggregatingSortedAlgorithm.cpp | 4 +++- src/Processors/Merges/AggregatingSortedAlgorithm.h | 9 +++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp index 5e2fe037d3d..a9ea347a66f 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp @@ -7,11 +7,13 @@ namespace DB { +AggregatingSortedAlgorithm::ColumnsDefinition::~ColumnsDefinition() = default; + /// Stores information for aggregation of AggregateFunction columns struct AggregatingSortedAlgorithm::AggregateDescription { ColumnAggregateFunction * column = nullptr; - const size_t column_number = 0; + const size_t column_number = 0; /// Position in header. AggregateDescription() = default; explicit AggregateDescription(size_t col_number) : column_number(col_number) {} diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/AggregatingSortedAlgorithm.h index f4c6c258297..16953d11e3c 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.h @@ -31,6 +31,8 @@ public: /// * simple aggregate functions, which store states into ordinary columns struct ColumnsDefinition { + ~ColumnsDefinition(); + /// Columns with which numbers should not be aggregated. ColumnNumbers column_numbers_not_to_aggregate; std::vector columns_to_aggregate; @@ -51,13 +53,16 @@ private: public: AggregatingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_); + /// Group is a group of rows with the same sorting key. It represents single row in result. + /// Algorithm is: start group, add several rows, finish group. + /// Then pull chunk when enough groups were added. void startGroup(const ColumnRawPtrs & raw_columns, size_t row); void finishGroup(); bool isGroupStarted() const { return is_group_started; } - void addRow(SortCursor & cursor); + void addRow(SortCursor & cursor); /// Possible only when group was started. - Chunk pull(); + Chunk pull(); /// Possible only if group was finished. private: ColumnsDefinition & def; From 9acea16b7f628bc8c601a5db7875d32d91965b37 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 17:57:21 +0300 Subject: [PATCH 200/743] Fix build. --- src/Processors/Merges/AggregatingSortedAlgorithm.cpp | 2 ++ src/Processors/Merges/AggregatingSortedAlgorithm.h | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp index a9ea347a66f..3d7075efc97 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp @@ -7,6 +7,8 @@ namespace DB { +AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition() = default; +AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition(ColumnsDefinition &&) = default; AggregatingSortedAlgorithm::ColumnsDefinition::~ColumnsDefinition() = default; /// Stores information for aggregation of AggregateFunction columns diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/AggregatingSortedAlgorithm.h index 16953d11e3c..6840b485ce5 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.h @@ -31,6 +31,8 @@ public: /// * simple aggregate functions, which store states into ordinary columns struct ColumnsDefinition { + ColumnsDefinition(); + ColumnsDefinition(ColumnsDefinition &&) noexcept; ~ColumnsDefinition(); /// Columns with which numbers should not be aggregated. From 2b81d68b56fcfe22804312f23f8ae1c3a02d5fe9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 17:58:23 +0300 Subject: [PATCH 201/743] Fix build. --- src/Processors/Merges/AggregatingSortedAlgorithm.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp index 3d7075efc97..e6138ecc6ab 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp @@ -8,7 +8,7 @@ namespace DB { AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition() = default; -AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition(ColumnsDefinition &&) = default; +AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition(ColumnsDefinition &&) noexcept = default; AggregatingSortedAlgorithm::ColumnsDefinition::~ColumnsDefinition() = default; /// Stores information for aggregation of AggregateFunction columns From 89aef7aaf932f706a9598b67c3c2d04d3f6a91cb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 18:04:54 +0300 Subject: [PATCH 202/743] Add comments. --- src/Processors/Merges/AggregatingSortedAlgorithm.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/AggregatingSortedAlgorithm.h index 6840b485ce5..ad5644f7189 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.h @@ -31,9 +31,9 @@ public: /// * simple aggregate functions, which store states into ordinary columns struct ColumnsDefinition { - ColumnsDefinition(); - ColumnsDefinition(ColumnsDefinition &&) noexcept; - ~ColumnsDefinition(); + ColumnsDefinition(); /// Is needed because destructor is defined. + ColumnsDefinition(ColumnsDefinition &&) noexcept; /// Is needed because destructor is defined. + ~ColumnsDefinition(); /// Is needed because otherwise std::vector's destructor uses incomplete types. /// Columns with which numbers should not be aggregated. ColumnNumbers column_numbers_not_to_aggregate; From 441bb13d4aa10f15056c9a14d109bad795158916 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Apr 2020 18:21:05 +0300 Subject: [PATCH 203/743] better --- src/Interpreters/InterserverIOHandler.h | 2 - .../MergeTree/MergeTreeDataMergerMutator.cpp | 13 ++- .../MergeTree/MergeTreeDataMergerMutator.h | 3 - src/Storages/StorageReplicatedMergeTree.cpp | 107 +++--------------- src/Storages/StorageReplicatedMergeTree.h | 8 +- tests/integration/test_quorum_inserts/test.py | 75 ++++++++++-- 6 files changed, 99 insertions(+), 109 deletions(-) diff --git a/src/Interpreters/InterserverIOHandler.h b/src/Interpreters/InterserverIOHandler.h index 0ffccb6c33c..952c99ae46d 100644 --- a/src/Interpreters/InterserverIOHandler.h +++ b/src/Interpreters/InterserverIOHandler.h @@ -13,8 +13,6 @@ #include #include #include -#include -#include namespace Poco { namespace Net { class HTTPServerResponse; } } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index f892302086d..5450f6f66de 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -222,19 +222,28 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( IMergeSelector::Partitions partitions; const String * prev_partition_id = nullptr; + /// Previous part only in boundaries of partition frame const MergeTreeData::DataPartPtr * prev_part = nullptr; bool has_part_with_expired_ttl = false; for (const MergeTreeData::DataPartPtr & part : data_parts) { - if (!single_merge(part, nullptr)) - continue; + /// Check predicate only for first part in each partition. + if (!prev_part) + /* Parts can be merged with themselves for TTL needs for example. + * So we have to check if this part is currently being inserted with quorum and so on and so forth. + * Obviously we have to check it manually only for the first part + * of each partition because it will be automatically checked for a pair of parts. */ + if (!single_merge(part, nullptr)) + continue; const String & partition_id = part->info.partition_id; if (!prev_partition_id || partition_id != *prev_partition_id || (prev_part && !can_merge_callback(*prev_part, part, nullptr))) { if (partitions.empty() || !partitions.back().empty()) partitions.emplace_back(); + /// New partition frame. prev_partition_id = &partition_id; + prev_part = nullptr; } IMergeSelector::Part part_info; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 0efdb403bf0..0be1909889b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -51,10 +51,7 @@ class MergeTreeDataMergerMutator public: using AllowedMergingPredicate = std::function; using AllowedSingleMergePredicate = std::function; -// template -// using AllowedMergingPredicate = std::function; -public: MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size); /** Get maximum total size of parts to do merge, at current moment of time. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7548edad48a..fec0d5fc287 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2188,15 +2188,14 @@ void StorageReplicatedMergeTree::mergeSelectingTask() UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); FutureMergedMutatedPart future_merged_part; + auto merge_pred_for_single_part = + [&merge_pred] (const MergeTreeData::DataPartPtr & part, String * explain) -> bool + { + return merge_pred.canMergeSinglePart(part, explain); + }; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, nullptr, - [&merge_pred](const MergeTreeData::DataPartPtr & part, String * explain) -> bool { return merge_pred.canMergeSinglePart(part, explain); })) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, nullptr, merge_pred_for_single_part)) { - /// We have to exclude parts, that are currently being writted with quorum. (check .../quorum/status node) - /// Also we have to exclude last parts written with quorum (check .../quorum/last_part node) -// if (global_context.getSettingsRef().insert_quorum) -// excludeSomePartsFromMerge(future_merged_part); - success = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, force_ttl); } @@ -2673,53 +2672,6 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } -void StorageReplicatedMergeTree::deletePartFromPendingQuorum(const String & part_name) -{ - auto zookeeper = getZooKeeper(); - /// Information on which replicas a part has been added, if the quorum has not yet been reached. - const String quorum_status_path = zookeeper_path + "/quorum/status"; - - /// Delete "status" node if required. - - String value; - Coordination::Stat stat; - - /// If there is no node, then all quorum INSERTs have already reached the quorum, and nothing is needed. - while (zookeeper->tryGet(quorum_status_path, value, &stat)) - { - ReplicatedMergeTreeQuorumEntry quorum_entry; - quorum_entry.fromString(value); - - if (quorum_entry.part_name != part_name) - { - /// There is no information about interested part in this node. - break; - } - - /// Since that we are sure that interested part is being involved in insert with quorum. - /// Our goal is to delete "status" node and information from "last_part" node. - - auto code = zookeeper->tryRemove(quorum_status_path, stat.version); - - if (code == Coordination::ZOK) - { - break; - } - else if (code == Coordination::ZNONODE) - { - /// The quorum has already been achieved. - break; - } - else if (code == Coordination::ZBADVERSION) - { - /// Node was updated meanwhile. We must re-read it and repeat all the actions. - continue; - } - else - throw Coordination::Exception(code, quorum_status_path); - } -} - void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id, const String & part_name) { auto zookeeper = getZooKeeper(); @@ -2752,7 +2704,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id, } /// De Morgan's law - if (part_name == "" || parts_with_quorum.added_parts[partition_id] == part_name) + if (part_name.empty() || parts_with_quorum.added_parts[partition_id] == part_name) parts_with_quorum.added_parts.erase(partition_id); else break; @@ -2781,32 +2733,6 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id, } -void StorageReplicatedMergeTree::excludeSomePartsFromMerge(FutureMergedMutatedPart & future_part) -{ - LOG_FATAL(&Poco::Logger::get("excludeSomePartsFromMerge"), "BEGIN!"); - - auto zookeeper = getZooKeeper(); - /// Information on which replicas a part has been added, if the quorum has not yet been reached. - const String quorum_status_path = zookeeper_path + "/quorum/status"; - - String value; - Coordination::Stat stat; - - if (zookeeper->tryGet(quorum_status_path, value, &stat)) { - ReplicatedMergeTreeQuorumEntry quorum_entry; - quorum_entry.fromString(value); - - MergeTreeData::DataPartsVector & parts_to_merge = future_part.parts; - - parts_to_merge.erase( - std::remove_if( - parts_to_merge.begin(), parts_to_merge.end(), - [&quorum_entry](const MergeTreeData::DataPartPtr & part_to_merge) { return part_to_merge->name == quorum_entry.part_name; }), - parts_to_merge.end()); - } -} - - bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & source_replica_path, bool to_detached, size_t quorum) { const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); @@ -5057,10 +4983,10 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ } } -void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) +void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context) { - auto lock1 = lockStructureForShare(false, context.getCurrentQueryId()); - auto lock2 = dest_table->lockStructureForShare(false, context.getCurrentQueryId()); + auto lock1 = lockStructureForShare(false, query_context.getCurrentQueryId()); + auto lock2 = dest_table->lockStructureForShare(false, query_context.getCurrentQueryId()); auto dest_table_storage = std::dynamic_pointer_cast(dest_table); if (!dest_table_storage) @@ -5075,7 +5001,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta Stopwatch watch; MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this); auto src_data_id = src_data.getStorageID(); - String partition_id = getPartitionIDFromQuery(partition, context); + String partition_id = getPartitionIDFromQuery(partition, query_context); DataPartsVector src_all_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); DataPartsVector src_parts; @@ -5231,7 +5157,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta parts_to_remove.clear(); cleanup_thread.wakeup(); - if (context.getSettingsRef().replication_alter_partitions_sync > 1) + if (query_context.getSettingsRef().replication_alter_partitions_sync > 1) { lock2.release(); dest_table_storage->waitForAllReplicasToProcessLogEntry(entry); @@ -5246,7 +5172,10 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta log_znode_path = dynamic_cast(*op_results.back()).path_created; entry_delete.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - if (context.getSettingsRef().replication_alter_partitions_sync > 1) + if (query_context.getSettingsRef().insert_quorum) + cleanLastPartNode(partition_id); + + if (query_context.getSettingsRef().replication_alter_partitions_sync > 1) { lock1.release(); waitForAllReplicasToProcessLogEntry(entry_delete); @@ -5320,10 +5249,10 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti return merger_mutator.ttl_merges_blocker.cancel(); if (action_type == ActionLocks::PartsFetch) - return fetcher.blocker.cancel(); + return fetcher.blocker.cancel(); if (action_type == ActionLocks::PartsSend) - return data_parts_exchange_endpoint ? data_parts_exchange_endpoint->blocker.cancel() : ActionLock(); + return data_parts_exchange_endpoint ? data_parts_exchange_endpoint->blocker.cancel() : ActionLock(); if (action_type == ActionLocks::ReplicationQueue) return queue.actions_blocker.cancel(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bf07b592e2f..2846e731926 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -464,14 +464,10 @@ private: /// With the quorum being tracked, add a replica to the quorum for the part. void updateQuorum(const String & part_name); - /// Delete particular part name not to track it in future. - void deletePartFromPendingQuorum(const String & part_name); - - + /// Deletes info from quorum/last_part node for particular partition_id. + /// If part_name if specified, deletes only if last_part == part_name. void cleanLastPartNode(const String & partition_id, const String & part_name = ""); - void excludeSomePartsFromMerge(FutureMergedMutatedPart & future_part); - /// Creates new block number if block with such block_id does not exist std::optional allocateBlockNumber( const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 64190c6d8aa..c85a1825b94 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -143,7 +143,7 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ "PARTITION BY d ORDER BY a " - print("Create Replicated table with two replicas") + print("Create Replicated table with three replicas") zero.query(create_query) first.query(create_query) second.query(create_query) @@ -182,6 +182,73 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): execute_on_all_cluster("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition") +@pytest.mark.parametrize( + ('add_new_data'), + [ + False, + True + ] +) + +def test_insert_quorum_with_move_partition(started_cluster, add_new_data): + execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source") + execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination") + + create_source = "CREATE TABLE test_insert_quorum_with_move_partition_source " \ + "(a Int8, d Date) " \ + "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ + "PARTITION BY d ORDER BY a " + + create_destination = "CREATE TABLE test_insert_quorum_with_move_partition_destination " \ + "(a Int8, d Date) " \ + "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ + "PARTITION BY d ORDER BY a " + + print("Create source Replicated table with three replicas") + zero.query(create_source) + first.query(create_source) + second.query(create_source) + + print("Create destination Replicated table with three replicas") + zero.query(create_destination) + first.query(create_destination) + second.query(create_destination) + + print("Stop fetches for test_insert_quorum_with_move_partition_source at first replica.") + first.query("SYSTEM STOP FETCHES test_insert_quorum_with_move_partition_source") + + print("Insert with quorum. (zero and second)") + zero.query("INSERT INTO test_insert_quorum_with_move_partition_source(a,d) VALUES(1, '2011-01-01')") + + print("Drop partition.") + zero.query("ALTER TABLE test_insert_quorum_with_move_partition_source MOVE PARTITION '2011-01-01' TO TABLE test_insert_quorum_with_move_partition_destination") + + if (add_new_data): + print("Insert to deleted partition") + zero.query("INSERT INTO test_insert_quorum_with_move_partition_source(a,d) VALUES(2, '2011-01-01')") + + print("Resume fetches for test_insert_quorum_with_move_partition_source at first replica.") + first.query("SYSTEM START FETCHES test_insert_quorum_with_move_partition_source") + + print("Sync first replica with others.") + first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_move_partition_source") + + assert "20110101" not in first.query("SELECT * FROM system.zookeeper " \ + "where path='/clickhouse/tables/0/test_insert_quorum_with_move_partition_source/quorum/last_part' " \ + "format Vertical") + + print("Select from updated partition.") + if (add_new_data): + assert TSV("2\t2011-01-01\n") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source")) + assert TSV("2\t2011-01-01\n") == TSV(second.query("SELECT * FROM test_insert_quorum_with_move_partition_source")) + else: + assert TSV("") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source")) + assert TSV("") == TSV(second.query("SELECT * FROM test_insert_quorum_with_move_partition_source")) + + execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source") + execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination") + + def test_insert_quorum_with_ttl(started_cluster): execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_ttl") @@ -204,8 +271,6 @@ def test_insert_quorum_with_ttl(started_cluster): settings={'insert_quorum_timeout' : 5000}) assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." - print(zero.query("SELECT * FROM system.parts format Vertical")) - print("Wait 10 seconds and TTL merge have to be executed. But it won't delete data.") time.sleep(10) assert TSV("1\t2011-01-01\n") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency' : 0})) @@ -216,10 +281,6 @@ def test_insert_quorum_with_ttl(started_cluster): print("Sync first replica.") first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_ttl") - - print(first.query("SELECT * from system.replicas format Vertical")) - print(first.query("SELECT * from system.zookeeper where path='/clickhouse/tables/test_insert_quorum_with_ttl/quorum' format Vertical")) - zero.query("INSERT INTO test_insert_quorum_with_ttl(a,d) VALUES(1, '2011-01-01')", settings={'insert_quorum_timeout' : 5000}) From 910ab8a26567e0e440d73549eeb10808bc038142 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 13 Apr 2020 18:42:30 +0300 Subject: [PATCH 204/743] =?UTF-8?q?Move=20contrib/libpcg-random=20?= =?UTF-8?q?=E2=86=92=20base/pcg-random?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- base/CMakeLists.txt | 3 +- base/pcg-random/CMakeLists.txt | 2 + .../pcg-random/LICENSE | 0 .../libpcg-random => base/pcg-random}/README | 0 .../pcg-random}/pcg_extras.hpp | 8 +-- .../pcg-random}/pcg_random.hpp | 0 .../pcg-random}/pcg_uint128.hpp | 0 cmake/lib_name.cmake | 1 - contrib/libpcg-random/README.md | 52 ------------------- programs/benchmark/CMakeLists.txt | 1 - programs/copier/CMakeLists.txt | 2 +- src/CMakeLists.txt | 4 +- 12 files changed, 10 insertions(+), 63 deletions(-) create mode 100644 base/pcg-random/CMakeLists.txt rename contrib/libpcg-random/LICENSE-APACHE.txt => base/pcg-random/LICENSE (100%) rename {contrib/libpcg-random => base/pcg-random}/README (100%) rename {contrib/libpcg-random/include => base/pcg-random}/pcg_extras.hpp (98%) rename {contrib/libpcg-random/include => base/pcg-random}/pcg_random.hpp (100%) rename {contrib/libpcg-random/include => base/pcg-random}/pcg_uint128.hpp (100%) delete mode 100644 contrib/libpcg-random/README.md diff --git a/base/CMakeLists.txt b/base/CMakeLists.txt index e202021fe48..1ead921f7e4 100644 --- a/base/CMakeLists.txt +++ b/base/CMakeLists.txt @@ -3,8 +3,9 @@ if (USE_CLANG_TIDY) endif () add_subdirectory (common) -add_subdirectory (loggers) add_subdirectory (daemon) +add_subdirectory (loggers) +add_subdirectory (pcg-random) if (USE_MYSQL) add_subdirectory (mysqlxx) diff --git a/base/pcg-random/CMakeLists.txt b/base/pcg-random/CMakeLists.txt new file mode 100644 index 00000000000..88acabba6a7 --- /dev/null +++ b/base/pcg-random/CMakeLists.txt @@ -0,0 +1,2 @@ +add_library(pcg_random INTERFACE) +target_include_directories(pcg_random INTERFACE .) diff --git a/contrib/libpcg-random/LICENSE-APACHE.txt b/base/pcg-random/LICENSE similarity index 100% rename from contrib/libpcg-random/LICENSE-APACHE.txt rename to base/pcg-random/LICENSE diff --git a/contrib/libpcg-random/README b/base/pcg-random/README similarity index 100% rename from contrib/libpcg-random/README rename to base/pcg-random/README diff --git a/contrib/libpcg-random/include/pcg_extras.hpp b/base/pcg-random/pcg_extras.hpp similarity index 98% rename from contrib/libpcg-random/include/pcg_extras.hpp rename to base/pcg-random/pcg_extras.hpp index 929c756b151..118b726dd57 100644 --- a/contrib/libpcg-random/include/pcg_extras.hpp +++ b/base/pcg-random/pcg_extras.hpp @@ -292,7 +292,7 @@ inline itype rotl(itype value, bitcount_t rot) { constexpr bitcount_t bits = sizeof(itype) * 8; constexpr bitcount_t mask = bits - 1; -#if PCG_USE_ZEROCHECK_ROTATE_IDIOM +#if defined(PCG_USE_ZEROCHECK_ROTATE_IDIOM) return rot ? (value << rot) | (value >> (bits - rot)) : value; #else return (value << rot) | (value >> ((- rot) & mask)); @@ -304,7 +304,7 @@ inline itype rotr(itype value, bitcount_t rot) { constexpr bitcount_t bits = sizeof(itype) * 8; constexpr bitcount_t mask = bits - 1; -#if PCG_USE_ZEROCHECK_ROTATE_IDIOM +#if defined(PCG_USE_ZEROCHECK_ROTATE_IDIOM) return rot ? (value >> rot) | (value << (bits - rot)) : value; #else return (value >> rot) | (value << ((- rot) & mask)); @@ -318,7 +318,7 @@ inline itype rotr(itype value, bitcount_t rot) * * These overloads will be preferred over the general template code above. */ -#if PCG_USE_INLINE_ASM && __GNUC__ && (__x86_64__ || __i386__) +#if defined(PCG_USE_INLINE_ASM) && __GNUC__ && (__x86_64__ || __i386__) inline uint8_t rotr(uint8_t value, bitcount_t rot) { @@ -600,7 +600,7 @@ std::ostream& operator<<(std::ostream& out, printable_typename) { #ifdef __GNUC__ int status; char* pretty_name = - abi::__cxa_demangle(implementation_typename, NULL, NULL, &status); + abi::__cxa_demangle(implementation_typename, nullptr, nullptr, &status); if (status == 0) out << pretty_name; free(static_cast(pretty_name)); diff --git a/contrib/libpcg-random/include/pcg_random.hpp b/base/pcg-random/pcg_random.hpp similarity index 100% rename from contrib/libpcg-random/include/pcg_random.hpp rename to base/pcg-random/pcg_random.hpp diff --git a/contrib/libpcg-random/include/pcg_uint128.hpp b/base/pcg-random/pcg_uint128.hpp similarity index 100% rename from contrib/libpcg-random/include/pcg_uint128.hpp rename to base/pcg-random/pcg_uint128.hpp diff --git a/cmake/lib_name.cmake b/cmake/lib_name.cmake index 8f5bebf4abe..f18b2e52576 100644 --- a/cmake/lib_name.cmake +++ b/cmake/lib_name.cmake @@ -2,4 +2,3 @@ set(DIVIDE_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libdivide) set(DBMS_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/src ${ClickHouse_BINARY_DIR}/src) set(DOUBLE_CONVERSION_CONTRIB_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/double-conversion) set(METROHASH_CONTRIB_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libmetrohash/src) -set(PCG_RANDOM_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libpcg-random/include) diff --git a/contrib/libpcg-random/README.md b/contrib/libpcg-random/README.md deleted file mode 100644 index c6e579cc2cc..00000000000 --- a/contrib/libpcg-random/README.md +++ /dev/null @@ -1,52 +0,0 @@ -# PCG Random Number Generation, C++ Edition - -[PCG-Random website]: http://www.pcg-random.org - -This code provides an implementation of the PCG family of random number -generators, which are fast, statistically excellent, and offer a number of -useful features. - -Full details can be found at the [PCG-Random website]. This version -of the code provides many family members -- if you just want one -simple generator, you may prefer the minimal C version of the library. - -There are two kinds of generator, normal generators and extended generators. -Extended generators provide *k* dimensional equidistribution and can perform -party tricks, but generally speaking most people only need the normal -generators. - -There are two ways to access the generators, using a convenience typedef -or by using the underlying templates directly (similar to C++11's `std::mt19937` typedef vs its `std::mersenne_twister_engine` template). For most users, the convenience typedef is what you want, and probably you're fine with `pcg32` for 32-bit numbers. If you want 64-bit numbers, either use `pcg64` (or, if you're on a 32-bit system, making 64 bits from two calls to `pcg32_k2` may be faster). - -## Documentation and Examples - -Visit [PCG-Random website] for information on how to use this library, or look -at the sample code in the `sample` directory -- hopefully it should be fairly -self explanatory. - -## Building - -The code is written in C++11, as an include-only library (i.e., there is -nothing you need to build). There are some provided demo programs and tests -however. On a Unix-style system (e.g., Linux, Mac OS X) you should be able -to just type - - make - -To build the demo programs. - -## Testing - -Run - - make test - -## Directory Structure - -The directories are arranged as follows: - -* `include` -- contains `pcg_random.hpp` and supporting include files -* `test-high` -- test code for the high-level API where the functions have - shorter, less scary-looking names. -* `sample` -- sample code, some similar to the code in `test-high` but more - human readable, some other examples too diff --git a/programs/benchmark/CMakeLists.txt b/programs/benchmark/CMakeLists.txt index 58096985037..be999aafe80 100644 --- a/programs/benchmark/CMakeLists.txt +++ b/programs/benchmark/CMakeLists.txt @@ -1,6 +1,5 @@ set(CLICKHOUSE_BENCHMARK_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/Benchmark.cpp) set(CLICKHOUSE_BENCHMARK_LINK PRIVATE dbms clickhouse_aggregate_functions clickhouse_common_config ${Boost_PROGRAM_OPTIONS_LIBRARY}) -set(CLICKHOUSE_BENCHMARK_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) clickhouse_program_add(benchmark) diff --git a/programs/copier/CMakeLists.txt b/programs/copier/CMakeLists.txt index 5573fbc5e5d..76db6ce1ffc 100644 --- a/programs/copier/CMakeLists.txt +++ b/programs/copier/CMakeLists.txt @@ -12,6 +12,6 @@ set(CLICKHOUSE_COPIER_LINK PRIVATE clickhouse_dictionaries string_utils ${Poco_XML_LIBRARY} PUBLIC daemon) -set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}) +set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}) clickhouse_program_add(copier) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 2393e0be18d..9949f13def0 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -398,6 +398,7 @@ endif() target_link_libraries(clickhouse_common_io PUBLIC ${CITYHASH_LIBRARIES} + pcg_random PRIVATE ${Poco_XML_LIBRARY} ${ZLIB_LIBRARIES} @@ -453,9 +454,6 @@ dbms_target_link_libraries ( target_include_directories(clickhouse_common_io PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/Core/include) # uses some includes from core dbms_target_include_directories(PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/Core/include) -target_include_directories(clickhouse_common_io SYSTEM PUBLIC ${PCG_RANDOM_INCLUDE_DIR}) -dbms_target_include_directories(SYSTEM PUBLIC ${PCG_RANDOM_INCLUDE_DIR}) - dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${PDQSORT_INCLUDE_DIR}) if (NOT USE_INTERNAL_LZ4_LIBRARY AND LZ4_INCLUDE_DIR) From d1024e76bd02148122126f0b810d833d54ec4510 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 13 Apr 2020 17:09:56 +0300 Subject: [PATCH 205/743] fix --- docker/test/performance-comparison/compare.sh | 6 +++--- programs/copier/ClusterCopierApp.cpp | 2 +- programs/local/LocalServer.cpp | 10 +++++----- src/Databases/DatabaseAtomic.cpp | 8 +++++++- src/Databases/DatabaseAtomic.h | 2 ++ src/Databases/DatabaseFactory.cpp | 2 +- src/Databases/DatabaseMemory.cpp | 11 ++++++++++- src/Databases/DatabaseMemory.h | 5 ++++- src/Databases/IDatabase.h | 2 ++ src/Interpreters/Context.cpp | 14 ++++++++++---- src/Interpreters/DatabaseCatalog.cpp | 11 ++++++++++- src/Interpreters/DatabaseCatalog.h | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 5 +++-- src/Interpreters/tests/expression_analyzer.cpp | 2 +- .../00693_max_block_size_system_tables_columns.sql | 4 +++- tests/queries/0_stateless/01004_rename_deadlock.sh | 1 + 16 files changed, 64 insertions(+), 23 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index f89631522f4..ccb8f77b8f0 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -309,7 +309,7 @@ create view addresses_src as select * from file('$version-addresses.tsv', TSVWithNamesAndTypes, '$(cat "$version-addresses.tsv.columns")'); -create table addresses_join engine Join(any, left, address) as +create table addresses_join_$version engine Join(any, left, address) as select addr address, name from addresses_src; create table unstable_query_runs engine File(TSVWithNamesAndTypes, @@ -348,7 +348,7 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes, 'unstable-run-traces.$version.rep') as select count() value, - joinGet(addresses_join, 'name', arrayJoin(trace)) metric, + joinGet(addresses_join_$version, 'name', arrayJoin(trace)) metric, unstable_query_runs.query_id, any(unstable_query_runs.query) query from unstable_query_runs @@ -374,7 +374,7 @@ create table stacks engine File(TSV, 'stacks.$version.rep') as select query, arrayStringConcat( - arrayMap(x -> joinGet(addresses_join, 'name', x), + arrayMap(x -> joinGet(addresses_join_$version, 'name', x), arrayReverse(trace) ), ';' diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 14bfd3a3339..e286bcb489c 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -110,7 +110,7 @@ void ClusterCopierApp::mainImpl() registerDisks(); static const std::string default_database = "_local"; - DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database)); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *context)); context->setCurrentDatabase(default_database); /// Initialize query scope just in case. diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 26752da5d87..8a2943a2df5 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -119,13 +119,13 @@ void LocalServer::tryInitPath() } -static void attachSystemTables() +static void attachSystemTables(const Context & context) { DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE); if (!system_database) { /// TODO: add attachTableDelayed into DatabaseMemory to speedup loading - system_database = std::make_shared(DatabaseCatalog::SYSTEM_DATABASE); + system_database = std::make_shared(DatabaseCatalog::SYSTEM_DATABASE, context); DatabaseCatalog::instance().attachDatabase(DatabaseCatalog::SYSTEM_DATABASE, system_database); } @@ -202,7 +202,7 @@ try * 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)); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *context)); context->setCurrentDatabase(default_database); applyCmdOptions(); @@ -213,14 +213,14 @@ try LOG_DEBUG(log, "Loading metadata from " << context->getPath()); loadMetadataSystem(*context); - attachSystemTables(); + attachSystemTables(*context); loadMetadata(*context); DatabaseCatalog::instance().loadDatabases(); LOG_DEBUG(log, "Loaded metadata."); } else { - attachSystemTables(); + attachSystemTables(*context); } processQueries(); diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 2620ce94b6d..539d2695e13 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -51,7 +51,6 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const auto tmp = data_path + DatabaseCatalog::getPathForUUID(query.uuid); assert(tmp != data_path && !tmp.empty()); return tmp; - } void DatabaseAtomic::drop(const Context &) @@ -285,6 +284,13 @@ DatabaseTablesIteratorPtr DatabaseAtomic::getTablesWithDictionaryTablesIterator( return std::make_unique(std::move(typeid_cast(*base_iter))); } +UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const +{ + if (auto table = tryGetTable(global_context, table_name)) + return table->getStorageID().uuid; + return UUIDHelpers::Nil; +} + void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) { /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 714976d18b9..5da071d2051 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -40,6 +40,8 @@ public: void assertCanBeDetached(bool cleenup); + UUID tryGetTableUUID(const String & table_name) const override; + private: void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 83448edfd0f..ae213d8c230 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -79,7 +79,7 @@ DatabasePtr DatabaseFactory::getImpl( else if (engine_name == "Atomic") return std::make_shared(database_name, metadata_path, context); else if (engine_name == "Memory") - return std::make_shared(database_name); + return std::make_shared(database_name, context); else if (engine_name == "Dictionary") return std::make_shared(database_name, context); diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 9ee5b45f064..aab35184dec 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -14,8 +15,9 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -DatabaseMemory::DatabaseMemory(const String & name_) +DatabaseMemory::DatabaseMemory(const String & name_, const Context & global_context_) : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")") + , global_context(global_context_.getGlobalContext()) , data_path("data/" + escapeForFileName(database_name) + "/") {} @@ -76,4 +78,11 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const Context &, const String & t return it->second; } +UUID DatabaseMemory::tryGetTableUUID(const String & table_name) const +{ + if (auto table = tryGetTable(global_context, table_name)) + return table->getStorageID().uuid; + return UUIDHelpers::Nil; +} + } diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index 5add69c824f..2740867878c 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -19,7 +19,7 @@ namespace DB class DatabaseMemory final : public DatabaseWithOwnTablesBase { public: - DatabaseMemory(const String & name_); + DatabaseMemory(const String & name_, const Context & global_context_); String getEngineName() const override { return "Memory"; } @@ -44,7 +44,10 @@ public: String getTableDataPath(const String & table_name) const override { return data_path + escapeForFileName(table_name) + "/"; } String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.table); } + UUID tryGetTableUUID(const String & table_name) const override; + private: + const Context & global_context; String data_path; using NameToASTCreate = std::unordered_map; NameToASTCreate create_queries; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index d9626fac934..648ff0fe3b1 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -141,6 +141,8 @@ public: const Context & context, const String & name) const = 0; + virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; } + using FilterByNameFunction = std::function; /// Get an iterator that allows you to pass through all the tables. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index bb25a1e630a..f340b3ad94c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2037,7 +2037,9 @@ StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where } if (exc) throw Exception(*exc); - return DatabaseCatalog::instance().getTable(resolved)->getStorageID(); + if (!resolved.hasUUID() && resolved.database_name != DatabaseCatalog::TEMPORARY_DATABASE) + resolved.uuid = DatabaseCatalog::instance().getDatabase(resolved.database_name)->tryGetTableUUID(resolved.table_name); + return resolved; } StorageID Context::tryResolveStorageID(StorageID storage_id, StorageNamespace where) const @@ -2050,9 +2052,13 @@ StorageID Context::tryResolveStorageID(StorageID storage_id, StorageNamespace wh auto lock = getLock(); resolved = resolveStorageIDImpl(std::move(storage_id), where, nullptr); } - if (auto table = DatabaseCatalog::instance().tryGetTable(resolved)) - return table->getStorageID(); - return StorageID::createEmpty(); + if (resolved && !resolved.hasUUID() && resolved.database_name != DatabaseCatalog::TEMPORARY_DATABASE) + { + auto db = DatabaseCatalog::instance().tryGetDatabase(resolved.database_name); + if (db) + resolved.uuid = db->tryGetTableUUID(resolved.table_name); + } + return resolved; } StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace where, std::optional * exception) const diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 99c7201993e..77aa7186086 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -108,7 +108,7 @@ void DatabaseCatalog::loadDatabases() { drop_delay_s = global_context->getConfigRef().getInt("database_atomic_delay_before_drop_table_s", 60); - auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE); + auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, *global_context); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); loadMarkedAsDroppedTables(); @@ -483,6 +483,15 @@ StoragePtr DatabaseCatalog::tryGetTable(const StorageID & table_id) const return getTableImpl(table_id, *global_context, nullptr).second; } +DatabaseAndTable DatabaseCatalog::getDatabaseAndTable(const StorageID & table_id) const +{ + std::optional exc; + auto res = getTableImpl(table_id, *global_context, &exc); + if (!res.second) + throw Exception(*exc); + return res; +} + DatabaseAndTable DatabaseCatalog::tryGetDatabaseAndTable(const StorageID & table_id) const { return getTableImpl(table_id, *global_context, nullptr); diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 0838a80ec0c..ee9f932521a 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -135,7 +135,7 @@ public: StoragePtr getTable(const StorageID & table_id) const; StoragePtr tryGetTable(const StorageID & table_id) const; - DatabaseAndTable getDatabaseAndTable(const StorageID & table_id) const { return getTableImpl(table_id, *global_context); } + DatabaseAndTable getDatabaseAndTable(const StorageID & table_id) const; DatabaseAndTable tryGetDatabaseAndTable(const StorageID & table_id) const; DatabaseAndTable getTableImpl(const StorageID & table_id, const Context & local_context, std::optional * exception = nullptr) const; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index c5bb3b5cf8b..ce925cef086 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -73,7 +73,6 @@ BlockIO InterpreterDropQuery::executeToTable( ErrorCodes::UNKNOWN_TABLE); } - /// If table was already dropped by anyone, an exception will be thrown auto table_id = query.if_exists ? context.tryResolveStorageID(table_id_, Context::ResolveOrdinary) : context.resolveStorageID(table_id_, Context::ResolveOrdinary); if (!table_id) @@ -81,7 +80,9 @@ BlockIO InterpreterDropQuery::executeToTable( auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); - auto [database, table] = DatabaseCatalog::instance().getDatabaseAndTable(table_id); + /// If table was already dropped by anyone, an exception will be thrown + auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id) + : DatabaseCatalog::instance().getDatabaseAndTable(table_id); if (database && table) { diff --git a/src/Interpreters/tests/expression_analyzer.cpp b/src/Interpreters/tests/expression_analyzer.cpp index 349c0ac7bf4..961831933b7 100644 --- a/src/Interpreters/tests/expression_analyzer.cpp +++ b/src/Interpreters/tests/expression_analyzer.cpp @@ -98,7 +98,7 @@ int main() Context context = Context::createGlobal(); context.makeGlobalContext(); - auto system_database = std::make_shared("system"); + auto system_database = std::make_shared("system", context); DatabaseCatalog::instance().attachDatabase("system", system_database); //context.setCurrentDatabase("system"); system_database->attachTable("one", StorageSystemOne::create("one"), {}); diff --git a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql index 8c1396381ad..6f7ccc29892 100644 --- a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql +++ b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql @@ -5,7 +5,9 @@ SELECT (SELECT count() FROM system.tables SETTINGS max_block_size = 100) = (SELE DROP TEMPORARY TABLE IF EXISTS t_00693; CREATE TEMPORARY TABLE t_00693 (x UInt8); -SELECT * FROM system.tables WHERE is_temporary AND name='t_00693'; +SELECT database, name, engine, is_temporary, data_paths, metadata_path, metadata_modification_time, dependencies_database, + dependencies_table, create_table_query, engine_full, partition_key, sorting_key, primary_key, sampling_key, storage_policy, + total_rows, total_bytes FROM system.tables WHERE is_temporary AND name='t_00693'; SELECT avg(blockSize()) <= 10000 FROM system.columns SETTINGS max_block_size = 10; SELECT avg(blockSize()) <= 10000 FROM system.columns LIMIT 10 SETTINGS max_block_size = 10; diff --git a/tests/queries/0_stateless/01004_rename_deadlock.sh b/tests/queries/0_stateless/01004_rename_deadlock.sh index 5d5726bb001..b4063dd7d00 100755 --- a/tests/queries/0_stateless/01004_rename_deadlock.sh +++ b/tests/queries/0_stateless/01004_rename_deadlock.sh @@ -55,6 +55,7 @@ timeout $TIMEOUT bash -c thread2 2> /dev/null & timeout $TIMEOUT bash -c thread3 2> /dev/null & wait +sleep 1 $CLICKHOUSE_CLIENT -q "DROP TABLE test1" $CLICKHOUSE_CLIENT -q "DROP TABLE test2" From 871d3f2aa0c77455cf31c47f40a1ad4904304403 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 13 Apr 2020 18:55:48 +0300 Subject: [PATCH 206/743] =?UTF-8?q?Move=20contrib/widecharwidth=20?= =?UTF-8?q?=E2=86=92=20base/widechar=5Fwidth?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- base/CMakeLists.txt | 1 + {contrib/widecharwidth => base/widechar_width}/CMakeLists.txt | 0 {contrib/widecharwidth => base/widechar_width}/LICENSE | 0 {contrib/widecharwidth => base/widechar_width}/README.md | 0 .../widecharwidth => base/widechar_width}/widechar_width.cpp | 0 {contrib/widecharwidth => base/widechar_width}/widechar_width.h | 0 contrib/CMakeLists.txt | 1 - 7 files changed, 1 insertion(+), 1 deletion(-) rename {contrib/widecharwidth => base/widechar_width}/CMakeLists.txt (100%) rename {contrib/widecharwidth => base/widechar_width}/LICENSE (100%) rename {contrib/widecharwidth => base/widechar_width}/README.md (100%) rename {contrib/widecharwidth => base/widechar_width}/widechar_width.cpp (100%) rename {contrib/widecharwidth => base/widechar_width}/widechar_width.h (100%) diff --git a/base/CMakeLists.txt b/base/CMakeLists.txt index 1ead921f7e4..cfa54fe2ca4 100644 --- a/base/CMakeLists.txt +++ b/base/CMakeLists.txt @@ -6,6 +6,7 @@ add_subdirectory (common) add_subdirectory (daemon) add_subdirectory (loggers) add_subdirectory (pcg-random) +add_subdirectory (widechar_width) if (USE_MYSQL) add_subdirectory (mysqlxx) diff --git a/contrib/widecharwidth/CMakeLists.txt b/base/widechar_width/CMakeLists.txt similarity index 100% rename from contrib/widecharwidth/CMakeLists.txt rename to base/widechar_width/CMakeLists.txt diff --git a/contrib/widecharwidth/LICENSE b/base/widechar_width/LICENSE similarity index 100% rename from contrib/widecharwidth/LICENSE rename to base/widechar_width/LICENSE diff --git a/contrib/widecharwidth/README.md b/base/widechar_width/README.md similarity index 100% rename from contrib/widecharwidth/README.md rename to base/widechar_width/README.md diff --git a/contrib/widecharwidth/widechar_width.cpp b/base/widechar_width/widechar_width.cpp similarity index 100% rename from contrib/widecharwidth/widechar_width.cpp rename to base/widechar_width/widechar_width.cpp diff --git a/contrib/widecharwidth/widechar_width.h b/base/widechar_width/widechar_width.h similarity index 100% rename from contrib/widecharwidth/widechar_width.h rename to base/widechar_width/widechar_width.h diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 9c0a0aac0dd..1d438e29dae 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -333,6 +333,5 @@ add_subdirectory(grpc-cmake) add_subdirectory(replxx-cmake) add_subdirectory(FastMemcpy) -add_subdirectory(widecharwidth) add_subdirectory(consistent-hashing) add_subdirectory(consistent-hashing-sumbur) From 68e82c11e07e9f6d95d68aa8b8c2a288be78bac6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Apr 2020 18:58:29 +0300 Subject: [PATCH 207/743] fix build --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 3e9f476e5ec..2eee24be477 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1808,7 +1808,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart(const MergeTreeData:: if (containing_part != part->name) { if (out_reason) - *out_reason = "Part " + part->name >+ " has already been assigned a merge into " + containing_part; + *out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; return false; } From 0b7ae2c2bf79b7da1a7977f8ca761e108d224fd5 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Mon, 13 Apr 2020 20:03:11 +0300 Subject: [PATCH 208/743] working version --- src/Interpreters/HashJoin.cpp | 19 ++-- src/Interpreters/TableJoin.cpp | 32 +++--- src/Interpreters/TableJoin.h | 2 + .../0_stateless/00561_storage_join.sql | 6 +- .../01115_join_with_dictionary.reference | 103 ++++++++++++++++++ .../01115_join_with_dictionary.sql | 90 +++++++++++++++ .../00065_loyalty_with_storage_join.sql | 11 +- 7 files changed, 231 insertions(+), 32 deletions(-) create mode 100644 tests/queries/0_stateless/01115_join_with_dictionary.reference create mode 100644 tests/queries/0_stateless/01115_join_with_dictionary.sql diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 1c0adb96f66..f58efa1920f 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -295,17 +295,22 @@ public: : key_columns(key_columns_) {} - FindResult findKey(const DictionaryReader & reader, size_t i, const Arena &) + FindResult findKey(const TableJoin & table_join, size_t row, const Arena &) { + const DictionaryReader & reader = *table_join.dictionary_reader; if (!read_result) { reader.readKeys(*key_columns[0], key_columns[0]->size(), read_result, found, positions); result.block = &read_result; - /// TODO: check types and correct nullability + + if (table_join.forceNullableRight()) + for (auto & column : read_result) + if (table_join.rightBecomeNullable(column.type)) + JoinCommon::convertColumnToNullable(column); } - result.row_num = positions[i]; - return FindResult(&result, found[i]); + result.row_num = positions[row]; + return FindResult(&result, found[row]); } private: @@ -985,14 +990,14 @@ IColumn::Filter switchJoinRightColumns(const Maps & maps_, AddedColumns & added_ } template -IColumn::Filter dictionaryJoinRightColumns(const DictionaryReader & reader, AddedColumns & added_columns, const ConstNullMapPtr & null_map) +IColumn::Filter dictionaryJoinRightColumns(const TableJoin & table_join, AddedColumns & added_columns, const ConstNullMapPtr & null_map) { if constexpr (KIND == ASTTableJoin::Kind::Left && (STRICTNESS == ASTTableJoin::Strictness::Any || STRICTNESS == ASTTableJoin::Strictness::Semi || STRICTNESS == ASTTableJoin::Strictness::Anti)) { - return joinRightColumnsSwitchNullability(reader, added_columns, null_map); + return joinRightColumnsSwitchNullability(table_join, added_columns, null_map); } throw Exception("Logical error: wrong JOIN combination", ErrorCodes::LOGICAL_ERROR); @@ -1059,7 +1064,7 @@ void HashJoin::joinBlockImpl( added_columns.need_filter = need_filter || has_required_right_keys; IColumn::Filter row_filter = overDictionary() ? - dictionaryJoinRightColumns(*table_join->dictionary_reader, added_columns, null_map) : + dictionaryJoinRightColumns(*table_join, added_columns, null_map) : switchJoinRightColumns(maps_, added_columns, data->type, null_map); for (size_t i = 0; i < added_columns.size(); ++i) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 5e57c740bf1..4c75b23a5eb 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -155,22 +155,26 @@ NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names return getNamesWithAliases(required_columns); } +bool TableJoin::leftBecomeNullable(const DataTypePtr & column_type) const +{ + return forceNullableLeft() && column_type->canBeInsideNullable(); +} + +bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const +{ + return forceNullableRight() && column_type->canBeInsideNullable(); +} + void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { - if (join_use_nulls && isLeftOrFull(table_join.kind)) - { - auto type = joined_column.type->canBeInsideNullable() ? makeNullable(joined_column.type) : joined_column.type; - columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, std::move(type))); - } + if (rightBecomeNullable(joined_column.type)) + columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, makeNullable(joined_column.type))); else columns_added_by_join.push_back(joined_column); } void TableJoin::addJoinedColumnsAndCorrectNullability(Block & sample_block) const { - bool right_or_full_join = isRightOrFull(table_join.kind); - bool left_or_full_join = isLeftOrFull(table_join.kind); - for (auto & col : sample_block) { /// Materialize column. @@ -179,9 +183,7 @@ void TableJoin::addJoinedColumnsAndCorrectNullability(Block & sample_block) cons if (col.column) col.column = nullptr; - bool make_nullable = join_use_nulls && right_or_full_join; - - if (make_nullable && col.type->canBeInsideNullable()) + if (leftBecomeNullable(col.type)) col.type = makeNullable(col.type); } @@ -189,9 +191,7 @@ void TableJoin::addJoinedColumnsAndCorrectNullability(Block & sample_block) cons { auto res_type = col.type; - bool make_nullable = join_use_nulls && left_or_full_join; - - if (make_nullable && res_type->canBeInsideNullable()) + if (rightBecomeNullable(res_type)) res_type = makeNullable(res_type); sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name)); @@ -240,6 +240,10 @@ bool TableJoin::allowMergeJoin() const bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_block, Names & names, NamesAndTypesList & result_columns) const { + /// Support ALL INNER, [ANY | ALL | SEMI | ANTI] LEFT + if (!isLeft(kind()) && !(isInner(kind()) && strictness() == ASTTableJoin::Strictness::All)) + return false; + const Names & right_keys = keyNamesRight(); if (right_keys.size() != 1) return false; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index c8c51918e27..2047f935966 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -122,6 +122,8 @@ public: size_t rightKeyInclusion(const String & name) const; NameSet requiredRightKeys() const; + bool leftBecomeNullable(const DataTypePtr & column_type) const; + bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); void addJoinedColumnsAndCorrectNullability(Block & sample_block) const; diff --git a/tests/queries/0_stateless/00561_storage_join.sql b/tests/queries/0_stateless/00561_storage_join.sql index 08f76815702..62ca80d31fe 100644 --- a/tests/queries/0_stateless/00561_storage_join.sql +++ b/tests/queries/0_stateless/00561_storage_join.sql @@ -1,5 +1,3 @@ -SET any_join_distinct_right_table_keys = 1; - drop table IF EXISTS joinbug; CREATE TABLE joinbug ( @@ -21,7 +19,7 @@ CREATE TABLE joinbug_join ( val UInt64, val2 Int32, created UInt64 -) ENGINE = Join(ANY, INNER, id2); +) ENGINE = Join(SEMI, LEFT, id2); insert into joinbug_join (id, id2, val, val2, created) select id, id2, val, val2, created @@ -36,7 +34,7 @@ select id, id2, val, val2, created from ( SELECT toUInt64(arrayJoin(range(50))) AS id2 ) js1 -ANY INNER JOIN joinbug_join using id2; +SEMI LEFT JOIN joinbug_join using id2; DROP TABLE joinbug; DROP TABLE joinbug_join; diff --git a/tests/queries/0_stateless/01115_join_with_dictionary.reference b/tests/queries/0_stateless/01115_join_with_dictionary.reference new file mode 100644 index 00000000000..f909a3d61f5 --- /dev/null +++ b/tests/queries/0_stateless/01115_join_with_dictionary.reference @@ -0,0 +1,103 @@ +flat: left on +0 0 0 0 0 +1 1 1 1 1 +2 2 2 2 2 +3 3 3 3 3 +4 0 0 0 +flat: left +0 0 0 0 +1 1 1 1 +2 2 2 2 +3 3 3 3 +4 0 0 +flat: any left +0 0 0 0 +1 1 1 1 +2 2 2 2 +3 3 3 3 +4 0 0 +flat: semi left +0 0 0 0 +1 1 1 1 +2 2 2 2 +3 3 3 3 +flat: anti left +4 0 0 +flat: inner +0 0 0 0 +1 1 1 1 +flat: inner on +0 0 0 0 0 +1 1 1 1 1 +2 2 2 2 2 +3 3 3 3 3 +hashed: left on +0 0 0 0 0 +1 1 1 1 1 +2 2 2 2 2 +3 3 3 3 3 +4 \N \N \N \N +hashed: left +0 0 0 0 +1 1 1 1 +2 2 2 2 +3 3 3 3 +4 \N \N \N +hashed: any left +0 0 0 0 +1 1 1 1 +2 2 2 2 +3 3 3 3 +4 \N \N \N +hashed: semi left +0 0 0 0 +1 1 1 1 +2 2 2 2 +3 3 3 3 +hashed: anti left +4 \N \N \N +hashed: inner +0 0 0 0 +1 1 1 1 +hashed: inner on +0 0 0 0 0 +1 1 1 1 1 +2 2 2 2 2 +3 3 3 3 3 +complex_cache (smoke) +0 \N \N \N \N +1 \N \N \N \N +2 \N \N \N \N +3 \N \N \N \N +4 \N \N \N \N +not optimized (smoke) +0 0 0 0 +1 1 1 1 +2 2 2 2 +3 3 3 3 +- +0 0 0 0 0 +1 1 1 1 1 +\N 2 2 2 2 +\N 3 3 3 3 +- +2 2 2 2 +3 3 3 3 +4 \N \N \N +5 \N \N \N +\N 0 0 0 +\N 1 1 1 +- +0 0 0 0 +1 1 1 1 +- +0 0 0 0 +1 1 1 1 +3 3 3 3 +2 2 2 2 +- +0 0 0 0 +1 1 1 1 +- +3 3 3 3 +2 2 2 2 diff --git a/tests/queries/0_stateless/01115_join_with_dictionary.sql b/tests/queries/0_stateless/01115_join_with_dictionary.sql new file mode 100644 index 00000000000..65704f2b3eb --- /dev/null +++ b/tests/queries/0_stateless/01115_join_with_dictionary.sql @@ -0,0 +1,90 @@ +SET send_logs_level = 'none'; + +DROP DATABASE IF EXISTS db_01115; +CREATE DATABASE db_01115 Engine = Ordinary; + +USE db_01115; + +DROP DICTIONARY IF EXISTS dict_flat; +DROP DICTIONARY IF EXISTS dict_hashed; +DROP DICTIONARY IF EXISTS dict_complex_cache; + +CREATE TABLE t1 (key UInt64, a UInt8, b String, c Float64) ENGINE = MergeTree() ORDER BY key; +INSERT INTO t1 SELECT number, number, toString(number), number from numbers(4); + +CREATE DICTIONARY dict_flat (key UInt64 DEFAULT 0, a UInt8 DEFAULT 42, b String DEFAULT 'x', c Float64 DEFAULT 42.0) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 't1' PASSWORD '' DB 'db_01115')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +CREATE DICTIONARY db_01115.dict_hashed (key UInt64 DEFAULT 0, a UInt8 DEFAULT 42, b String DEFAULT 'x', c Float64 DEFAULT 42.0) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 't1' DB 'db_01115')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(HASHED()); + +CREATE DICTIONARY dict_complex_cache (key UInt64 DEFAULT 0, a UInt8 DEFAULT 42, b String DEFAULT 'x', c Float64 DEFAULT 42.0) +PRIMARY KEY key, b +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 't1' DB 'db_01115')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 1)); + +SET join_use_nulls = 0; + +SELECT 'flat: left on'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 LEFT JOIN dict_flat d ON s1.key = d.key ORDER BY s1.key; +SELECT 'flat: left'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 LEFT JOIN dict_flat d USING(key) ORDER BY key; +SELECT 'flat: any left'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 ANY LEFT JOIN dict_flat d USING(key) ORDER BY key; +SELECT 'flat: semi left'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 SEMI JOIN dict_flat d USING(key) ORDER BY key; +SELECT 'flat: anti left'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 ANTI JOIN dict_flat d USING(key) ORDER BY key; +SELECT 'flat: inner'; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 JOIN dict_flat d USING(key); +SELECT 'flat: inner on'; +SELECT * FROM (SELECT number AS k FROM numbers(100)) s1 JOIN dict_flat d ON k = key ORDER BY k; + +SET join_use_nulls = 1; + +SELECT 'hashed: left on'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 LEFT JOIN dict_hashed d ON s1.key = d.key ORDER BY s1.key; +SELECT 'hashed: left'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 LEFT JOIN dict_hashed d USING(key) ORDER BY key; +SELECT 'hashed: any left'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 ANY LEFT JOIN dict_hashed d USING(key) ORDER BY key; +SELECT 'hashed: semi left'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 SEMI JOIN dict_hashed d USING(key) ORDER BY key; +SELECT 'hashed: anti left'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 ANTI JOIN dict_hashed d USING(key) ORDER BY key; +SELECT 'hashed: inner'; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 JOIN dict_hashed d USING(key); +SELECT 'hashed: inner on'; +SELECT * FROM (SELECT number AS k FROM numbers(100)) s1 JOIN dict_hashed d ON k = key ORDER BY k; + +SELECT 'complex_cache (smoke)'; +SELECT * FROM (SELECT number AS key FROM numbers(5)) s1 LEFT JOIN dict_complex_cache d ON s1.key = d.key ORDER BY s1.key; + +SELECT 'not optimized (smoke)'; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 RIGHT JOIN dict_flat d USING(key) ORDER BY key; +SELECT '-'; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 RIGHT JOIN dict_flat d ON s1.key = d.key ORDER BY d.key; +SELECT '-'; +SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s1 FULL JOIN dict_flat d USING(key) ORDER BY s1.key, d.key; +SELECT '-'; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY INNER JOIN dict_flat d USING(key); +SELECT '-'; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY RIGHT JOIN dict_flat d USING(key); +SELECT '-'; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 SEMI RIGHT JOIN dict_flat d USING(key); +SELECT '-'; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANTI RIGHT JOIN dict_flat d USING(key); + +DROP DICTIONARY dict_flat; +DROP DICTIONARY dict_hashed; +DROP DICTIONARY dict_complex_cache; + +DROP TABLE t1; +DROP DATABASE IF EXISTS db_01115; diff --git a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index 2fa337b6ebc..15a2a75cf58 100644 --- a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -1,10 +1,7 @@ -SET any_join_distinct_right_table_keys = 1; - USE test; DROP TABLE IF EXISTS join; -CREATE TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID) -SETTINGS any_join_distinct_right_table_keys = 1; +CREATE TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(SEMI, LEFT, UserID); INSERT INTO join SELECT @@ -20,17 +17,17 @@ HAVING (yandex + google) > 10; SELECT loyalty, count() -FROM hits ANY INNER JOIN join USING UserID +FROM hits SEMI LEFT JOIN join USING UserID GROUP BY loyalty ORDER BY loyalty ASC; DETACH TABLE join; -ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); +ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(SEMI, LEFT, UserID); SELECT loyalty, count() -FROM hits ANY INNER JOIN join USING UserID +FROM hits SEMI LEFT JOIN join USING UserID GROUP BY loyalty ORDER BY loyalty ASC; From df01227bd24ead0d7568c47ef122ae3299323c76 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Mon, 13 Apr 2020 20:36:07 +0300 Subject: [PATCH 209/743] move DictionaryReader code out of header --- src/Interpreters/DictionaryReader.cpp | 167 ++++++++++++++++++++++++++ src/Interpreters/DictionaryReader.h | 158 ++---------------------- src/Interpreters/HashJoin.cpp | 2 +- 3 files changed, 176 insertions(+), 151 deletions(-) create mode 100644 src/Interpreters/DictionaryReader.cpp diff --git a/src/Interpreters/DictionaryReader.cpp b/src/Interpreters/DictionaryReader.cpp new file mode 100644 index 00000000000..301fe9d57c6 --- /dev/null +++ b/src/Interpreters/DictionaryReader.cpp @@ -0,0 +1,167 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; + extern const int TYPE_MISMATCH; +} + + +DictionaryReader::FunctionWrapper::FunctionWrapper(FunctionOverloadResolverPtr resolver, const ColumnsWithTypeAndName & arguments, + Block & block, const ColumnNumbers & arg_positions_, const String & column_name, + TypeIndex expected_type) + : arg_positions(arg_positions_) + , result_pos(block.columns()) +{ + FunctionBasePtr prepared_function = resolver->build(arguments); + + ColumnWithTypeAndName result; + result.name = "get_" + column_name; + result.type = prepared_function->getReturnType(); + if (result.type->getTypeId() != expected_type) + throw Exception("Type mismatch in dictionary reader for: " + column_name, ErrorCodes::TYPE_MISMATCH); + block.insert(result); + + function = prepared_function->prepare(block, arg_positions, result_pos); +} + +static constexpr const size_t key_size = 1; + +DictionaryReader::DictionaryReader(const String & dictionary_name, const Names & src_column_names, const NamesAndTypesList & result_columns, + const Context & context) + : result_header(makeResultBlock(result_columns)) + , key_position(key_size + result_header.columns()) +{ + if (src_column_names.size() != result_columns.size()) + throw Exception("Columns number mismatch in dictionary reader", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); + + ColumnWithTypeAndName dict_name; + ColumnWithTypeAndName key; + ColumnWithTypeAndName column_name; + + { + dict_name.name = "dict"; + dict_name.type = std::make_shared(); + dict_name.column = dict_name.type->createColumnConst(1, dictionary_name); + + /// TODO: composite key (key_size > 1) + key.name = "key"; + key.type = std::make_shared(); + + column_name.name = "column"; + column_name.type = std::make_shared(); + } + + /// dictHas('dict_name', id) + ColumnsWithTypeAndName arguments_has; + arguments_has.push_back(dict_name); + arguments_has.push_back(key); + + /// dictGet('dict_name', 'attr_name', id) + ColumnsWithTypeAndName arguments_get; + arguments_get.push_back(dict_name); + arguments_get.push_back(column_name); + arguments_get.push_back(key); + + sample_block.insert(dict_name); + + for (auto & columns_name : src_column_names) + { + ColumnWithTypeAndName name; + name.name = "col_" + columns_name; + name.type = std::make_shared(); + name.column = name.type->createColumnConst(1, columns_name); + + sample_block.insert(name); + } + + sample_block.insert(key); + + ColumnNumbers positions_has{0, key_position}; + function_has = std::make_unique(FunctionFactory::instance().get("dictHas", context), + arguments_has, sample_block, positions_has, "has", DataTypeUInt8().getTypeId()); + functions_get.reserve(result_header.columns()); + + for (size_t i = 0; i < result_header.columns(); ++i) + { + size_t column_name_pos = key_size + i; + auto & column = result_header.getByPosition(i); + arguments_get[1].column = DataTypeString().createColumnConst(1, src_column_names[i]); + ColumnNumbers positions_get{0, column_name_pos, key_position}; + functions_get.emplace_back( + FunctionWrapper(FunctionFactory::instance().get("dictGet", context), + arguments_get, sample_block, positions_get, column.name, column.type->getTypeId())); + } +} + +void DictionaryReader::readKeys(const IColumn & keys, Block & out_block, ColumnVector::Container & found, + std::vector & positions) const +{ + Block working_block = sample_block; + size_t has_position = key_position + 1; + size_t size = keys.size(); + + /// set keys for dictHas() + ColumnWithTypeAndName & key_column = working_block.getByPosition(key_position); + key_column.column = keys.cloneResized(size); /// just a copy we cannot avoid + + /// calculate and extract dictHas() + function_has->execute(working_block, size); + ColumnWithTypeAndName & has_column = working_block.getByPosition(has_position); + auto mutable_has = (*std::move(has_column.column)).mutate(); + found.swap(typeid_cast &>(*mutable_has).getData()); + has_column.column = nullptr; + + /// set mapping form source keys to resulting rows in output block + positions.clear(); + positions.resize(size, 0); + size_t pos = 0; + for (size_t i = 0; i < size; ++i) + if (found[i]) + positions[i] = pos++; + + /// set keys for dictGet(): remove not found keys + key_column.column = key_column.column->filter(found, -1); + size_t rows = key_column.column->size(); + + /// calculate dictGet() + for (auto & func : functions_get) + func.execute(working_block, rows); + + /// make result: copy header block with correct names and move data columns + out_block = result_header.cloneEmpty(); + size_t first_get_position = has_position + 1; + for (size_t i = 0; i < out_block.columns(); ++i) + { + auto & src_column = working_block.getByPosition(first_get_position + i); + auto & dst_column = out_block.getByPosition(i); + dst_column.column = src_column.column; + src_column.column = nullptr; + } +} + +Block DictionaryReader::makeResultBlock(const NamesAndTypesList & names) +{ + Block block; + for (auto & nm : names) + { + ColumnWithTypeAndName column{nullptr, nm.type, nm.name}; + if (column.type->isNullable()) + column.type = typeid_cast(*column.type).getNestedType(); + block.insert(std::move(column)); + } + return block; +} + +} diff --git a/src/Interpreters/DictionaryReader.h b/src/Interpreters/DictionaryReader.h index 823a3690669..92e4924ae80 100644 --- a/src/Interpreters/DictionaryReader.h +++ b/src/Interpreters/DictionaryReader.h @@ -1,25 +1,16 @@ #pragma once -#include #include -#include -#include -#include #include -#include -#include -#include -#include +#include namespace DB { -namespace ErrorCodes -{ - extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; - extern const int TYPE_MISMATCH; -} +class Context; +/// Read block of required columns from Dictionary by UInt64 key column. Rename columns if needed. +/// Current implementation uses dictHas() + N * dictGet() functions. class DictionaryReader { public: @@ -30,21 +21,7 @@ public: size_t result_pos = 0; FunctionWrapper(FunctionOverloadResolverPtr resolver, const ColumnsWithTypeAndName & arguments, Block & block, - const ColumnNumbers & arg_positions_, const String & column_name, TypeIndex expected_type) - : arg_positions(arg_positions_) - , result_pos(block.columns()) - { - FunctionBasePtr prepared_function = resolver->build(arguments); - - ColumnWithTypeAndName result; - result.name = "get_" + column_name; - result.type = prepared_function->getReturnType(); - if (result.type->getTypeId() != expected_type) - throw Exception("Type mismatch in dictionary reader for: " + column_name, ErrorCodes::TYPE_MISMATCH); - block.insert(result); - - function = prepared_function->prepare(block, arg_positions, result_pos); - } + const ColumnNumbers & arg_positions_, const String & column_name, TypeIndex expected_type); void execute(Block & block, size_t rows) const { @@ -53,116 +30,8 @@ public: }; DictionaryReader(const String & dictionary_name, const Names & src_column_names, const NamesAndTypesList & result_columns, - const Context & context, size_t key_size = 1) - : result_header(makeResultBlock(result_columns)) - , key_position(key_size + result_header.columns()) - { - if (src_column_names.size() != result_columns.size()) - throw Exception("Columns number mismatch in dictionary reader", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); - - ColumnWithTypeAndName dict_name; - ColumnWithTypeAndName key; - ColumnWithTypeAndName column_name; - - { - dict_name.name = "dict"; - dict_name.type = std::make_shared(); - dict_name.column = dict_name.type->createColumnConst(1, dictionary_name); - - /// TODO: composite key (key_size > 1) - key.name = "key"; - key.type = std::make_shared(); - - column_name.name = "column"; - column_name.type = std::make_shared(); - } - - /// dictHas('dict_name', id) - ColumnsWithTypeAndName arguments_has; - arguments_has.push_back(dict_name); - arguments_has.push_back(key); - - /// dictGet('dict_name', 'attr_name', id) - ColumnsWithTypeAndName arguments_get; - arguments_get.push_back(dict_name); - arguments_get.push_back(column_name); - arguments_get.push_back(key); - - sample_block.insert(dict_name); - - for (auto & columns_name : src_column_names) - { - ColumnWithTypeAndName name; - name.name = "col_" + columns_name; - name.type = std::make_shared(); - name.column = name.type->createColumnConst(1, columns_name); - - sample_block.insert(name); - } - - sample_block.insert(key); - - ColumnNumbers positions_has{0, key_position}; - function_has = std::make_unique(FunctionFactory::instance().get("dictHas", context), - arguments_has, sample_block, positions_has, "has", DataTypeUInt8().getTypeId()); - functions_get.reserve(result_header.columns()); - - for (size_t i = 0; i < result_header.columns(); ++i) - { - size_t column_name_pos = key_size + i; - auto & column = result_header.getByPosition(i); - arguments_get[1].column = DataTypeString().createColumnConst(1, src_column_names[i]); - ColumnNumbers positions_get{0, column_name_pos, key_position}; - functions_get.emplace_back( - FunctionWrapper(FunctionFactory::instance().get("dictGet", context), - arguments_get, sample_block, positions_get, column.name, column.type->getTypeId())); - } - } - - void readKeys(const IColumn & keys, size_t size, Block & out_block, ColumnVector::Container & found, - std::vector & positions) const - { - Block working_block = sample_block; - size_t has_position = key_position + 1; - - /// set keys for dictHas() - ColumnWithTypeAndName & key_column = working_block.getByPosition(key_position); - key_column.column = keys.cloneResized(size); /// just a copy we cannot avoid - - /// calculate and extract dictHas() - function_has->execute(working_block, size); - ColumnWithTypeAndName & has_column = working_block.getByPosition(has_position); - auto mutable_has = (*std::move(has_column.column)).mutate(); - found.swap(typeid_cast &>(*mutable_has).getData()); - has_column.column = nullptr; - - /// set mapping form source keys to resulting rows in output block - positions.clear(); - positions.resize(size, 0); - size_t pos = 0; - for (size_t i = 0; i < size; ++i) - if (found[i]) - positions[i] = pos++; - - /// set keys for dictGet(): remove not found keys - key_column.column = key_column.column->filter(found, -1); - size_t rows = key_column.column->size(); - - /// calculate dictGet() - for (auto & func : functions_get) - func.execute(working_block, rows); - - /// make result: copy header block with correct names and move data columns - out_block = result_header.cloneEmpty(); - size_t first_get_position = has_position + 1; - for (size_t i = 0; i < out_block.columns(); ++i) - { - auto & src_column = working_block.getByPosition(first_get_position + i); - auto & dst_column = out_block.getByPosition(i); - dst_column.column = src_column.column; - src_column.column = nullptr; - } - } + const Context & context); + void readKeys(const IColumn & keys, Block & out_block, ColumnVector::Container & found, std::vector & positions) const; private: Block result_header; @@ -171,18 +40,7 @@ private: std::unique_ptr function_has; std::vector functions_get; - static Block makeResultBlock(const NamesAndTypesList & names) - { - Block block; - for (auto & nm : names) - { - ColumnWithTypeAndName column{nullptr, nm.type, nm.name}; - if (column.type->isNullable()) - column.type = typeid_cast(*column.type).getNestedType(); - block.insert(std::move(column)); - } - return block; - } + static Block makeResultBlock(const NamesAndTypesList & names); }; } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index f58efa1920f..22a8a87cbe0 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -300,7 +300,7 @@ public: const DictionaryReader & reader = *table_join.dictionary_reader; if (!read_result) { - reader.readKeys(*key_columns[0], key_columns[0]->size(), read_result, found, positions); + reader.readKeys(*key_columns[0], read_result, found, positions); result.block = &read_result; if (table_join.forceNullableRight()) From 0931885773600d35e029cfb936b2c9397f9f781a Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 13 Apr 2020 20:49:40 +0300 Subject: [PATCH 210/743] Update 01225_show_create_table_from_dictionary.sql --- .../0_stateless/01225_show_create_table_from_dictionary.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index 6c1ae2b907b..7550d5292d0 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -15,7 +15,7 @@ LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.dict` FORMAT TSVRaw; -SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 60; } +SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 36; } DROP DATABASE dict_db_01225; DROP DATABASE dict_db_01225_dictionary; From 586c295b944ed2c78c46f0ab64d78826093f5134 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 21:51:17 +0300 Subject: [PATCH 211/743] Added SummingSortedAlgorithm --- .../Merges/AggregatingSortedAlgorithm.cpp | 48 +- .../Merges/AggregatingSortedAlgorithm.h | 5 - .../Merges/SummingSortedAlgorithm.cpp | 594 +++++++++++++++++ .../Merges/SummingSortedAlgorithm.h | 152 +++++ .../Merges/SummingSortedTransform.cpp | 626 ------------------ .../Merges/SummingSortedTransform.h | 158 +---- 6 files changed, 786 insertions(+), 797 deletions(-) create mode 100644 src/Processors/Merges/SummingSortedAlgorithm.cpp create mode 100644 src/Processors/Merges/SummingSortedAlgorithm.h delete mode 100644 src/Processors/Merges/SummingSortedTransform.cpp diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp index e6138ecc6ab..85d45afd168 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp @@ -1,5 +1,7 @@ #include +#include +#include #include #include #include @@ -136,8 +138,9 @@ static MutableColumns getMergedColumns(const Block & header, const AggregatingSo for (auto & desc : def.columns_to_simple_aggregate) { - auto & type = header.getByPosition(desc.column_number).type; - columns[desc.column_number] = recursiveRemoveLowCardinality(type)->createColumn(); + auto & type = desc.nested_type ? desc.nested_type + : desc.real_type; + columns[desc.column_number] = type->createColumn(); } for (size_t i = 0; i < columns.size(); ++i) @@ -147,7 +150,8 @@ static MutableColumns getMergedColumns(const Block & header, const AggregatingSo return columns; } -static void prepareChunk(Chunk & chunk, const AggregatingSortedAlgorithm::ColumnsDefinition & def) +/// Remove constants and LowCardinality for SimpleAggregateFunction +static void preprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::ColumnsDefinition & def) { auto num_rows = chunk.getNumRows(); auto columns = chunk.detachColumns(); @@ -162,6 +166,25 @@ static void prepareChunk(Chunk & chunk, const AggregatingSortedAlgorithm::Column chunk.setColumns(std::move(columns), num_rows); } +/// Return back LowCardinality for SimpleAggregateFunction +static void postprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::ColumnsDefinition & def) +{ + size_t num_rows = chunk.getNumRows(); + auto columns_ = chunk.detachColumns(); + + for (auto & desc : def.columns_to_simple_aggregate) + { + if (desc.nested_type) + { + auto & from_type = desc.nested_type; + auto & to_type = desc.real_type; + columns_[desc.column_number] = recursiveTypeConversion(columns_[desc.column_number], from_type, to_type); + } + } + + chunk.setColumns(std::move(columns_), num_rows); +} + AggregatingSortedAlgorithm::AggregatingMergedData::AggregatingMergedData( MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_) @@ -226,21 +249,8 @@ Chunk AggregatingSortedAlgorithm::AggregatingMergedData::pull() throw Exception("Can't pull chunk because group was not finished.", ErrorCodes::LOGICAL_ERROR); auto chunk = MergedData::pull(); + postprocessChunk(chunk, def); - size_t num_rows = chunk.getNumRows(); - auto columns_ = chunk.detachColumns(); - - for (auto & desc : def.columns_to_simple_aggregate) - { - if (desc.nested_type) - { - auto & from_type = desc.nested_type; - auto & to_type = desc.real_type; - columns_[desc.column_number] = recursiveTypeConversion(columns_[desc.column_number], from_type, to_type); - } - } - - chunk.setColumns(std::move(columns_), num_rows); initAggregateDescription(); return chunk; @@ -269,14 +279,14 @@ void AggregatingSortedAlgorithm::initialize(Chunks chunks) { for (auto & chunk : chunks) if (chunk) - prepareChunk(chunk, columns_definition); + preprocessChunk(chunk, columns_definition); initializeQueue(std::move(chunks)); } void AggregatingSortedAlgorithm::consume(Chunk chunk, size_t source_num) { - prepareChunk(chunk, columns_definition); + preprocessChunk(chunk, columns_definition); updateCursor(std::move(chunk), source_num); } diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/AggregatingSortedAlgorithm.h index ad5644f7189..2457bb54e5e 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.h @@ -2,11 +2,6 @@ #include #include -#include -#include -#include -#include -#include namespace DB { diff --git a/src/Processors/Merges/SummingSortedAlgorithm.cpp b/src/Processors/Merges/SummingSortedAlgorithm.cpp new file mode 100644 index 00000000000..357ecc7114e --- /dev/null +++ b/src/Processors/Merges/SummingSortedAlgorithm.cpp @@ -0,0 +1,594 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int CORRUPTED_DATA; +} + +static bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number) +{ + for (auto & desc : description) + if (desc.column_name == name || (desc.column_name.empty() && desc.column_number == number)) + return true; + + return false; +} + +/// Returns true if merge result is not empty +static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row & row, SortCursor & cursor) +{ + /// Strongly non-optimal. + + Row & left = row; + Row right(left.size()); + + for (size_t col_num : desc.key_col_nums) + right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); + + for (size_t col_num : desc.val_col_nums) + right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); + + auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & + { + return matrix[i].get()[j]; + }; + + auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array + { + size_t size = col_nums.size(); + Array res(size); + for (size_t col_num_index = 0; col_num_index < size; ++col_num_index) + res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j); + return res; + }; + + std::map merged; + + auto accumulate = [](Array & dst, const Array & src) + { + bool has_non_zero = false; + size_t size = dst.size(); + for (size_t i = 0; i < size; ++i) + if (applyVisitor(FieldVisitorSum(src[i]), dst[i])) + has_non_zero = true; + return has_non_zero; + }; + + auto merge = [&](const Row & matrix) + { + size_t rows = matrix[desc.key_col_nums[0]].get().size(); + + for (size_t j = 0; j < rows; ++j) + { + Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j); + Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j); + + auto it = merged.find(key); + if (merged.end() == it) + merged.emplace(std::move(key), std::move(value)); + else + { + if (!accumulate(it->second, value)) + merged.erase(it); + } + } + }; + + merge(left); + merge(right); + + for (size_t col_num : desc.key_col_nums) + row[col_num] = Array(merged.size()); + for (size_t col_num : desc.val_col_nums) + row[col_num] = Array(merged.size()); + + size_t row_num = 0; + for (const auto & key_value : merged) + { + for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) + row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; + + for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) + row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; + + ++row_num; + } + + return row_num != 0; +} + +static SummingSortedAlgorithm::ColumnsDefinition defineColumns( + const Block & header, + const SortDescription & description, + const Names & column_names_to_sum) +{ + size_t num_columns = header.columns(); + SummingSortedAlgorithm::ColumnsDefinition def; + + /// name of nested structure -> the column numbers that refer to it. + std::unordered_map> discovered_maps; + + /** Fill in the column numbers, which must be summed. + * This can only be numeric columns that are not part of the sort key. + * If a non-empty column_names_to_sum is specified, then we only take these columns. + * Some columns from column_names_to_sum may not be found. This is ignored. + */ + for (size_t i = 0; i < num_columns; ++i) + { + const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + + /// Discover nested Maps and find columns for summation + if (typeid_cast(column.type.get())) + { + const auto map_name = Nested::extractTableName(column.name); + /// if nested table name ends with `Map` it is a possible candidate for special handling + if (map_name == column.name || !endsWith(map_name, "Map")) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + discovered_maps[map_name].emplace_back(i); + } + else + { + bool is_agg_func = WhichDataType(column.type).isAggregateFunction(); + + /// There are special const columns for example after prewhere sections. + if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column)) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + /// Are they inside the PK? + if (isInPrimaryKey(description, column.name, i)) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + + if (column_names_to_sum.empty() + || column_names_to_sum.end() != + std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name)) + { + // Create aggregator to sum this column + SummingSortedAlgorithm::AggregateDescription desc; + desc.is_agg_func_type = is_agg_func; + desc.column_numbers = {i}; + + if (!is_agg_func) + { + desc.init("sumWithOverflow", {column.type}); + } + + def.columns_to_aggregate.emplace_back(std::move(desc)); + } + else + { + // Column is not going to be summed, use last value + def.column_numbers_not_to_aggregate.push_back(i); + } + } + } + + /// select actual nested Maps from list of candidates + for (const auto & map : discovered_maps) + { + /// map should contain at least two elements (key -> value) + if (map.second.size() < 2) + { + for (auto col : map.second) + def.column_numbers_not_to_aggregate.push_back(col); + continue; + } + + /// no elements of map could be in primary key + auto column_num_it = map.second.begin(); + for (; column_num_it != map.second.end(); ++column_num_it) + if (isInPrimaryKey(description, header.safeGetByPosition(*column_num_it).name, *column_num_it)) + break; + if (column_num_it != map.second.end()) + { + for (auto col : map.second) + def.column_numbers_not_to_aggregate.push_back(col); + continue; + } + + DataTypes argument_types; + SummingSortedAlgorithm::AggregateDescription desc; + SummingSortedAlgorithm::MapDescription map_desc; + + column_num_it = map.second.begin(); + for (; column_num_it != map.second.end(); ++column_num_it) + { + const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it); + const String & name = key_col.name; + const IDataType & nested_type = *assert_cast(*key_col.type).getNestedType(); + + if (column_num_it == map.second.begin() + || endsWith(name, "ID") + || endsWith(name, "Key") + || endsWith(name, "Type")) + { + if (!nested_type.isValueRepresentedByInteger() && !isStringOrFixedString(nested_type)) + break; + + map_desc.key_col_nums.push_back(*column_num_it); + } + else + { + if (!nested_type.isSummable()) + break; + + map_desc.val_col_nums.push_back(*column_num_it); + } + + // Add column to function arguments + desc.column_numbers.push_back(*column_num_it); + argument_types.push_back(key_col.type); + } + + if (column_num_it != map.second.end()) + { + for (auto col : map.second) + def.column_numbers_not_to_aggregate.push_back(col); + continue; + } + + if (map_desc.key_col_nums.size() == 1) + { + // Create summation for all value columns in the map + desc.init("sumMapWithOverflow", argument_types); + def.columns_to_aggregate.emplace_back(std::move(desc)); + } + else + { + // Fall back to legacy mergeMaps for composite keys + for (auto col : map.second) + def.column_numbers_not_to_aggregate.push_back(col); + def.maps_to_sum.emplace_back(std::move(map_desc)); + } + } + + return def; +} + +static MutableColumns getMergedDataColumns( + const Block & header, + const SummingSortedAlgorithm::ColumnsDefinition & columns_definition) +{ + MutableColumns columns; + columns.reserve(columns_definition.getNumColumns()); + + for (auto & desc : columns_definition.columns_to_aggregate) + { + // Wrap aggregated columns in a tuple to match function signature + if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) + { + size_t tuple_size = desc.column_numbers.size(); + MutableColumns tuple_columns(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + tuple_columns[i] = header.safeGetByPosition(desc.column_numbers[i]).column->cloneEmpty(); + + columns.emplace_back(ColumnTuple::create(std::move(tuple_columns))); + } + else + columns.emplace_back(header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty()); + } + + for (auto & column_number : columns_definition.column_numbers_not_to_aggregate) + columns.emplace_back(header.safeGetByPosition(column_number).type->createColumn()); + + return columns; +} + +static void preprocessChunk(Chunk & chunk) +{ + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + + chunk.setColumns(std::move(columns), num_rows); +} + +static void postprocessChunk( + Chunk & chunk, size_t num_result_columns, + const SummingSortedAlgorithm::ColumnsDefinition & def) +{ + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + Columns res_columns(num_result_columns); + size_t next_column = 0; + + for (auto & desc : def.columns_to_aggregate) + { + auto column = std::move(columns[next_column]); + ++next_column; + + if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) + { + /// Unpack tuple into block. + size_t tuple_size = desc.column_numbers.size(); + for (size_t i = 0; i < tuple_size; ++i) + res_columns[desc.column_numbers[i]] = assert_cast(*column).getColumnPtr(i); + } + else + res_columns[desc.column_numbers[0]] = std::move(column); + } + + for (auto column_number : def.column_numbers_not_to_aggregate) + { + auto column = std::move(columns[next_column]); + ++next_column; + + res_columns[column_number] = std::move(column); + } + + chunk.setColumns(std::move(res_columns), num_rows); +} + +static void setRow(Row & row, SortCursor & cursor, const Names & column_names) +{ + size_t num_columns = row.size(); + for (size_t i = 0; i < num_columns; ++i) + { + try + { + cursor->all_columns[i]->get(cursor->pos, row[i]); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + + /// Find out the name of the column and throw more informative exception. + + String column_name; + if (i < column_names.size()) + column_name = column_names[i]; + + throw Exception("MergingSortedBlockInputStream failed to read row " + toString(cursor->pos) + + " of column " + toString(i) + (column_name.empty() ? "" : " (" + column_name + ")"), + ErrorCodes::CORRUPTED_DATA); + } + } +} + + +Chunk SummingSortedAlgorithm::SummingMergedData::pull(size_t num_result_columns, const ColumnsDefinition & def) +{ + auto chunk = MergedData::pull(); + postprocessChunk(chunk, num_result_columns, def); + return chunk; +} + +SummingSortedAlgorithm::SummingSortedAlgorithm( + const Block & header, size_t num_inputs, + SortDescription description_, + const Names & column_names_to_sum, + size_t max_block_size) + : IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_)) + , columns_definition(defineColumns(header, description_, column_names_to_sum)) + , merged_data(getMergedDataColumns(header, columns_definition), false, max_block_size) + , column_names(header.getNames()) +{ + current_row.resize(header.columns()); + merged_data.initAggregateDescription(columns_definition.columns_to_aggregate); +} + +void SummingSortedAlgorithm::initialize(Chunks chunks) +{ + for (auto & chunk : chunks) + if (chunk) + preprocessChunk(chunk); + + initializeQueue(std::move(chunks)); +} + +void SummingSortedAlgorithm::consume(Chunk chunk, size_t source_num) +{ + preprocessChunk(chunk); + updateCursor(std::move(chunk), source_num); +} + + +void SummingSortedAlgorithm::insertCurrentRowIfNeeded() +{ + /// We have nothing to aggregate. It means that it could be non-zero, because we have columns_not_to_aggregate. + if (columns_definition.columns_to_aggregate.empty()) + current_row_is_zero = false; + + for (auto & desc : columns_definition.columns_to_aggregate) + { + // Do not insert if the aggregation state hasn't been created + if (desc.created) + { + if (desc.is_agg_func_type) + { + current_row_is_zero = false; + } + else + { + try + { + desc.function->insertResultInto(desc.state.data(), *desc.merged_column); + + /// Update zero status of current row + if (desc.column_numbers.size() == 1) + { + // Flag row as non-empty if at least one column number if non-zero + current_row_is_zero = current_row_is_zero && desc.merged_column->isDefaultAt(desc.merged_column->size() - 1); + } + else + { + /// It is sumMapWithOverflow aggregate function. + /// Assume that the row isn't empty in this case (just because it is compatible with previous version) + current_row_is_zero = false; + } + } + catch (...) + { + desc.destroyState(); + throw; + } + } + desc.destroyState(); + } + else + desc.merged_column->insertDefault(); + } + + /// If it is "zero" row, then rollback the insertion + /// (at this moment we need rollback only cols from columns_to_aggregate) + if (current_row_is_zero) + { + for (auto & desc : columns_definition.columns_to_aggregate) + desc.merged_column->popBack(1); + + return; + } + + merged_data.insertRow(current_row, columns_definition.column_numbers_not_to_aggregate); +} + +void SummingSortedAlgorithm::addRow(SortCursor & cursor) +{ + for (auto & desc : columns_definition.columns_to_aggregate) + { + if (!desc.created) + throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR); + + if (desc.is_agg_func_type) + { + // desc.state is not used for AggregateFunction types + auto & col = cursor->all_columns[desc.column_numbers[0]]; + assert_cast(*desc.merged_column).insertMergeFrom(*col, cursor->pos); + } + else + { + // Specialized case for unary functions + if (desc.column_numbers.size() == 1) + { + auto & col = cursor->all_columns[desc.column_numbers[0]]; + desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr); + } + else + { + // Gather all source columns into a vector + ColumnRawPtrs columns(desc.column_numbers.size()); + for (size_t i = 0; i < desc.column_numbers.size(); ++i) + columns[i] = cursor->all_columns[desc.column_numbers[i]]; + + desc.add_function(desc.function.get(), desc.state.data(), columns.data(), cursor->pos, nullptr); + } + } + } +} + +IMergingAlgorithm::Status SummingSortedAlgorithm::merge() +{ + /// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size` + while (queue.isValid()) + { + bool key_differs; + bool has_previous_group = !last_key.empty(); + + SortCursor current = queue.current(); + + { + detail::RowRef current_key; + current_key.set(current); + + if (!has_previous_group) /// The first key encountered. + { + key_differs = true; + current_row_is_zero = true; + } + else + key_differs = !last_key.hasEqualSortColumnsWith(current_key); + + last_key = current_key; + last_chunk_sort_columns.clear(); + } + + if (key_differs) + { + if (has_previous_group) + /// Write the data for the previous group. + insertCurrentRowIfNeeded(); + + if (merged_data.hasEnoughRows()) + { + /// The block is now full and the last row is calculated completely. + last_key.reset(); + return Status(merged_data.pull(column_names.size(), columns_definition)); + } + + setRow(current_row, current, column_names); + + /// Reset aggregation states for next row + for (auto & desc : columns_definition.columns_to_aggregate) + desc.createState(); + + // Start aggregations with current row + addRow(current); + + if (columns_definition.maps_to_sum.empty()) + { + /// We have only columns_to_aggregate. The status of current row will be determined + /// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions. + current_row_is_zero = true; // NOLINT + } + else + { + /// We have complex maps that will be summed with 'mergeMap' method. + /// The single row is considered non zero, and the status after merging with other rows + /// will be determined in the branch below (when key_differs == false). + current_row_is_zero = false; // NOLINT + } + } + else + { + addRow(current); + + // Merge maps only for same rows + for (const auto & desc : columns_definition.maps_to_sum) + if (mergeMap(desc, current_row, current)) + current_row_is_zero = false; + } + + if (!current->isLast()) + { + queue.next(); + } + else + { + /// We get the next block from the corresponding source, if there is one. + queue.removeTop(); + return Status(current.impl->order); + } + } + + /// We will write the data for the last group, if it is non-zero. + /// If it is zero, and without it the output stream will be empty, we will write it anyway. + insertCurrentRowIfNeeded(); + last_chunk_sort_columns.clear(); + return Status(merged_data.pull(column_names.size(), columns_definition), true); +} + + +} diff --git a/src/Processors/Merges/SummingSortedAlgorithm.h b/src/Processors/Merges/SummingSortedAlgorithm.h new file mode 100644 index 00000000000..178e6c13f3b --- /dev/null +++ b/src/Processors/Merges/SummingSortedAlgorithm.h @@ -0,0 +1,152 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class SummingSortedAlgorithm : public IMergingAlgorithmWithDelayedChunk +{ +public: + SummingSortedAlgorithm( + const Block & header, size_t num_inputs, + SortDescription description_, + /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. + const Names & column_names_to_sum, + size_t max_block_size); + + void initialize(Chunks chunks) override; + void consume(Chunk chunk, size_t source_num) override; + Status merge() override; + + struct AggregateDescription; + + /// Stores numbers of key-columns and value-columns. + struct MapDescription + { + std::vector key_col_nums; + std::vector val_col_nums; + }; + + /// This structure define columns into one of three types: + /// * columns which values not needed to be aggregated + /// * aggregate functions and columns which needed to be summed + /// * mapping for nested columns + struct ColumnsDefinition + { + /// Columns with which values should not be aggregated. + ColumnNumbers column_numbers_not_to_aggregate; + /// Columns which should be aggregated. + std::vector columns_to_aggregate; + /// Mapping for nested columns. + std::vector maps_to_sum; + + size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } + }; + + /// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns. + class SummingMergedData : public MergedData + { + private: + using MergedData::pull; + + public: + using MergedData::MergedData; + + void insertRow(const Row & row, const ColumnNumbers & column_numbers) + { + size_t next_column = columns.size() - column_numbers.size(); + for (auto column_number : column_numbers) + { + columns[next_column]->insert(row[column_number]); + ++next_column; + } + + ++total_merged_rows; + ++merged_rows; + /// TODO: sum_blocks_granularity += block_size; + } + + /// Initialize aggregate descriptions with columns. + void initAggregateDescription(std::vector & columns_to_aggregate) + { + size_t num_columns = columns_to_aggregate.size(); + for (size_t column_number = 0; column_number < num_columns; ++column_number) + columns_to_aggregate[column_number].merged_column = columns[column_number].get(); + } + + Chunk pull(size_t num_result_columns, const ColumnsDefinition & def); + }; + +private: + Row current_row; + bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. + + ColumnsDefinition columns_definition; + SummingMergedData merged_data; + + Names column_names; + + void addRow(SortCursor & cursor); + void insertCurrentRowIfNeeded(); + +public: + /// Stores aggregation function, state, and columns to be used as function arguments. + struct AggregateDescription + { + /// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing. + AggregateFunctionPtr function; + IAggregateFunction::AddFunc add_function = nullptr; + std::vector column_numbers; + IColumn * merged_column = nullptr; + AlignedBuffer state; + bool created = false; + + /// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above. + bool is_agg_func_type = false; + + void init(const char * function_name, const DataTypes & argument_types) + { + function = AggregateFunctionFactory::instance().get(function_name, argument_types); + add_function = function->getAddressOfAddFunction(); + state.reset(function->sizeOfData(), function->alignOfData()); + } + + void createState() + { + if (created) + return; + if (is_agg_func_type) + merged_column->insertDefault(); + else + function->create(state.data()); + created = true; + } + + void destroyState() + { + if (!created) + return; + if (!is_agg_func_type) + function->destroy(state.data()); + created = false; + } + + /// Explicitly destroy aggregation state if the stream is terminated + ~AggregateDescription() + { + destroyState(); + } + + AggregateDescription() = default; + AggregateDescription(AggregateDescription &&) = default; + AggregateDescription(const AggregateDescription &) = delete; + }; +}; + +} diff --git a/src/Processors/Merges/SummingSortedTransform.cpp b/src/Processors/Merges/SummingSortedTransform.cpp deleted file mode 100644 index f34a3b479d7..00000000000 --- a/src/Processors/Merges/SummingSortedTransform.cpp +++ /dev/null @@ -1,626 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int CORRUPTED_DATA; -} - -namespace -{ - bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number) - { - for (auto & desc : description) - if (desc.column_name == name || (desc.column_name.empty() && desc.column_number == number)) - return true; - - return false; - } - - /// Returns true if merge result is not empty - bool mergeMap(const SummingSortedTransform::MapDescription & desc, Row & row, SortCursor & cursor) - { - /// Strongly non-optimal. - - Row & left = row; - Row right(left.size()); - - for (size_t col_num : desc.key_col_nums) - right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); - - for (size_t col_num : desc.val_col_nums) - right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); - - auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & - { - return matrix[i].get()[j]; - }; - - auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array - { - size_t size = col_nums.size(); - Array res(size); - for (size_t col_num_index = 0; col_num_index < size; ++col_num_index) - res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j); - return res; - }; - - std::map merged; - - auto accumulate = [](Array & dst, const Array & src) - { - bool has_non_zero = false; - size_t size = dst.size(); - for (size_t i = 0; i < size; ++i) - if (applyVisitor(FieldVisitorSum(src[i]), dst[i])) - has_non_zero = true; - return has_non_zero; - }; - - auto merge = [&](const Row & matrix) - { - size_t rows = matrix[desc.key_col_nums[0]].get().size(); - - for (size_t j = 0; j < rows; ++j) - { - Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j); - Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j); - - auto it = merged.find(key); - if (merged.end() == it) - merged.emplace(std::move(key), std::move(value)); - else - { - if (!accumulate(it->second, value)) - merged.erase(it); - } - } - }; - - merge(left); - merge(right); - - for (size_t col_num : desc.key_col_nums) - row[col_num] = Array(merged.size()); - for (size_t col_num : desc.val_col_nums) - row[col_num] = Array(merged.size()); - - size_t row_num = 0; - for (const auto & key_value : merged) - { - for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; - - for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; - - ++row_num; - } - - return row_num != 0; - } - - SummingSortedTransform::ColumnsDefinition defineColumns( - const Block & header, - const SortDescription & description, - const Names & column_names_to_sum) - { - size_t num_columns = header.columns(); - SummingSortedTransform::ColumnsDefinition def; - - /// name of nested structure -> the column numbers that refer to it. - std::unordered_map> discovered_maps; - - /** Fill in the column numbers, which must be summed. - * This can only be numeric columns that are not part of the sort key. - * If a non-empty column_names_to_sum is specified, then we only take these columns. - * Some columns from column_names_to_sum may not be found. This is ignored. - */ - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnWithTypeAndName & column = header.safeGetByPosition(i); - - /// Discover nested Maps and find columns for summation - if (typeid_cast(column.type.get())) - { - const auto map_name = Nested::extractTableName(column.name); - /// if nested table name ends with `Map` it is a possible candidate for special handling - if (map_name == column.name || !endsWith(map_name, "Map")) - { - def.column_numbers_not_to_aggregate.push_back(i); - continue; - } - - discovered_maps[map_name].emplace_back(i); - } - else - { - bool is_agg_func = WhichDataType(column.type).isAggregateFunction(); - - /// There are special const columns for example after prewhere sections. - if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column)) - { - def.column_numbers_not_to_aggregate.push_back(i); - continue; - } - - /// Are they inside the PK? - if (isInPrimaryKey(description, column.name, i)) - { - def.column_numbers_not_to_aggregate.push_back(i); - continue; - } - - if (column_names_to_sum.empty() - || column_names_to_sum.end() != - std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name)) - { - // Create aggregator to sum this column - SummingSortedTransform::AggregateDescription desc; - desc.is_agg_func_type = is_agg_func; - desc.column_numbers = {i}; - - if (!is_agg_func) - { - desc.init("sumWithOverflow", {column.type}); - } - - def.columns_to_aggregate.emplace_back(std::move(desc)); - } - else - { - // Column is not going to be summed, use last value - def.column_numbers_not_to_aggregate.push_back(i); - } - } - } - - /// select actual nested Maps from list of candidates - for (const auto & map : discovered_maps) - { - /// map should contain at least two elements (key -> value) - if (map.second.size() < 2) - { - for (auto col : map.second) - def.column_numbers_not_to_aggregate.push_back(col); - continue; - } - - /// no elements of map could be in primary key - auto column_num_it = map.second.begin(); - for (; column_num_it != map.second.end(); ++column_num_it) - if (isInPrimaryKey(description, header.safeGetByPosition(*column_num_it).name, *column_num_it)) - break; - if (column_num_it != map.second.end()) - { - for (auto col : map.second) - def.column_numbers_not_to_aggregate.push_back(col); - continue; - } - - DataTypes argument_types; - SummingSortedTransform::AggregateDescription desc; - SummingSortedTransform::MapDescription map_desc; - - column_num_it = map.second.begin(); - for (; column_num_it != map.second.end(); ++column_num_it) - { - const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it); - const String & name = key_col.name; - const IDataType & nested_type = *assert_cast(*key_col.type).getNestedType(); - - if (column_num_it == map.second.begin() - || endsWith(name, "ID") - || endsWith(name, "Key") - || endsWith(name, "Type")) - { - if (!nested_type.isValueRepresentedByInteger() && !isStringOrFixedString(nested_type)) - break; - - map_desc.key_col_nums.push_back(*column_num_it); - } - else - { - if (!nested_type.isSummable()) - break; - - map_desc.val_col_nums.push_back(*column_num_it); - } - - // Add column to function arguments - desc.column_numbers.push_back(*column_num_it); - argument_types.push_back(key_col.type); - } - - if (column_num_it != map.second.end()) - { - for (auto col : map.second) - def.column_numbers_not_to_aggregate.push_back(col); - continue; - } - - if (map_desc.key_col_nums.size() == 1) - { - // Create summation for all value columns in the map - desc.init("sumMapWithOverflow", argument_types); - def.columns_to_aggregate.emplace_back(std::move(desc)); - } - else - { - // Fall back to legacy mergeMaps for composite keys - for (auto col : map.second) - def.column_numbers_not_to_aggregate.push_back(col); - def.maps_to_sum.emplace_back(std::move(map_desc)); - } - } - - return def; - } - - MutableColumns getMergedDataColumns( - const Block & header, - const SummingSortedTransform::ColumnsDefinition & columns_definition) - { - MutableColumns columns; - columns.reserve(columns_definition.getNumColumns()); - - for (auto & desc : columns_definition.columns_to_aggregate) - { - // Wrap aggregated columns in a tuple to match function signature - if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) - { - size_t tuple_size = desc.column_numbers.size(); - MutableColumns tuple_columns(tuple_size); - for (size_t i = 0; i < tuple_size; ++i) - tuple_columns[i] = header.safeGetByPosition(desc.column_numbers[i]).column->cloneEmpty(); - - columns.emplace_back(ColumnTuple::create(std::move(tuple_columns))); - } - else - columns.emplace_back(header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty()); - } - - for (auto & column_number : columns_definition.column_numbers_not_to_aggregate) - columns.emplace_back(header.safeGetByPosition(column_number).type->createColumn()); - - return columns; - } - - void finalizeChunk( - Chunk & chunk, size_t num_result_columns, - const SummingSortedTransform::ColumnsDefinition & columns_definition) - { - size_t num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - - Columns res_columns(num_result_columns); - size_t next_column = 0; - - for (auto & desc : columns_definition.columns_to_aggregate) - { - auto column = std::move(columns[next_column]); - ++next_column; - - if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) - { - /// Unpack tuple into block. - size_t tuple_size = desc.column_numbers.size(); - for (size_t i = 0; i < tuple_size; ++i) - res_columns[desc.column_numbers[i]] = assert_cast(*column).getColumnPtr(i); - } - else - res_columns[desc.column_numbers[0]] = std::move(column); - } - - for (auto column_number : columns_definition.column_numbers_not_to_aggregate) - { - auto column = std::move(columns[next_column]); - ++next_column; - - res_columns[column_number] = std::move(column); - } - - chunk.setColumns(std::move(res_columns), num_rows); - } - - void setRow(Row & row, SortCursor & cursor, const Block & header) - { - size_t num_columns = row.size(); - for (size_t i = 0; i < num_columns; ++i) - { - try - { - cursor->all_columns[i]->get(cursor->pos, row[i]); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - - /// Find out the name of the column and throw more informative exception. - - String column_name; - if (i < header.columns()) - { - column_name = header.safeGetByPosition(i).name; - break; - } - - throw Exception("MergingSortedBlockInputStream failed to read row " + toString(cursor->pos) - + " of column " + toString(i) + (column_name.empty() ? "" : " (" + column_name + ")"), - ErrorCodes::CORRUPTED_DATA); - } - } - } -} - -SummingSortedTransform::SummingSortedTransform( - const Block & header, size_t num_inputs, - SortDescription description_, - /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. - const Names & column_names_to_sum, - size_t max_block_size) - : IMergingTransform(num_inputs, header, header, true) - , columns_definition(defineColumns(header, description_, column_names_to_sum)) - , merged_data(getMergedDataColumns(header, columns_definition), false, max_block_size) - , description(std::move(description_)) - , source_chunks(num_inputs) - , cursors(num_inputs) -{ - current_row.resize(header.columns()); - merged_data.initAggregateDescription(columns_definition.columns_to_aggregate); -} - -void SummingSortedTransform::initializeInputs() -{ - queue = SortingHeap(cursors); - is_queue_initialized = true; -} - -void SummingSortedTransform::consume(Chunk chunk, size_t input_number) -{ - updateCursor(std::move(chunk), input_number); - - if (is_queue_initialized) - queue.push(cursors[input_number]); -} - -void SummingSortedTransform::updateCursor(Chunk chunk, size_t source_num) -{ - auto num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfConst(); - - chunk.setColumns(std::move(columns), num_rows); - - auto & source_chunk = source_chunks[source_num]; - - if (source_chunk) - { - /// Extend lifetime of last chunk. - last_chunk = std::move(source_chunk); - last_chunk_sort_columns = std::move(cursors[source_num].sort_columns); - - source_chunk = std::move(chunk); - cursors[source_num].reset(source_chunk.getColumns(), {}); - } - else - { - if (cursors[source_num].has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - source_chunk = std::move(chunk); - cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num); - } -} - -void SummingSortedTransform::work() -{ - merge(); - prepareOutputChunk(merged_data); - - if (has_output_chunk) - { - finalizeChunk(output_chunk, getOutputs().back().getHeader().columns(), columns_definition); - merged_data.initAggregateDescription(columns_definition.columns_to_aggregate); - } -} - -void SummingSortedTransform::insertCurrentRowIfNeeded() -{ - /// We have nothing to aggregate. It means that it could be non-zero, because we have columns_not_to_aggregate. - if (columns_definition.columns_to_aggregate.empty()) - current_row_is_zero = false; - - for (auto & desc : columns_definition.columns_to_aggregate) - { - // Do not insert if the aggregation state hasn't been created - if (desc.created) - { - if (desc.is_agg_func_type) - { - current_row_is_zero = false; - } - else - { - try - { - desc.function->insertResultInto(desc.state.data(), *desc.merged_column); - - /// Update zero status of current row - if (desc.column_numbers.size() == 1) - { - // Flag row as non-empty if at least one column number if non-zero - current_row_is_zero = current_row_is_zero && desc.merged_column->isDefaultAt(desc.merged_column->size() - 1); - } - else - { - /// It is sumMapWithOverflow aggregate function. - /// Assume that the row isn't empty in this case (just because it is compatible with previous version) - current_row_is_zero = false; - } - } - catch (...) - { - desc.destroyState(); - throw; - } - } - desc.destroyState(); - } - else - desc.merged_column->insertDefault(); - } - - /// If it is "zero" row, then rollback the insertion - /// (at this moment we need rollback only cols from columns_to_aggregate) - if (current_row_is_zero) - { - for (auto & desc : columns_definition.columns_to_aggregate) - desc.merged_column->popBack(1); - - return; - } - - merged_data.insertRow(current_row, columns_definition.column_numbers_not_to_aggregate); -} - -void SummingSortedTransform::addRow(SortCursor & cursor) -{ - for (auto & desc : columns_definition.columns_to_aggregate) - { - if (!desc.created) - throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR); - - if (desc.is_agg_func_type) - { - // desc.state is not used for AggregateFunction types - auto & col = cursor->all_columns[desc.column_numbers[0]]; - assert_cast(*desc.merged_column).insertMergeFrom(*col, cursor->pos); - } - else - { - // Specialized case for unary functions - if (desc.column_numbers.size() == 1) - { - auto & col = cursor->all_columns[desc.column_numbers[0]]; - desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr); - } - else - { - // Gather all source columns into a vector - ColumnRawPtrs columns(desc.column_numbers.size()); - for (size_t i = 0; i < desc.column_numbers.size(); ++i) - columns[i] = cursor->all_columns[desc.column_numbers[i]]; - - desc.add_function(desc.function.get(), desc.state.data(), columns.data(), cursor->pos, nullptr); - } - } - } -} - -void SummingSortedTransform::merge() -{ - /// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size` - while (queue.isValid()) - { - bool key_differs; - bool has_previous_group = !last_key.empty(); - - SortCursor current = queue.current(); - - { - detail::RowRef current_key; - current_key.set(current); - - if (!has_previous_group) /// The first key encountered. - { - key_differs = true; - current_row_is_zero = true; - } - else - key_differs = !last_key.hasEqualSortColumnsWith(current_key); - - last_key = current_key; - last_chunk_sort_columns.clear(); - } - - if (key_differs) - { - if (has_previous_group) - /// Write the data for the previous group. - insertCurrentRowIfNeeded(); - - if (merged_data.hasEnoughRows()) - { - /// The block is now full and the last row is calculated completely. - last_key.reset(); - return; - } - - setRow(current_row, current, getInputs().front().getHeader()); - - /// Reset aggregation states for next row - for (auto & desc : columns_definition.columns_to_aggregate) - desc.createState(); - - // Start aggregations with current row - addRow(current); - - if (columns_definition.maps_to_sum.empty()) - { - /// We have only columns_to_aggregate. The status of current row will be determined - /// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions. - current_row_is_zero = true; // NOLINT - } - else - { - /// We have complex maps that will be summed with 'mergeMap' method. - /// The single row is considered non zero, and the status after merging with other rows - /// will be determined in the branch below (when key_differs == false). - current_row_is_zero = false; // NOLINT - } - } - else - { - addRow(current); - - // Merge maps only for same rows - for (const auto & desc : columns_definition.maps_to_sum) - if (mergeMap(desc, current_row, current)) - current_row_is_zero = false; - } - - if (!current->isLast()) - { - queue.next(); - } - else - { - /// We get the next block from the corresponding source, if there is one. - queue.removeTop(); - requestDataForInput(current.impl->order); - return; - } - } - - /// We will write the data for the last group, if it is non-zero. - /// If it is zero, and without it the output stream will be empty, we will write it anyway. - insertCurrentRowIfNeeded(); - last_chunk_sort_columns.clear(); - is_finished = true; -} - -} diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index cc651197647..37859e1b88b 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -1,15 +1,7 @@ #pragma once #include -#include -#include - -#include -#include -#include -#include -#include -#include +#include namespace DB { @@ -19,7 +11,7 @@ namespace DB * collapses them into one row, summing all the numeric columns except the primary key. * If in all numeric columns, except for the primary key, the result is zero, it deletes the row. */ -class SummingSortedTransform final : public IMergingTransform +class SummingSortedTransform final : public IMergingTransform2 { public: @@ -28,146 +20,18 @@ public: SortDescription description_, /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. const Names & column_names_to_sum, - size_t max_block_size); - - struct AggregateDescription; - - /// Stores numbers of key-columns and value-columns. - struct MapDescription + size_t max_block_size) + : IMergingTransform2( + num_inputs, header, header, true, + header, + num_inputs, + std::move(description_), + column_names_to_sum, + max_block_size) { - std::vector key_col_nums; - std::vector val_col_nums; - }; - - struct ColumnsDefinition - { - /// Columns with which values should be summed. - ColumnNumbers column_numbers_not_to_aggregate; - /// Columns which should be aggregated. - std::vector columns_to_aggregate; - /// Mapping for nested columns. - std::vector maps_to_sum; - - size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } - }; - - /// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns. - struct SummingMergedData : public MergedData - { - public: - using MergedData::MergedData; - - void insertRow(const Row & row, const ColumnNumbers & column_numbers) - { - size_t next_column = columns.size() - column_numbers.size(); - for (auto column_number : column_numbers) - { - columns[next_column]->insert(row[column_number]); - ++next_column; - } - - ++total_merged_rows; - ++merged_rows; - /// TODO: sum_blocks_granularity += block_size; - } - - /// Initialize aggregate descriptions with columns. - void initAggregateDescription(std::vector & columns_to_aggregate) - { - size_t num_columns = columns_to_aggregate.size(); - for (size_t column_number = 0; column_number < num_columns; ++column_number) - columns_to_aggregate[column_number].merged_column = columns[column_number].get(); - } - }; + } String getName() const override { return "SummingSortedTransform"; } - void work() override; - -protected: - void initializeInputs() override; - void consume(Chunk chunk, size_t input_number) override; - -private: - Row current_row; - bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. - - ColumnsDefinition columns_definition; - SummingMergedData merged_data; - - SortDescription description; - - /// Chunks currently being merged. - std::vector source_chunks; - SortCursorImpls cursors; - - /// In merging algorithm, we need to compare current sort key with the last one. - /// So, sorting columns for last row needed to be stored. - /// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor). - Chunk last_chunk; - ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. - - detail::RowRef last_key; - - SortingHeap queue; - bool is_queue_initialized = false; - - void merge(); - void updateCursor(Chunk chunk, size_t source_num); - void addRow(SortCursor & cursor); - void insertCurrentRowIfNeeded(); - -public: - /// Stores aggregation function, state, and columns to be used as function arguments. - struct AggregateDescription - { - /// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing. - AggregateFunctionPtr function; - IAggregateFunction::AddFunc add_function = nullptr; - std::vector column_numbers; - IColumn * merged_column = nullptr; - AlignedBuffer state; - bool created = false; - - /// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above. - bool is_agg_func_type = false; - - void init(const char * function_name, const DataTypes & argument_types) - { - function = AggregateFunctionFactory::instance().get(function_name, argument_types); - add_function = function->getAddressOfAddFunction(); - state.reset(function->sizeOfData(), function->alignOfData()); - } - - void createState() - { - if (created) - return; - if (is_agg_func_type) - merged_column->insertDefault(); - else - function->create(state.data()); - created = true; - } - - void destroyState() - { - if (!created) - return; - if (!is_agg_func_type) - function->destroy(state.data()); - created = false; - } - - /// Explicitly destroy aggregation state if the stream is terminated - ~AggregateDescription() - { - destroyState(); - } - - AggregateDescription() = default; - AggregateDescription(AggregateDescription &&) = default; - AggregateDescription(const AggregateDescription &) = delete; - }; }; } From b7d4bccc770d0c12b5d2ec05f8c64242a9cd0f5e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 13 Apr 2020 22:02:13 +0300 Subject: [PATCH 212/743] fix build --- .../tests/gtest_transform_query_for_external_database.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 385e47201a5..2041472f7f2 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -30,7 +30,7 @@ struct State State() { registerFunctions(); - DatabasePtr database = std::make_shared("test"); + DatabasePtr database = std::make_shared("test", context); database->attachTable("table", StorageMemory::create(StorageID("test", "table"), ColumnsDescription{columns}, ConstraintsDescription{})); context.makeGlobalContext(); DatabaseCatalog::instance().attachDatabase("test", database); From d7d29d8250022696b3ab2b13c3883a8e0e81d940 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 22:02:57 +0300 Subject: [PATCH 213/743] Fix build. --- src/Processors/Merges/SummingSortedAlgorithm.cpp | 12 +++++++----- src/Processors/Merges/SummingSortedAlgorithm.h | 11 ++++++++++- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Processors/Merges/SummingSortedAlgorithm.cpp b/src/Processors/Merges/SummingSortedAlgorithm.cpp index 357ecc7114e..4f437654a19 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/SummingSortedAlgorithm.cpp @@ -369,10 +369,13 @@ static void setRow(Row & row, SortCursor & cursor, const Names & column_names) } -Chunk SummingSortedAlgorithm::SummingMergedData::pull(size_t num_result_columns, const ColumnsDefinition & def) +Chunk SummingSortedAlgorithm::SummingMergedData::pull(size_t num_result_columns) { auto chunk = MergedData::pull(); postprocessChunk(chunk, num_result_columns, def); + + initAggregateDescription(def.columns_to_aggregate); + return chunk; } @@ -383,7 +386,7 @@ SummingSortedAlgorithm::SummingSortedAlgorithm( size_t max_block_size) : IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_)) , columns_definition(defineColumns(header, description_, column_names_to_sum)) - , merged_data(getMergedDataColumns(header, columns_definition), false, max_block_size) + , merged_data(getMergedDataColumns(header, columns_definition), max_block_size, columns_definition) , column_names(header.getNames()) { current_row.resize(header.columns()); @@ -535,7 +538,7 @@ IMergingAlgorithm::Status SummingSortedAlgorithm::merge() { /// The block is now full and the last row is calculated completely. last_key.reset(); - return Status(merged_data.pull(column_names.size(), columns_definition)); + return Status(merged_data.pull(column_names.size())); } setRow(current_row, current, column_names); @@ -587,8 +590,7 @@ IMergingAlgorithm::Status SummingSortedAlgorithm::merge() /// If it is zero, and without it the output stream will be empty, we will write it anyway. insertCurrentRowIfNeeded(); last_chunk_sort_columns.clear(); - return Status(merged_data.pull(column_names.size(), columns_definition), true); + return Status(merged_data.pull(column_names.size()), true); } - } diff --git a/src/Processors/Merges/SummingSortedAlgorithm.h b/src/Processors/Merges/SummingSortedAlgorithm.h index 178e6c13f3b..8cd7fb1ff13 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/SummingSortedAlgorithm.h @@ -58,6 +58,12 @@ public: public: using MergedData::MergedData; + SummingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_) + : MergedData(std::move(columns_), false, max_block_size_) + , def(def_) + { + } + void insertRow(const Row & row, const ColumnNumbers & column_numbers) { size_t next_column = columns.size() - column_numbers.size(); @@ -80,7 +86,10 @@ public: columns_to_aggregate[column_number].merged_column = columns[column_number].get(); } - Chunk pull(size_t num_result_columns, const ColumnsDefinition & def); + Chunk pull(size_t num_result_columns); + + private: + ColumnsDefinition & def; }; private: From 84b1dd4c1e642b259fd10ff3cd25abcb6f6eb4e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 22:20:26 +0300 Subject: [PATCH 214/743] Fix tests. --- src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h | 3 +-- src/Processors/Merges/SummingSortedAlgorithm.cpp | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h index 2a53f22f5aa..214f262a5cc 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h +++ b/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h @@ -16,6 +16,7 @@ public: protected: SortingHeap queue; + SortDescription description; /// Previous row. May refer to last_chunk_sort_columns or row from source_chunks. detail::RowRef last_key; @@ -26,8 +27,6 @@ protected: void updateCursor(Chunk chunk, size_t source_num); private: - SortDescription description; - /// Chunks currently being merged. std::vector source_chunks; SortCursorImpls cursors; diff --git a/src/Processors/Merges/SummingSortedAlgorithm.cpp b/src/Processors/Merges/SummingSortedAlgorithm.cpp index 4f437654a19..43f282e516d 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/SummingSortedAlgorithm.cpp @@ -385,7 +385,7 @@ SummingSortedAlgorithm::SummingSortedAlgorithm( const Names & column_names_to_sum, size_t max_block_size) : IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_)) - , columns_definition(defineColumns(header, description_, column_names_to_sum)) + , columns_definition(defineColumns(header, description, column_names_to_sum)) , merged_data(getMergedDataColumns(header, columns_definition), max_block_size, columns_definition) , column_names(header.getNames()) { From e46322fcefb1d3e1d88fcd24c4776f91ae0a037a Mon Sep 17 00:00:00 2001 From: Avogar Date: Mon, 13 Apr 2020 22:33:02 +0300 Subject: [PATCH 215/743] Update MsgPack input format. --- src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index ee32aeb6bfe..53c5a623a35 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -119,8 +119,8 @@ void MsgPackRowInputFormat::insertObject(IColumn & column, DataTypePtr data_type case TypeIndex::FixedString: [[fallthrough]]; case TypeIndex::String: { - String str = object.as(); - column.insertData(str.data(), str.size()); + msgpack::object_str obj_str = object.via.str; + column.insertData(obj_str.ptr, obj_str.size); return; } case TypeIndex::Array: From 801320983b61c5985d1ccb5bbbcdfe61dab9bb6a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 23:50:13 +0300 Subject: [PATCH 216/743] Refactor SummingSortedAlgorithm. --- .../Merges/SummingSortedAlgorithm.cpp | 297 ++++++++++++------ .../Merges/SummingSortedAlgorithm.h | 123 ++------ 2 files changed, 222 insertions(+), 198 deletions(-) diff --git a/src/Processors/Merges/SummingSortedAlgorithm.cpp b/src/Processors/Merges/SummingSortedAlgorithm.cpp index 43f282e516d..923e890abd7 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/SummingSortedAlgorithm.cpp @@ -16,6 +16,70 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; } +SummingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition() = default; +SummingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition(ColumnsDefinition &&) noexcept = default; +SummingSortedAlgorithm::ColumnsDefinition::~ColumnsDefinition() = default; + +/// Stores numbers of key-columns and value-columns. +struct SummingSortedAlgorithm::MapDescription +{ + std::vector key_col_nums; + std::vector val_col_nums; +}; + +/// Stores aggregation function, state, and columns to be used as function arguments. +struct SummingSortedAlgorithm::AggregateDescription +{ + /// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing. + AggregateFunctionPtr function; + IAggregateFunction::AddFunc add_function = nullptr; + std::vector column_numbers; + IColumn * merged_column = nullptr; + AlignedBuffer state; + bool created = false; + + /// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above. + bool is_agg_func_type = false; + + void init(const char * function_name, const DataTypes & argument_types) + { + function = AggregateFunctionFactory::instance().get(function_name, argument_types); + add_function = function->getAddressOfAddFunction(); + state.reset(function->sizeOfData(), function->alignOfData()); + } + + void createState() + { + if (created) + return; + if (is_agg_func_type) + merged_column->insertDefault(); + else + function->create(state.data()); + created = true; + } + + void destroyState() + { + if (!created) + return; + if (!is_agg_func_type) + function->destroy(state.data()); + created = false; + } + + /// Explicitly destroy aggregation state if the stream is terminated + ~AggregateDescription() + { + destroyState(); + } + + AggregateDescription() = default; + AggregateDescription(AggregateDescription &&) = default; + AggregateDescription(const AggregateDescription &) = delete; +}; + + static bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number) { for (auto & desc : description) @@ -26,7 +90,8 @@ static bool isInPrimaryKey(const SortDescription & description, const std::strin } /// Returns true if merge result is not empty -static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row & row, SortCursor & cursor) +static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, + Row & row, const ColumnRawPtrs & raw_columns, size_t row_num) { /// Strongly non-optimal. @@ -34,10 +99,10 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row & Row right(left.size()); for (size_t col_num : desc.key_col_nums) - right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); + right[col_num] = (*raw_columns[col_num])[row_num].template get(); for (size_t col_num : desc.val_col_nums) - right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); + right[col_num] = (*raw_columns[col_num])[row_num].template get(); auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & { @@ -93,19 +158,19 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row & for (size_t col_num : desc.val_col_nums) row[col_num] = Array(merged.size()); - size_t row_num = 0; + size_t row_num_ = 0; for (const auto & key_value : merged) { for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; + row[desc.key_col_nums[col_num_index]].get()[row_num_] = key_value.first[col_num_index]; for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; + row[desc.val_col_nums[col_num_index]].get()[row_num_] = key_value.second[col_num_index]; - ++row_num; + ++row_num_; } - return row_num != 0; + return row_num_ != 0; } static SummingSortedAlgorithm::ColumnsDefinition defineColumns( @@ -115,6 +180,7 @@ static SummingSortedAlgorithm::ColumnsDefinition defineColumns( { size_t num_columns = header.columns(); SummingSortedAlgorithm::ColumnsDefinition def; + def.column_names = header.getNames(); /// name of nested structure -> the column numbers that refer to it. std::unordered_map> discovered_maps; @@ -342,14 +408,14 @@ static void postprocessChunk( chunk.setColumns(std::move(res_columns), num_rows); } -static void setRow(Row & row, SortCursor & cursor, const Names & column_names) +static void setRow(Row & row, const ColumnRawPtrs & raw_columns, size_t row_num, const Names & column_names) { size_t num_columns = row.size(); for (size_t i = 0; i < num_columns; ++i) { try { - cursor->all_columns[i]->get(cursor->pos, row[i]); + raw_columns[i]->get(row_num, row[i]); } catch (...) { @@ -361,7 +427,7 @@ static void setRow(Row & row, SortCursor & cursor, const Names & column_names) if (i < column_names.size()) column_name = column_names[i]; - throw Exception("MergingSortedBlockInputStream failed to read row " + toString(cursor->pos) + throw Exception("MergingSortedBlockInputStream failed to read row " + toString(row_num) + " of column " + toString(i) + (column_name.empty() ? "" : " (" + column_name + ")"), ErrorCodes::CORRUPTED_DATA); } @@ -369,53 +435,49 @@ static void setRow(Row & row, SortCursor & cursor, const Names & column_names) } -Chunk SummingSortedAlgorithm::SummingMergedData::pull(size_t num_result_columns) +SummingSortedAlgorithm::SummingMergedData::SummingMergedData( + MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_) + : MergedData(std::move(columns_), false, max_block_size_) + , def(def_) { - auto chunk = MergedData::pull(); - postprocessChunk(chunk, num_result_columns, def); - - initAggregateDescription(def.columns_to_aggregate); - - return chunk; + current_row.resize(def.column_names.size()); + initAggregateDescription(); } -SummingSortedAlgorithm::SummingSortedAlgorithm( - const Block & header, size_t num_inputs, - SortDescription description_, - const Names & column_names_to_sum, - size_t max_block_size) - : IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_)) - , columns_definition(defineColumns(header, description, column_names_to_sum)) - , merged_data(getMergedDataColumns(header, columns_definition), max_block_size, columns_definition) - , column_names(header.getNames()) +void SummingSortedAlgorithm::SummingMergedData::startGroup(ColumnRawPtrs & raw_columns, size_t row) { - current_row.resize(header.columns()); - merged_data.initAggregateDescription(columns_definition.columns_to_aggregate); + setRow(current_row, raw_columns, row, def.column_names); + + /// Reset aggregation states for next row + for (auto & desc : def.columns_to_aggregate) + desc.createState(); + + if (def.maps_to_sum.empty()) + { + /// We have only columns_to_aggregate. The status of current row will be determined + /// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions. + current_row_is_zero = true; // NOLINT + } + else + { + /// We have complex maps that will be summed with 'mergeMap' method. + /// The single row is considered non zero, and the status after merging with other rows + /// will be determined in the branch below (when key_differs == false). + current_row_is_zero = false; // NOLINT + } + + addRowImpl(raw_columns, row); } -void SummingSortedAlgorithm::initialize(Chunks chunks) +void SummingSortedAlgorithm::SummingMergedData::finishGroup() { - for (auto & chunk : chunks) - if (chunk) - preprocessChunk(chunk); + is_group_started = false; - initializeQueue(std::move(chunks)); -} - -void SummingSortedAlgorithm::consume(Chunk chunk, size_t source_num) -{ - preprocessChunk(chunk); - updateCursor(std::move(chunk), source_num); -} - - -void SummingSortedAlgorithm::insertCurrentRowIfNeeded() -{ /// We have nothing to aggregate. It means that it could be non-zero, because we have columns_not_to_aggregate. - if (columns_definition.columns_to_aggregate.empty()) + if (def.columns_to_aggregate.empty()) current_row_is_zero = false; - for (auto & desc : columns_definition.columns_to_aggregate) + for (auto & desc : def.columns_to_aggregate) { // Do not insert if the aggregation state hasn't been created if (desc.created) @@ -459,56 +521,120 @@ void SummingSortedAlgorithm::insertCurrentRowIfNeeded() /// (at this moment we need rollback only cols from columns_to_aggregate) if (current_row_is_zero) { - for (auto & desc : columns_definition.columns_to_aggregate) + for (auto & desc : def.columns_to_aggregate) desc.merged_column->popBack(1); return; } - merged_data.insertRow(current_row, columns_definition.column_numbers_not_to_aggregate); + size_t next_column = columns.size() - def.column_numbers_not_to_aggregate.size(); + for (auto column_number : def.column_numbers_not_to_aggregate) + { + columns[next_column]->insert(current_row[column_number]); + ++next_column; + } + + ++total_merged_rows; + ++merged_rows; + /// TODO: sum_blocks_granularity += block_size; } -void SummingSortedAlgorithm::addRow(SortCursor & cursor) +void SummingSortedAlgorithm::SummingMergedData::addRow(ColumnRawPtrs & raw_columns, size_t row) { - for (auto & desc : columns_definition.columns_to_aggregate) + // Merge maps only for same rows + for (const auto & desc : def.maps_to_sum) + if (mergeMap(desc, current_row, raw_columns, row)) + current_row_is_zero = false; + + addRowImpl(raw_columns, row); +} + +void SummingSortedAlgorithm::SummingMergedData::addRowImpl(ColumnRawPtrs & raw_columns, size_t row) +{ + for (auto & desc : def.columns_to_aggregate) { if (!desc.created) - throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR); + throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", + ErrorCodes::LOGICAL_ERROR); if (desc.is_agg_func_type) { // desc.state is not used for AggregateFunction types - auto & col = cursor->all_columns[desc.column_numbers[0]]; - assert_cast(*desc.merged_column).insertMergeFrom(*col, cursor->pos); + auto & col = raw_columns[desc.column_numbers[0]]; + assert_cast(*desc.merged_column).insertMergeFrom(*col, row); } else { // Specialized case for unary functions if (desc.column_numbers.size() == 1) { - auto & col = cursor->all_columns[desc.column_numbers[0]]; - desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr); + auto & col = raw_columns[desc.column_numbers[0]]; + desc.add_function(desc.function.get(), desc.state.data(), &col, row, nullptr); } else { // Gather all source columns into a vector ColumnRawPtrs columns(desc.column_numbers.size()); for (size_t i = 0; i < desc.column_numbers.size(); ++i) - columns[i] = cursor->all_columns[desc.column_numbers[i]]; + columns[i] = raw_columns[desc.column_numbers[i]]; - desc.add_function(desc.function.get(), desc.state.data(), columns.data(), cursor->pos, nullptr); + desc.add_function(desc.function.get(), desc.state.data(), columns.data(), row, nullptr); } } } } +void SummingSortedAlgorithm::SummingMergedData::initAggregateDescription() +{ + size_t num_columns = def.columns_to_aggregate.size(); + for (size_t column_number = 0; column_number < num_columns; ++column_number) + def.columns_to_aggregate[column_number].merged_column = columns[column_number].get(); +} + + +Chunk SummingSortedAlgorithm::SummingMergedData::pull() +{ + auto chunk = MergedData::pull(); + postprocessChunk(chunk, def.column_names.size(), def); + + initAggregateDescription(); + + return chunk; +} + + +SummingSortedAlgorithm::SummingSortedAlgorithm( + const Block & header, size_t num_inputs, + SortDescription description_, + const Names & column_names_to_sum, + size_t max_block_size) + : IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_)) + , columns_definition(defineColumns(header, description, column_names_to_sum)) + , merged_data(getMergedDataColumns(header, columns_definition), max_block_size, columns_definition) +{ +} + +void SummingSortedAlgorithm::initialize(Chunks chunks) +{ + for (auto & chunk : chunks) + if (chunk) + preprocessChunk(chunk); + + initializeQueue(std::move(chunks)); +} + +void SummingSortedAlgorithm::consume(Chunk chunk, size_t source_num) +{ + preprocessChunk(chunk); + updateCursor(std::move(chunk), source_num); +} + IMergingAlgorithm::Status SummingSortedAlgorithm::merge() { /// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size` while (queue.isValid()) { bool key_differs; - bool has_previous_group = !last_key.empty(); SortCursor current = queue.current(); @@ -516,13 +642,7 @@ IMergingAlgorithm::Status SummingSortedAlgorithm::merge() detail::RowRef current_key; current_key.set(current); - if (!has_previous_group) /// The first key encountered. - { - key_differs = true; - current_row_is_zero = true; - } - else - key_differs = !last_key.hasEqualSortColumnsWith(current_key); + key_differs = last_key.empty() || !last_key.hasEqualSortColumnsWith(current_key); last_key = current_key; last_chunk_sort_columns.clear(); @@ -530,49 +650,21 @@ IMergingAlgorithm::Status SummingSortedAlgorithm::merge() if (key_differs) { - if (has_previous_group) + if (merged_data.isGroupStarted()) /// Write the data for the previous group. - insertCurrentRowIfNeeded(); + merged_data.finishGroup(); if (merged_data.hasEnoughRows()) { /// The block is now full and the last row is calculated completely. last_key.reset(); - return Status(merged_data.pull(column_names.size())); + return Status(merged_data.pull()); } - setRow(current_row, current, column_names); - - /// Reset aggregation states for next row - for (auto & desc : columns_definition.columns_to_aggregate) - desc.createState(); - - // Start aggregations with current row - addRow(current); - - if (columns_definition.maps_to_sum.empty()) - { - /// We have only columns_to_aggregate. The status of current row will be determined - /// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions. - current_row_is_zero = true; // NOLINT - } - else - { - /// We have complex maps that will be summed with 'mergeMap' method. - /// The single row is considered non zero, and the status after merging with other rows - /// will be determined in the branch below (when key_differs == false). - current_row_is_zero = false; // NOLINT - } + merged_data.startGroup(current->all_columns, current->pos); } else - { - addRow(current); - - // Merge maps only for same rows - for (const auto & desc : columns_definition.maps_to_sum) - if (mergeMap(desc, current_row, current)) - current_row_is_zero = false; - } + merged_data.addRow(current->all_columns, current->pos); if (!current->isLast()) { @@ -587,10 +679,11 @@ IMergingAlgorithm::Status SummingSortedAlgorithm::merge() } /// We will write the data for the last group, if it is non-zero. - /// If it is zero, and without it the output stream will be empty, we will write it anyway. - insertCurrentRowIfNeeded(); + if (merged_data.isGroupStarted()) + merged_data.finishGroup(); + last_chunk_sort_columns.clear(); - return Status(merged_data.pull(column_names.size()), true); + return Status(merged_data.pull(), true); } } diff --git a/src/Processors/Merges/SummingSortedAlgorithm.h b/src/Processors/Merges/SummingSortedAlgorithm.h index 8cd7fb1ff13..4b750063df5 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/SummingSortedAlgorithm.h @@ -25,13 +25,7 @@ public: Status merge() override; struct AggregateDescription; - - /// Stores numbers of key-columns and value-columns. - struct MapDescription - { - std::vector key_col_nums; - std::vector val_col_nums; - }; + struct MapDescription; /// This structure define columns into one of three types: /// * columns which values not needed to be aggregated @@ -39,6 +33,10 @@ public: /// * mapping for nested columns struct ColumnsDefinition { + ColumnsDefinition(); /// Is needed because destructor is defined. + ColumnsDefinition(ColumnsDefinition &&) noexcept; /// Is needed because destructor is defined. + ~ColumnsDefinition(); /// Is needed because otherwise std::vector's destructor uses incomplete types. + /// Columns with which values should not be aggregated. ColumnNumbers column_numbers_not_to_aggregate; /// Columns which should be aggregated. @@ -46,6 +44,10 @@ public: /// Mapping for nested columns. std::vector maps_to_sum; + /// Names of columns from header. + Names column_names; + + /// It's not the same as column_names.size() size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } }; @@ -54,108 +56,37 @@ public: { private: using MergedData::pull; + using MergedData::insertRow; public: - using MergedData::MergedData; + SummingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_); - SummingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_) - : MergedData(std::move(columns_), false, max_block_size_) - , def(def_) - { - } + void startGroup(ColumnRawPtrs & raw_columns, size_t row); + void finishGroup(); - void insertRow(const Row & row, const ColumnNumbers & column_numbers) - { - size_t next_column = columns.size() - column_numbers.size(); - for (auto column_number : column_numbers) - { - columns[next_column]->insert(row[column_number]); - ++next_column; - } + bool isGroupStarted() const { return is_group_started; } + void addRow(ColumnRawPtrs & raw_columns, size_t row); /// Possible only when group was started. - ++total_merged_rows; - ++merged_rows; - /// TODO: sum_blocks_granularity += block_size; - } - - /// Initialize aggregate descriptions with columns. - void initAggregateDescription(std::vector & columns_to_aggregate) - { - size_t num_columns = columns_to_aggregate.size(); - for (size_t column_number = 0; column_number < num_columns; ++column_number) - columns_to_aggregate[column_number].merged_column = columns[column_number].get(); - } - - Chunk pull(size_t num_result_columns); + Chunk pull(); private: ColumnsDefinition & def; + + bool is_group_started = false; + + Row current_row; + bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. + + void addRowImpl(ColumnRawPtrs & raw_columns, size_t row); + + /// Initialize aggregate descriptions with columns. + void initAggregateDescription(); }; private: - Row current_row; - bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. - + /// Order between members is important because merged_data has reference to columns_definition. ColumnsDefinition columns_definition; SummingMergedData merged_data; - - Names column_names; - - void addRow(SortCursor & cursor); - void insertCurrentRowIfNeeded(); - -public: - /// Stores aggregation function, state, and columns to be used as function arguments. - struct AggregateDescription - { - /// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing. - AggregateFunctionPtr function; - IAggregateFunction::AddFunc add_function = nullptr; - std::vector column_numbers; - IColumn * merged_column = nullptr; - AlignedBuffer state; - bool created = false; - - /// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above. - bool is_agg_func_type = false; - - void init(const char * function_name, const DataTypes & argument_types) - { - function = AggregateFunctionFactory::instance().get(function_name, argument_types); - add_function = function->getAddressOfAddFunction(); - state.reset(function->sizeOfData(), function->alignOfData()); - } - - void createState() - { - if (created) - return; - if (is_agg_func_type) - merged_column->insertDefault(); - else - function->create(state.data()); - created = true; - } - - void destroyState() - { - if (!created) - return; - if (!is_agg_func_type) - function->destroy(state.data()); - created = false; - } - - /// Explicitly destroy aggregation state if the stream is terminated - ~AggregateDescription() - { - destroyState(); - } - - AggregateDescription() = default; - AggregateDescription(AggregateDescription &&) = default; - AggregateDescription(const AggregateDescription &) = delete; - }; }; } From 1ef6e8f7c77bd137a3ab96354aa693e48f1127b4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Apr 2020 23:58:29 +0300 Subject: [PATCH 217/743] Fix build. --- .../Merges/SummingSortedAlgorithm.cpp | 24 +++++++++++-------- .../Merges/SummingSortedAlgorithm.h | 3 --- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/Processors/Merges/SummingSortedAlgorithm.cpp b/src/Processors/Merges/SummingSortedAlgorithm.cpp index 923e890abd7..eb61cbd5333 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/SummingSortedAlgorithm.cpp @@ -38,7 +38,8 @@ struct SummingSortedAlgorithm::AggregateDescription AlignedBuffer state; bool created = false; - /// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above. + /// In case when column has type AggregateFunction: + /// use the aggregate function from itself instead of 'function' above. bool is_agg_func_type = false; void init(const char * function_name, const DataTypes & argument_types) @@ -333,12 +334,13 @@ static SummingSortedAlgorithm::ColumnsDefinition defineColumns( static MutableColumns getMergedDataColumns( const Block & header, - const SummingSortedAlgorithm::ColumnsDefinition & columns_definition) + const SummingSortedAlgorithm::ColumnsDefinition & def) { MutableColumns columns; - columns.reserve(columns_definition.getNumColumns()); + size_t num_columns = def.column_numbers_not_to_aggregate.size() + def.columns_to_aggregate.size(); + columns.reserve(num_columns); - for (auto & desc : columns_definition.columns_to_aggregate) + for (auto & desc : def.columns_to_aggregate) { // Wrap aggregated columns in a tuple to match function signature if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) @@ -354,7 +356,7 @@ static MutableColumns getMergedDataColumns( columns.emplace_back(header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty()); } - for (auto & column_number : columns_definition.column_numbers_not_to_aggregate) + for (auto & column_number : def.column_numbers_not_to_aggregate) columns.emplace_back(header.safeGetByPosition(column_number).type->createColumn()); return columns; @@ -496,12 +498,14 @@ void SummingSortedAlgorithm::SummingMergedData::finishGroup() if (desc.column_numbers.size() == 1) { // Flag row as non-empty if at least one column number if non-zero - current_row_is_zero = current_row_is_zero && desc.merged_column->isDefaultAt(desc.merged_column->size() - 1); + current_row_is_zero = current_row_is_zero + && desc.merged_column->isDefaultAt(desc.merged_column->size() - 1); } else { /// It is sumMapWithOverflow aggregate function. - /// Assume that the row isn't empty in this case (just because it is compatible with previous version) + /// Assume that the row isn't empty in this case + /// (just because it is compatible with previous version) current_row_is_zero = false; } } @@ -574,11 +578,11 @@ void SummingSortedAlgorithm::SummingMergedData::addRowImpl(ColumnRawPtrs & raw_c else { // Gather all source columns into a vector - ColumnRawPtrs columns(desc.column_numbers.size()); + ColumnRawPtrs column_ptrs(desc.column_numbers.size()); for (size_t i = 0; i < desc.column_numbers.size(); ++i) - columns[i] = raw_columns[desc.column_numbers[i]]; + column_ptrs[i] = raw_columns[desc.column_numbers[i]]; - desc.add_function(desc.function.get(), desc.state.data(), columns.data(), row, nullptr); + desc.add_function(desc.function.get(), desc.state.data(), column_ptrs.data(), row, nullptr); } } } diff --git a/src/Processors/Merges/SummingSortedAlgorithm.h b/src/Processors/Merges/SummingSortedAlgorithm.h index 4b750063df5..efa67cebe88 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/SummingSortedAlgorithm.h @@ -46,9 +46,6 @@ public: /// Names of columns from header. Names column_names; - - /// It's not the same as column_names.size() - size_t getNumColumns() const { return column_numbers_not_to_aggregate.size() + columns_to_aggregate.size(); } }; /// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns. From 2543741a9afbb9909c0c52670b13fba606b7fbfb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 00:03:11 +0300 Subject: [PATCH 218/743] Fix build. --- src/Processors/Merges/SummingSortedAlgorithm.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Merges/SummingSortedAlgorithm.cpp b/src/Processors/Merges/SummingSortedAlgorithm.cpp index eb61cbd5333..ad005c77ea8 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/SummingSortedAlgorithm.cpp @@ -448,6 +448,8 @@ SummingSortedAlgorithm::SummingMergedData::SummingMergedData( void SummingSortedAlgorithm::SummingMergedData::startGroup(ColumnRawPtrs & raw_columns, size_t row) { + is_group_started = true; + setRow(current_row, raw_columns, row, def.column_names); /// Reset aggregation states for next row From 93e0313ed46a56ff470e4ddc09046bae38d6da56 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 14 Apr 2020 01:40:07 +0300 Subject: [PATCH 219/743] boop the CI --- .../config/config.d/perf-comparison-tweaks-config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml index 090d8ebe581..e41ab8eb75d 100644 --- a/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/perf-comparison-tweaks-config.xml @@ -1,4 +1,4 @@ - + From 848026ac4cec78c2a7959962602041e852a56902 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 11:48:38 +0300 Subject: [PATCH 220/743] Fix style. --- src/Processors/Merges/AggregatingSortedAlgorithm.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp index 85d45afd168..22fe2902988 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/AggregatingSortedAlgorithm.cpp @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition() = default; AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition(ColumnsDefinition &&) noexcept = default; AggregatingSortedAlgorithm::ColumnsDefinition::~ColumnsDefinition() = default; From 71feedea6ffb1a7277ea2edb015ba47061d73184 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 12:05:29 +0300 Subject: [PATCH 221/743] Remove some code. --- .../Merges/AggregatingSortedTransform.h | 4 +- .../Merges/CollapsingSortedTransform.h | 4 +- .../Merges/GraphiteRollupSortedTransform.h | 4 +- src/Processors/Merges/IMergingTransform.cpp | 209 +----------------- src/Processors/Merges/IMergingTransform.h | 107 +++------ .../Merges/MergingSortedTransform.cpp | 2 +- .../Merges/MergingSortedTransform.h | 2 +- .../Merges/ReplacingSortedTransform.h | 4 +- .../Merges/SummingSortedTransform.h | 4 +- .../Merges/VersionedCollapsingTransform.h | 4 +- 10 files changed, 48 insertions(+), 296 deletions(-) diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index 89e22173a13..4a70f3e0128 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -14,13 +14,13 @@ class ColumnAggregateFunction; * corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type, * when merged, the first value is selected. */ -class AggregatingSortedTransform : public IMergingTransform2 +class AggregatingSortedTransform : public IMergingTransform { public: AggregatingSortedTransform( const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size) - : IMergingTransform2( + : IMergingTransform( num_inputs, header, header, true, header, num_inputs, diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index 5a9ed16ea2a..cc59bac324c 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -18,7 +18,7 @@ namespace DB * If negative by 1 is greater than positive rows, then only the first negative row is written. * Otherwise, a logical error. */ -class CollapsingSortedTransform final : public IMergingTransform2 +class CollapsingSortedTransform final : public IMergingTransform { public: CollapsingSortedTransform( @@ -29,7 +29,7 @@ public: size_t max_block_size, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) - : IMergingTransform2( + : IMergingTransform( num_inputs, header, header, true, header, num_inputs, diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index 0d8493e1eb1..e610903ae5d 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -16,14 +16,14 @@ namespace DB * merge `value` values using the specified aggregate functions, * as well as keeping the maximum value of the `version` column. */ -class GraphiteRollupSortedTransform : public IMergingTransform2 +class GraphiteRollupSortedTransform : public IMergingTransform { public: GraphiteRollupSortedTransform( const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size, Graphite::Params params_, time_t time_of_merge_) - : IMergingTransform2( + : IMergingTransform( num_inputs, header, header, true, header, num_inputs, diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 11ab063dce7..0dc4cd41991 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { @@ -10,196 +9,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -IMergingTransform::IMergingTransform( - size_t num_inputs, - const Block & input_header, - const Block & output_header, - bool have_all_inputs_) - : IProcessor(InputPorts(num_inputs, input_header), {output_header}) - , have_all_inputs(have_all_inputs_) -{ -} - -void IMergingTransform::onNewInput() -{ - throw Exception("onNewInput is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); -} - -void IMergingTransform::addInput() -{ - if (have_all_inputs) - throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); - - inputs.emplace_back(outputs.front().getHeader(), this); - onNewInput(); -} - -void IMergingTransform::setHaveAllInputs() -{ - if (have_all_inputs) - throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR); - - have_all_inputs = true; -} - -void IMergingTransform::requestDataForInput(size_t input_number) -{ - if (need_data) - throw Exception("Data was requested for several inputs in IMergingTransform:" - " " + std::to_string(next_input_to_read) + " and " + std::to_string(input_number), - ErrorCodes::LOGICAL_ERROR); - - need_data = true; - next_input_to_read = input_number; -} - -void IMergingTransform::prepareOutputChunk(MergedData & merged_data) -{ - if (need_data) - return; - - has_output_chunk = (is_finished && merged_data.mergedRows()) || merged_data.hasEnoughRows(); - if (has_output_chunk) - output_chunk = merged_data.pull(); -} - -IProcessor::Status IMergingTransform::prepareInitializeInputs() -{ - /// Add information about inputs. - if (input_states.empty()) - { - input_states.reserve(inputs.size()); - for (auto & input : inputs) - input_states.emplace_back(input); - } - - /// Check for inputs we need. - bool all_inputs_has_data = true; - auto it = inputs.begin(); - for (size_t i = 0; it != inputs.end(); ++i, ++it) - { - auto & input = *it; - if (input.isFinished()) - continue; - - if (input_states[i].is_initialized) - { - // input.setNotNeeded(); - continue; - } - - input.setNeeded(); - - if (!input.hasData()) - { - all_inputs_has_data = false; - continue; - } - - auto chunk = input.pull(); - if (!chunk.hasRows()) - { - - if (!input.isFinished()) - all_inputs_has_data = false; - - continue; - } - - consume(std::move(chunk), i); - input_states[i].is_initialized = true; - } - - if (!all_inputs_has_data) - return Status::NeedData; - - initializeInputs(); - - is_initialized = true; - return Status::Ready; -} - - -IProcessor::Status IMergingTransform::prepare() -{ - if (!have_all_inputs) - return Status::NeedData; - - auto & output = outputs.front(); - - /// Special case for no inputs. - if (inputs.empty()) - { - output.finish(); - onFinish(); - return Status::Finished; - } - - /// Check can output. - - if (output.isFinished()) - { - for (auto & in : inputs) - in.close(); - - onFinish(); - return Status::Finished; - } - - /// Do not disable inputs, so it will work in the same way as with AsynchronousBlockInputStream, like before. - bool is_port_full = !output.canPush(); - - /// Push if has data. - if (has_output_chunk && !is_port_full) - { - output.push(std::move(output_chunk)); - has_output_chunk = false; - } - - if (!is_initialized) - return prepareInitializeInputs(); - - if (is_finished) - { - - if (is_port_full) - return Status::PortFull; - - for (auto & input : inputs) - input.close(); - - outputs.front().finish(); - - onFinish(); - return Status::Finished; - } - - if (need_data) - { - auto & input = input_states[next_input_to_read].port; - if (!input.isFinished()) - { - input.setNeeded(); - - if (!input.hasData()) - return Status::NeedData; - - auto chunk = input.pull(); - if (!chunk.hasRows() && !input.isFinished()) - return Status::NeedData; - - consume(std::move(chunk), next_input_to_read); - } - - need_data = false; - } - - if (is_port_full) - return Status::PortFull; - - return Status::Ready; -} - IMergingTransformBase::IMergingTransformBase( size_t num_inputs, const Block & input_header, @@ -241,7 +50,7 @@ IProcessor::Status IMergingTransformBase::prepareInitializeInputs() for (auto & input : inputs) input_states.emplace_back(input); - init_chunks.resize(inputs.size()); + state.init_chunks.resize(inputs.size()); } /// Check for inputs we need. @@ -277,7 +86,7 @@ IProcessor::Status IMergingTransformBase::prepareInitializeInputs() continue; } - init_chunks[i] = std::move(chunk); + state.init_chunks[i] = std::move(chunk); input_states[i].is_initialized = true; } @@ -318,13 +127,13 @@ IProcessor::Status IMergingTransformBase::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - if (output_chunk && !is_port_full) - output.push(std::move(output_chunk)); + if (state.output_chunk && !is_port_full) + output.push(std::move(state.output_chunk)); if (!is_initialized) return prepareInitializeInputs(); - if (is_finished) + if (state.is_finished) { if (is_port_full) @@ -339,9 +148,9 @@ IProcessor::Status IMergingTransformBase::prepare() return Status::Finished; } - if (need_data) + if (state.need_data) { - auto & input = input_states[next_input_to_read].port; + auto & input = input_states[state.next_input_to_read].port; if (!input.isFinished()) { input.setNeeded(); @@ -353,10 +162,10 @@ IProcessor::Status IMergingTransformBase::prepare() if (!chunk.hasRows() && !input.isFinished()) return Status::NeedData; - input_chunk = std::move(chunk); + state.input_chunk = std::move(chunk); } - need_data = false; + state.need_data = false; } if (is_port_full) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 260f57806a7..2c5213d255b 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -7,66 +7,8 @@ namespace DB { -class MergedData; - -/// Base class for merging transforms. -class IMergingTransform : public IProcessor -{ -public: - IMergingTransform( - size_t num_inputs, - const Block & input_header, - const Block & output_header, - //size_t max_block_size, - //bool use_average_block_size, /// For adaptive granularity. Return chunks with the same avg size as inputs. - bool have_all_inputs_); - - /// Methods to add additional input port. It is possible to do only before the first call of `prepare`. - void addInput(); - /// Need to be called after all inputs are added. (only if have_all_inputs was not specified). - void setHaveAllInputs(); - - Status prepare() override; - -protected: - - virtual void onNewInput(); /// Is called when new input is added. To initialize input's data. - virtual void initializeInputs() = 0; /// Is called after first chunk was read for every input. - virtual void consume(Chunk chunk, size_t input_number) = 0; /// Is called after chunk was consumed from input. - virtual void onFinish() {} /// Is called when all data is processed. - - void requestDataForInput(size_t input_number); /// Call it to say that next chunk of data is required for input. - void prepareOutputChunk(MergedData & merged_data); /// Moves chunk from merged_data to output_chunk if needed. - - /// Profile info. - Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; - - Chunk output_chunk; - bool has_output_chunk = false; - bool is_finished = false; - -private: - /// Processor state. - bool is_initialized = false; - bool need_data = false; - size_t next_input_to_read = 0; - - std::atomic have_all_inputs; - - struct InputState - { - explicit InputState(InputPort & port_) : port(port_) {} - - InputPort & port; - bool is_initialized = false; - }; - - std::vector input_states; - - Status prepareInitializeInputs(); -}; - -/// Base class for merging transforms. +/// Base class for IMergingTransform. +/// It is needed to extract all non-template methods in single translation unit. class IMergingTransformBase : public IProcessor { public: @@ -88,13 +30,18 @@ protected: virtual void onFinish() {} /// Is called when all data is processed. /// Processor state. - Chunk output_chunk; - Chunk input_chunk; - bool is_finished = false; - bool need_data = false; - size_t next_input_to_read = 0; + struct State + { + Chunk output_chunk; + Chunk input_chunk; + bool is_finished = false; + bool need_data = false; + size_t next_input_to_read = 0; - Chunks init_chunks; + Chunks init_chunks; + }; + + State state; private: struct InputState @@ -112,12 +59,13 @@ private: IProcessor::Status prepareInitializeInputs(); }; +/// Implementation of MergingTransform using IMergingAlgorithm. template -class IMergingTransform2 : public IMergingTransformBase +class IMergingTransform : public IMergingTransformBase { public: template - IMergingTransform2( + IMergingTransform( size_t num_inputs, const Block & input_header, const Block & output_header, @@ -130,25 +78,25 @@ public: void work() override { - if (!init_chunks.empty()) - algorithm.initialize(std::move(init_chunks)); + if (!state.init_chunks.empty()) + algorithm.initialize(std::move(state.init_chunks)); - if (input_chunk) - algorithm.consume(std::move(input_chunk), next_input_to_read); + if (state.input_chunk) + algorithm.consume(std::move(state.input_chunk), state.next_input_to_read); IMergingAlgorithm::Status status = algorithm.merge(); if (status.chunk && status.chunk.hasRows()) - output_chunk = std::move(status.chunk); + state.output_chunk = std::move(status.chunk); if (status.required_source >= 0) { - next_input_to_read = status.required_source; - need_data = true; + state.next_input_to_read = status.required_source; + state.need_data = true; } if (status.is_finished) - is_finished = true; + state.is_finished = true; } protected: @@ -158,12 +106,7 @@ protected: Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; private: - using IMergingTransformBase::output_chunk; - using IMergingTransformBase::input_chunk; - using IMergingTransformBase::is_finished; - using IMergingTransformBase::need_data; - using IMergingTransformBase::next_input_to_read; - using IMergingTransformBase::init_chunks; + using IMergingTransformBase::state; }; } diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 68af48062ba..1bd0b289529 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -18,7 +18,7 @@ MergingSortedTransform::MergingSortedTransform( bool quiet_, bool use_average_block_sizes, bool have_all_inputs_) - : IMergingTransform2( + : IMergingTransform( num_inputs, header, header, have_all_inputs_, header, num_inputs, diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 8763fbe1aa2..9e783a0730a 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -8,7 +8,7 @@ namespace DB { /// Merges several sorted inputs into one sorted output. -class MergingSortedTransform final : public IMergingTransform2 +class MergingSortedTransform final : public IMergingTransform { public: MergingSortedTransform( diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index a6c36cedb71..b36db3288cc 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -11,7 +11,7 @@ namespace DB * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), * keeps row with max `version` value. */ -class ReplacingSortedTransform final : public IMergingTransform2 +class ReplacingSortedTransform final : public IMergingTransform { public: ReplacingSortedTransform( @@ -20,7 +20,7 @@ public: size_t max_block_size, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) - : IMergingTransform2( + : IMergingTransform( num_inputs, header, header, true, header, num_inputs, diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index 37859e1b88b..7b7f688a7dd 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -11,7 +11,7 @@ namespace DB * collapses them into one row, summing all the numeric columns except the primary key. * If in all numeric columns, except for the primary key, the result is zero, it deletes the row. */ -class SummingSortedTransform final : public IMergingTransform2 +class SummingSortedTransform final : public IMergingTransform { public: @@ -21,7 +21,7 @@ public: /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. const Names & column_names_to_sum, size_t max_block_size) - : IMergingTransform2( + : IMergingTransform( num_inputs, header, header, true, header, num_inputs, diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index 31b5673ee4d..efd5e96fac7 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -12,7 +12,7 @@ namespace DB * (the columns by which the data is sorted, including specially specified version column), * merges any pair of consecutive rows with opposite sign. */ -class VersionedCollapsingTransform final : public IMergingTransform2 +class VersionedCollapsingTransform final : public IMergingTransform { public: /// Don't need version column. It's in primary key. @@ -22,7 +22,7 @@ public: size_t max_block_size, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) - : IMergingTransform2( + : IMergingTransform( num_inputs, header, header, true, header, num_inputs, From 434a13a8bdd336a66d2bf6a9bff9afdb5daaf17a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 12:21:24 +0300 Subject: [PATCH 222/743] Move algorithm to separate files. --- src/CMakeLists.txt | 1 + src/Processors/Merges/AggregatingSortedTransform.h | 2 +- .../{ => Algorithms}/AggregatingSortedAlgorithm.cpp | 2 +- .../{ => Algorithms}/AggregatingSortedAlgorithm.h | 4 ++-- .../{ => Algorithms}/CollapsingSortedAlgorithm.cpp | 2 +- .../{ => Algorithms}/CollapsingSortedAlgorithm.h | 4 ++-- .../Merges/{ => Algorithms}/FixedSizeDequeWithGaps.h | 0 src/Processors/Merges/{ => Algorithms}/Graphite.h | 0 .../GraphiteRollupSortedAlgorithm.cpp | 2 +- .../{ => Algorithms}/GraphiteRollupSortedAlgorithm.h | 6 +++--- .../Merges/{ => Algorithms}/IMergingAlgorithm.h | 0 .../IMergingAlgorithmWithDelayedChunk.cpp | 6 +----- .../IMergingAlgorithmWithDelayedChunk.h | 4 ++-- .../IMergingAlgorithmWithSharedChunks.cpp | 2 +- .../IMergingAlgorithmWithSharedChunks.h | 4 ++-- src/Processors/Merges/{ => Algorithms}/MergedData.h | 0 .../{ => Algorithms}/MergingSortedAlgorithm.cpp | 2 +- .../Merges/{ => Algorithms}/MergingSortedAlgorithm.h | 4 ++-- .../{ => Algorithms}/ReplacingSortedAlgorithm.cpp | 2 +- .../{ => Algorithms}/ReplacingSortedAlgorithm.h | 4 ++-- src/Processors/Merges/{ => Algorithms}/RowRef.h | 0 .../{ => Algorithms}/SummingSortedAlgorithm.cpp | 11 +++++++---- .../Merges/{ => Algorithms}/SummingSortedAlgorithm.h | 7 ++----- .../{ => Algorithms}/VersionedCollapsingAlgorithm.cpp | 2 +- .../{ => Algorithms}/VersionedCollapsingAlgorithm.h | 6 +++--- src/Processors/Merges/CollapsingSortedTransform.h | 2 +- src/Processors/Merges/GraphiteRollupSortedTransform.h | 2 +- src/Processors/Merges/IMergingTransform.h | 2 +- src/Processors/Merges/MergingSortedTransform.h | 2 +- src/Processors/Merges/ReplacingSortedTransform.h | 2 +- src/Processors/Merges/SummingSortedTransform.h | 2 +- src/Processors/Merges/VersionedCollapsingTransform.h | 2 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- 33 files changed, 45 insertions(+), 48 deletions(-) rename src/Processors/Merges/{ => Algorithms}/AggregatingSortedAlgorithm.cpp (99%) rename src/Processors/Merges/{ => Algorithms}/AggregatingSortedAlgorithm.h (95%) rename src/Processors/Merges/{ => Algorithms}/CollapsingSortedAlgorithm.cpp (98%) rename src/Processors/Merges/{ => Algorithms}/CollapsingSortedAlgorithm.h (93%) rename src/Processors/Merges/{ => Algorithms}/FixedSizeDequeWithGaps.h (100%) rename src/Processors/Merges/{ => Algorithms}/Graphite.h (100%) rename src/Processors/Merges/{ => Algorithms}/GraphiteRollupSortedAlgorithm.cpp (99%) rename src/Processors/Merges/{ => Algorithms}/GraphiteRollupSortedAlgorithm.h (95%) rename src/Processors/Merges/{ => Algorithms}/IMergingAlgorithm.h (100%) rename src/Processors/Merges/{ => Algorithms}/IMergingAlgorithmWithDelayedChunk.cpp (90%) rename src/Processors/Merges/{ => Algorithms}/IMergingAlgorithmWithDelayedChunk.h (90%) rename src/Processors/Merges/{ => Algorithms}/IMergingAlgorithmWithSharedChunks.cpp (96%) rename src/Processors/Merges/{ => Algorithms}/IMergingAlgorithmWithSharedChunks.h (91%) rename src/Processors/Merges/{ => Algorithms}/MergedData.h (100%) rename src/Processors/Merges/{ => Algorithms}/MergingSortedAlgorithm.cpp (99%) rename src/Processors/Merges/{ => Algorithms}/MergingSortedAlgorithm.h (92%) rename src/Processors/Merges/{ => Algorithms}/ReplacingSortedAlgorithm.cpp (97%) rename src/Processors/Merges/{ => Algorithms}/ReplacingSortedAlgorithm.h (88%) rename src/Processors/Merges/{ => Algorithms}/RowRef.h (100%) rename src/Processors/Merges/{ => Algorithms}/SummingSortedAlgorithm.cpp (99%) rename src/Processors/Merges/{ => Algorithms}/SummingSortedAlgorithm.h (91%) rename src/Processors/Merges/{ => Algorithms}/VersionedCollapsingAlgorithm.cpp (98%) rename src/Processors/Merges/{ => Algorithms}/VersionedCollapsingAlgorithm.h (83%) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 6549105318e..2ceba670e2e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -312,6 +312,7 @@ add_object_library(clickhouse_processors_formats_impl Processors/Formats/Impl) add_object_library(clickhouse_processors_transforms Processors/Transforms) add_object_library(clickhouse_processors_sources Processors/Sources) add_object_library(clickhouse_processors_merges Processors/Merges) +add_object_library(clickhouse_processors_merges_algorithms Processors/Merges/Algorithms) if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index 4a70f3e0128..8d38f8b7adf 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp similarity index 99% rename from src/Processors/Merges/AggregatingSortedAlgorithm.cpp rename to src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index 22fe2902988..ff9368ab21e 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Processors/Merges/AggregatingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h similarity index 95% rename from src/Processors/Merges/AggregatingSortedAlgorithm.h rename to src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h index 2457bb54e5e..bce6da52752 100644 --- a/src/Processors/Merges/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h @@ -1,7 +1,7 @@ #pragma once -#include -#include +#include +#include namespace DB { diff --git a/src/Processors/Merges/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp similarity index 98% rename from src/Processors/Merges/CollapsingSortedAlgorithm.cpp rename to src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index a0273a470da..cd3a193105d 100644 --- a/src/Processors/Merges/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Processors/Merges/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h similarity index 93% rename from src/Processors/Merges/CollapsingSortedAlgorithm.h rename to src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 8deacabb9d6..836777b9838 100644 --- a/src/Processors/Merges/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include -#include +#include +#include #include namespace Poco diff --git a/src/Processors/Merges/FixedSizeDequeWithGaps.h b/src/Processors/Merges/Algorithms/FixedSizeDequeWithGaps.h similarity index 100% rename from src/Processors/Merges/FixedSizeDequeWithGaps.h rename to src/Processors/Merges/Algorithms/FixedSizeDequeWithGaps.h diff --git a/src/Processors/Merges/Graphite.h b/src/Processors/Merges/Algorithms/Graphite.h similarity index 100% rename from src/Processors/Merges/Graphite.h rename to src/Processors/Merges/Algorithms/Graphite.h diff --git a/src/Processors/Merges/GraphiteRollupSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp similarity index 99% rename from src/Processors/Merges/GraphiteRollupSortedAlgorithm.cpp rename to src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp index 20015dc508a..f26fe96876f 100644 --- a/src/Processors/Merges/GraphiteRollupSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Processors/Merges/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h similarity index 95% rename from src/Processors/Merges/GraphiteRollupSortedAlgorithm.h rename to src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h index effbc3fd266..eeb0e03b028 100644 --- a/src/Processors/Merges/GraphiteRollupSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h @@ -1,7 +1,7 @@ #pragma once -#include -#include -#include +#include +#include +#include #include namespace DB diff --git a/src/Processors/Merges/IMergingAlgorithm.h b/src/Processors/Merges/Algorithms/IMergingAlgorithm.h similarity index 100% rename from src/Processors/Merges/IMergingAlgorithm.h rename to src/Processors/Merges/Algorithms/IMergingAlgorithm.h diff --git a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp similarity index 90% rename from src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp rename to src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp index 16eac826f16..6777109982e 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.cpp +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp @@ -1,8 +1,4 @@ -// -// Created by nik-kochetov on 4/13/20. -// - -#include +#include namespace DB diff --git a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h similarity index 90% rename from src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h rename to src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h index 214f262a5cc..d02b9dfcb7a 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithDelayedChunk.h +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h @@ -1,7 +1,7 @@ #pragma once -#include -#include +#include +#include #include namespace DB diff --git a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp similarity index 96% rename from src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp rename to src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp index e234b29abbe..bcea74b5f38 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.cpp +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h similarity index 91% rename from src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h rename to src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h index dfce4fd3825..1ef7f540f96 100644 --- a/src/Processors/Merges/IMergingAlgorithmWithSharedChunks.h +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h @@ -1,6 +1,6 @@ #pragma once -#include -#include +#include +#include #include namespace DB diff --git a/src/Processors/Merges/MergedData.h b/src/Processors/Merges/Algorithms/MergedData.h similarity index 100% rename from src/Processors/Merges/MergedData.h rename to src/Processors/Merges/Algorithms/MergedData.h diff --git a/src/Processors/Merges/MergingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp similarity index 99% rename from src/Processors/Merges/MergingSortedAlgorithm.cpp rename to src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp index 383d8fe90de..d06e3c1179a 100644 --- a/src/Processors/Merges/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Processors/Merges/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h similarity index 92% rename from src/Processors/Merges/MergingSortedAlgorithm.h rename to src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index 734caec3424..98c4e296ba5 100644 --- a/src/Processors/Merges/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include -#include +#include +#include #include #include diff --git a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp similarity index 97% rename from src/Processors/Merges/ReplacingSortedAlgorithm.cpp rename to src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index bdebf2aba66..4a0f7493637 100644 --- a/src/Processors/Merges/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB diff --git a/src/Processors/Merges/ReplacingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h similarity index 88% rename from src/Processors/Merges/ReplacingSortedAlgorithm.h rename to src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h index 4dbdf5d7ad8..e312144eda5 100644 --- a/src/Processors/Merges/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include -#include +#include +#include #include namespace Poco diff --git a/src/Processors/Merges/RowRef.h b/src/Processors/Merges/Algorithms/RowRef.h similarity index 100% rename from src/Processors/Merges/RowRef.h rename to src/Processors/Merges/Algorithms/RowRef.h diff --git a/src/Processors/Merges/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp similarity index 99% rename from src/Processors/Merges/SummingSortedAlgorithm.cpp rename to src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index ad005c77ea8..d275ce51cba 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -1,11 +1,14 @@ -#include +#include + +#include +#include +#include +#include #include +#include #include #include -#include -#include #include -#include namespace DB { diff --git a/src/Processors/Merges/SummingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h similarity index 91% rename from src/Processors/Merges/SummingSortedAlgorithm.h rename to src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h index efa67cebe88..c417a217bb5 100644 --- a/src/Processors/Merges/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h @@ -1,11 +1,8 @@ #pragma once -#include -#include +#include +#include #include -#include -#include -#include namespace DB { diff --git a/src/Processors/Merges/VersionedCollapsingAlgorithm.cpp b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp similarity index 98% rename from src/Processors/Merges/VersionedCollapsingAlgorithm.cpp rename to src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp index 50cce60b484..508c4b43949 100644 --- a/src/Processors/Merges/VersionedCollapsingAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Processors/Merges/VersionedCollapsingAlgorithm.h b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h similarity index 83% rename from src/Processors/Merges/VersionedCollapsingAlgorithm.h rename to src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h index 78719d4e3dd..4cb1f1d3850 100644 --- a/src/Processors/Merges/VersionedCollapsingAlgorithm.h +++ b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include #include diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index cc59bac324c..5afaa40b87f 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index e610903ae5d..54f58c0d720 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 2c5213d255b..b5e97efaca6 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 9e783a0730a..a5e6b922e38 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index b36db3288cc..e05a06adb2f 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index 7b7f688a7dd..a1ce9691325 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index efd5e96fac7..69af82655cf 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 30bfde17df3..32740d15ada 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include From fc42851186d02b89828bbe2f75d26c0666874e3f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 12:43:17 +0300 Subject: [PATCH 223/743] Move comments. --- .../Merges/AggregatingSortedTransform.h | 9 ++------- .../Algorithms/AggregatingSortedAlgorithm.h | 6 ++++++ .../Merges/Algorithms/CollapsingSortedAlgorithm.h | 14 +++++++++++++- .../Algorithms/GraphiteRollupSortedAlgorithm.h | 13 ++++++++++++- .../Merges/Algorithms/MergingSortedAlgorithm.h | 1 + .../Merges/Algorithms/ReplacingSortedAlgorithm.h | 6 +++++- .../Merges/Algorithms/SummingSortedAlgorithm.h | 7 ++++++- .../Algorithms/VersionedCollapsingAlgorithm.h | 5 +++++ src/Processors/Merges/CollapsingSortedTransform.h | 13 +------------ .../Merges/GraphiteRollupSortedTransform.h | 15 +++------------ src/Processors/Merges/MergingSortedTransform.h | 2 +- src/Processors/Merges/ReplacingSortedTransform.h | 5 +---- src/Processors/Merges/SummingSortedTransform.h | 6 +----- .../Merges/VersionedCollapsingTransform.h | 6 +----- 14 files changed, 58 insertions(+), 50 deletions(-) diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index 8d38f8b7adf..a0425d4c376 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -8,13 +8,8 @@ namespace DB class ColumnAggregateFunction; -/** Merges several sorted ports to one. - * During this for each group of consecutive identical values of the primary key (the columns by which the data is sorted), - * merges them into one row. When merging, the data is pre-aggregated - merge of states of aggregate functions, - * corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type, - * when merged, the first value is selected. - */ -class AggregatingSortedTransform : public IMergingTransform +/// Implementation of IMergingTransform via AggregatingSortedAlgorithm. +class AggregatingSortedTransform final : public IMergingTransform { public: AggregatingSortedTransform( diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h index bce6da52752..b4819ad030c 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h @@ -6,6 +6,12 @@ namespace DB { +/** Merges several sorted inputs to one. + * During this for each group of consecutive identical values of the primary key (the columns by which the data is sorted), + * merges them into one row. When merging, the data is pre-aggregated - merge of states of aggregate functions, + * corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type, + * when merged, the first value is selected. + */ class AggregatingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChunk { public: diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 836777b9838..4158f55a7cd 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -11,7 +11,19 @@ namespace Poco namespace DB { -class CollapsingSortedAlgorithm : public IMergingAlgorithmWithSharedChunks +/** Merges several sorted inputs to one. + * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), + * keeps no more than one row with the value of the column `sign_column = -1` ("negative row") + * and no more than a row with the value of the column `sign_column = 1` ("positive row"). + * That is, it collapses the records from the change log. + * + * If the number of positive and negative rows is the same, and the last row is positive, then the first negative and last positive rows are written. + * If the number of positive and negative rows is the same, and the last line is negative, it writes nothing. + * If the positive by 1 is greater than the negative rows, then only the last positive row is written. + * If negative by 1 is greater than positive rows, then only the first negative row is written. + * Otherwise, a logical error. + */ +class CollapsingSortedAlgorithm final : public IMergingAlgorithmWithSharedChunks { public: CollapsingSortedAlgorithm( diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h index eeb0e03b028..a0e8f1662aa 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h @@ -7,7 +7,18 @@ namespace DB { -class GraphiteRollupSortedAlgorithm : public IMergingAlgorithmWithSharedChunks +/** Merges several sorted inputs into one. + * + * For each group of consecutive identical values of the `path` column, + * and the same `time` values, rounded to some precision + * (where rounding accuracy depends on the template set for `path` + * and the amount of time elapsed from `time` to the specified time), + * keeps one line, + * performing the rounding of time, + * merge `value` values using the specified aggregate functions, + * as well as keeping the maximum value of the `version` column. + */ +class GraphiteRollupSortedAlgorithm final : public IMergingAlgorithmWithSharedChunks { public: GraphiteRollupSortedAlgorithm( diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index 98c4e296ba5..6ff48b520bd 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -7,6 +7,7 @@ namespace DB { +/// Merges several sorted inputs into one sorted output. class MergingSortedAlgorithm final : public IMergingAlgorithm { public: diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h index e312144eda5..7a193470f89 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h @@ -11,7 +11,11 @@ class Logger; namespace DB { -class ReplacingSortedAlgorithm : public IMergingAlgorithmWithSharedChunks +/** Merges several sorted inputs into one. + * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), + * keeps row with max `version` value. + */ +class ReplacingSortedAlgorithm final : public IMergingAlgorithmWithSharedChunks { public: ReplacingSortedAlgorithm( diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h index c417a217bb5..a38df215ccc 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h @@ -7,7 +7,12 @@ namespace DB { -class SummingSortedAlgorithm : public IMergingAlgorithmWithDelayedChunk +/** Merges several sorted inputs into one. + * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), + * collapses them into one row, summing all the numeric columns except the primary key. + * If in all numeric columns, except for the primary key, the result is zero, it deletes the row. + */ +class SummingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChunk { public: SummingSortedAlgorithm( diff --git a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h index 4cb1f1d3850..a603f04d613 100644 --- a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h +++ b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h @@ -7,6 +7,11 @@ namespace DB { +/** Merges several sorted inputs to one. + * For each group of consecutive identical values of the sorting key + * (the columns by which the data is sorted, including specially specified version column), + * merges any pair of consecutive rows with opposite sign. + */ class VersionedCollapsingAlgorithm : public IMergingAlgorithmWithSharedChunks { public: diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index 5afaa40b87f..d4f40c60938 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -6,18 +6,7 @@ namespace DB { -/** Merges several sorted ports to one. - * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), - * keeps no more than one row with the value of the column `sign_column = -1` ("negative row") - * and no more than a row with the value of the column `sign_column = 1` ("positive row"). - * That is, it collapses the records from the change log. - * - * If the number of positive and negative rows is the same, and the last row is positive, then the first negative and last positive rows are written. - * If the number of positive and negative rows is the same, and the last line is negative, it writes nothing. - * If the positive by 1 is greater than the negative rows, then only the last positive row is written. - * If negative by 1 is greater than positive rows, then only the first negative row is written. - * Otherwise, a logical error. - */ +/// Implementation of IMergingTransform via CollapsingSortedAlgorithm. class CollapsingSortedTransform final : public IMergingTransform { public: diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index 54f58c0d720..5104801aa0d 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -5,18 +5,9 @@ namespace DB { -/** Merges several sorted ports into one. - * - * For each group of consecutive identical values of the `path` column, - * and the same `time` values, rounded to some precision - * (where rounding accuracy depends on the template set for `path` - * and the amount of time elapsed from `time` to the specified time), - * keeps one line, - * performing the rounding of time, - * merge `value` values using the specified aggregate functions, - * as well as keeping the maximum value of the `version` column. - */ -class GraphiteRollupSortedTransform : public IMergingTransform + +/// Implementation of IMergingTransform via GraphiteRollupSortedAlgorithm. +class GraphiteRollupSortedTransform final : public IMergingTransform { public: GraphiteRollupSortedTransform( diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index a5e6b922e38..93bd36d8aec 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -7,7 +7,7 @@ namespace DB { -/// Merges several sorted inputs into one sorted output. +/// Implementation of IMergingTransform via MergingSortedAlgorithm. class MergingSortedTransform final : public IMergingTransform { public: diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index e05a06adb2f..757e19e2cbe 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -7,10 +7,7 @@ namespace DB { -/** Merges several sorted ports into one. - * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), - * keeps row with max `version` value. - */ +/// Implementation of IMergingTransform via ReplacingSortedAlgorithm. class ReplacingSortedTransform final : public IMergingTransform { public: diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index a1ce9691325..6fc22681132 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -6,11 +6,7 @@ namespace DB { -/** Merges several sorted ports into one. - * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), - * collapses them into one row, summing all the numeric columns except the primary key. - * If in all numeric columns, except for the primary key, the result is zero, it deletes the row. - */ +/// Implementation of IMergingTransform via SummingSortedAlgorithm. class SummingSortedTransform final : public IMergingTransform { public: diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index 69af82655cf..f593734c603 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -7,11 +7,7 @@ namespace DB { -/** Merges several sorted ports to one. - * For each group of consecutive identical values of the sorting key - * (the columns by which the data is sorted, including specially specified version column), - * merges any pair of consecutive rows with opposite sign. - */ +/// Implementation of IMergingTransform via VersionedCollapsingAlgorithm. class VersionedCollapsingTransform final : public IMergingTransform { public: From 1ce2e1b93b3eff5562b1c73b759f3374bb4bb2e0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 12:43:53 +0300 Subject: [PATCH 224/743] Move comments. --- src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h index a603f04d613..e5f861a268e 100644 --- a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h +++ b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h @@ -12,7 +12,7 @@ namespace DB * (the columns by which the data is sorted, including specially specified version column), * merges any pair of consecutive rows with opposite sign. */ -class VersionedCollapsingAlgorithm : public IMergingAlgorithmWithSharedChunks +class VersionedCollapsingAlgorithm final : public IMergingAlgorithmWithSharedChunks { public: /// Don't need version column. It's in primary key. From 12f4cfb2c7f272ba26972f075ca6194d43e292b6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 13:04:49 +0300 Subject: [PATCH 225/743] Use processors for final. --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 41 ++++++++++--------- 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 4a7a411f45a..7c621513cab 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -38,18 +38,9 @@ namespace std } #endif -#include -#include #include -#include #include -#include -#include -#include -#include #include -#include -#include #include #include #include @@ -59,6 +50,10 @@ namespace std #include #include #include +#include +#include +#include +#include #include #include #include @@ -1096,16 +1091,14 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( }; BlockInputStreamPtr merged; + ProcessorPtr merged_processor; switch (data.merging_params.mode) { case MergeTreeData::MergingParams::Ordinary: { - auto merged_processor = - std::make_shared(header, pipes.size(), sort_description, max_block_size); - Pipe pipe(std::move(pipes), std::move(merged_processor)); - pipes = Pipes(); - pipes.emplace_back(std::move(pipe)); - return pipes; + merged_processor = std::make_shared(header, pipes.size(), + sort_description, max_block_size); + break; } case MergeTreeData::MergingParams::Collapsing: @@ -1114,28 +1107,36 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( break; case MergeTreeData::MergingParams::Summing: - merged = std::make_shared(streams_to_merge(), + merged_processor = std::make_shared(header, pipes.size(), sort_description, data.merging_params.columns_to_sum, max_block_size); break; case MergeTreeData::MergingParams::Aggregating: - merged = std::make_shared(streams_to_merge(), sort_description, max_block_size); + merged_processor = std::make_shared(header, pipes.size(), + sort_description, max_block_size); break; case MergeTreeData::MergingParams::Replacing: /// TODO Make ReplacingFinalBlockInputStream - merged = std::make_shared(streams_to_merge(), + merged_processor = std::make_shared(header, pipes.size(), sort_description, data.merging_params.version_column, max_block_size); break; case MergeTreeData::MergingParams::VersionedCollapsing: /// TODO Make VersionedCollapsingFinalBlockInputStream - merged = std::make_shared( - streams_to_merge(), sort_description, data.merging_params.sign_column, max_block_size); + merged_processor = std::make_shared(header, pipes.size(), + sort_description, data.merging_params.sign_column, max_block_size); break; case MergeTreeData::MergingParams::Graphite: throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); } + if (merged_processor) + { + Pipe pipe(std::move(pipes), std::move(merged_processor)); + pipes = Pipes(); + pipes.emplace_back(std::move(pipe)); + } + if (merged) pipes.emplace_back(std::make_shared(merged)); From edb3f2f3fca3a69d7aa602568967a407f904c924 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 13:23:16 +0300 Subject: [PATCH 226/743] Remove old code. --- .../AggregatingSortedBlockInputStream.cpp | 249 --------- .../AggregatingSortedBlockInputStream.h | 123 ----- .../CollapsingSortedBlockInputStream.cpp | 201 ------- .../CollapsingSortedBlockInputStream.h | 84 --- .../GraphiteRollupSortedBlockInputStream.cpp | 324 ----------- .../GraphiteRollupSortedBlockInputStream.h | 241 -------- .../MergingSortedBlockInputStream.h | 52 -- .../ReplacingSortedBlockInputStream.cpp | 116 ---- .../ReplacingSortedBlockInputStream.h | 61 -- .../SummingSortedBlockInputStream.cpp | 522 ------------------ .../SummingSortedBlockInputStream.h | 155 ------ ...sionedCollapsingSortedBlockInputStream.cpp | 181 ------ ...ersionedCollapsingSortedBlockInputStream.h | 215 -------- 13 files changed, 2524 deletions(-) delete mode 100644 src/DataStreams/AggregatingSortedBlockInputStream.cpp delete mode 100644 src/DataStreams/AggregatingSortedBlockInputStream.h delete mode 100644 src/DataStreams/CollapsingSortedBlockInputStream.cpp delete mode 100644 src/DataStreams/CollapsingSortedBlockInputStream.h delete mode 100644 src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp delete mode 100644 src/DataStreams/GraphiteRollupSortedBlockInputStream.h delete mode 100644 src/DataStreams/ReplacingSortedBlockInputStream.cpp delete mode 100644 src/DataStreams/ReplacingSortedBlockInputStream.h delete mode 100644 src/DataStreams/SummingSortedBlockInputStream.cpp delete mode 100644 src/DataStreams/SummingSortedBlockInputStream.h delete mode 100644 src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp delete mode 100644 src/DataStreams/VersionedCollapsingSortedBlockInputStream.h diff --git a/src/DataStreams/AggregatingSortedBlockInputStream.cpp b/src/DataStreams/AggregatingSortedBlockInputStream.cpp deleted file mode 100644 index c36ba4968d3..00000000000 --- a/src/DataStreams/AggregatingSortedBlockInputStream.cpp +++ /dev/null @@ -1,249 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -class RemovingLowCardinalityBlockInputStream : public IBlockInputStream -{ -public: - RemovingLowCardinalityBlockInputStream(BlockInputStreamPtr input_, ColumnNumbers positions_) - : input(std::move(input_)), positions(std::move(positions_)) - { - header = transform(input->getHeader()); - } - - Block transform(Block block) - { - if (block) - { - for (auto & pos : positions) - { - auto & col = block.safeGetByPosition(pos); - col.column = recursiveRemoveLowCardinality(col.column); - col.type = recursiveRemoveLowCardinality(col.type); - } - } - - return block; - } - - String getName() const override { return "RemovingLowCardinality"; } - Block getHeader() const override { return header; } - const BlockMissingValues & getMissingValues() const override { return input->getMissingValues(); } - bool isSortedOutput() const override { return input->isSortedOutput(); } - const SortDescription & getSortDescription() const override { return input->getSortDescription(); } - -protected: - Block readImpl() override { return transform(input->read()); } - -private: - Block header; - BlockInputStreamPtr input; - ColumnNumbers positions; -}; - - -AggregatingSortedBlockInputStream::AggregatingSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_) - : MergingSortedBlockInputStream(inputs_, description_, max_block_size_) -{ - /// Fill in the column numbers that need to be aggregated. - for (size_t i = 0; i < num_columns; ++i) - { - ColumnWithTypeAndName & column = header.safeGetByPosition(i); - - /// We leave only states of aggregate functions. - if (!dynamic_cast(column.type.get()) && !dynamic_cast(column.type->getCustomName())) - { - column_numbers_not_to_aggregate.push_back(i); - continue; - } - - /// Included into PK? - SortDescription::const_iterator it = description.begin(); - for (; it != description.end(); ++it) - if (it->column_name == column.name || (it->column_name.empty() && it->column_number == i)) - break; - - if (it != description.end()) - { - column_numbers_not_to_aggregate.push_back(i); - continue; - } - - if (auto simple_aggr = dynamic_cast(column.type->getCustomName())) - { - // simple aggregate function - SimpleAggregateDescription desc{simple_aggr->getFunction(), i}; - if (desc.function->allocatesMemoryInArena()) - allocatesMemoryInArena = true; - - columns_to_simple_aggregate.emplace_back(std::move(desc)); - - if (recursiveRemoveLowCardinality(column.type).get() != column.type.get()) - converted_lc_columns.emplace_back(i); - } - else - { - // standard aggregate function - column_numbers_to_aggregate.push_back(i); - } - } - - result_header = header; - - if (!converted_lc_columns.empty()) - { - for (auto & input : children) - input = std::make_shared(input, converted_lc_columns); - - header = children.at(0)->getHeader(); - } -} - - -Block AggregatingSortedBlockInputStream::readImpl() -{ - if (finished) - return Block(); - - MutableColumns merged_columns; - init(merged_columns); - - if (has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - if (merged_columns.empty()) - return Block(); - - columns_to_aggregate.resize(column_numbers_to_aggregate.size()); - for (size_t i = 0, size = columns_to_aggregate.size(); i < size; ++i) - columns_to_aggregate[i] = typeid_cast(merged_columns[column_numbers_to_aggregate[i]].get()); - - merge(merged_columns, queue_without_collation); - - for (auto & pos : converted_lc_columns) - { - auto & from_type = header.getByPosition(pos).type; - auto & to_type = result_header.getByPosition(pos).type; - merged_columns[pos] = (*recursiveTypeConversion(std::move(merged_columns[pos]), from_type, to_type)).mutate(); - } - - return result_header.cloneWithColumns(std::move(merged_columns)); -} - - -void AggregatingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap & queue) -{ - size_t merged_rows = 0; - - /// We take the rows in the correct order and put them in `merged_block`, while the rows are no more than `max_block_size` - while (queue.isValid()) - { - SortCursor current = queue.current(); - - setPrimaryKeyRef(next_key, current); - - bool key_differs; - - if (current_key.empty()) /// The first key encountered. - { - setPrimaryKeyRef(current_key, current); - key_differs = true; - } - else - key_differs = next_key != current_key; - - /// if there are enough rows accumulated and the last one is calculated completely - if (key_differs && merged_rows >= max_block_size) - { - /// Write the simple aggregation result for the previous group. - insertSimpleAggregationResult(merged_columns); - return; - } - - if (key_differs) - { - current_key.swap(next_key); - - /// We will write the data for the group. We copy the values of ordinary columns. - for (size_t j : column_numbers_not_to_aggregate) - merged_columns[j]->insertFrom(*current->all_columns[j], current->pos); - - /// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function. - for (auto & column_to_aggregate : columns_to_aggregate) - column_to_aggregate->insertDefault(); - - /// Write the simple aggregation result for the previous group. - if (merged_rows > 0) - insertSimpleAggregationResult(merged_columns); - - /// Reset simple aggregation states for next row - for (auto & desc : columns_to_simple_aggregate) - desc.createState(); - - if (allocatesMemoryInArena) - arena = std::make_unique(); - - ++merged_rows; - } - - addRow(current); - - if (!current->isLast()) - { - queue.next(); - } - else - { - /// We fetch the next block from the appropriate source, if there is one. - fetchNextBlock(current, queue); - } - } - - /// Write the simple aggregation result for the previous group. - if (merged_rows > 0) - insertSimpleAggregationResult(merged_columns); - - finished = true; -} - - -void AggregatingSortedBlockInputStream::addRow(SortCursor & cursor) -{ - for (size_t i = 0, size = column_numbers_to_aggregate.size(); i < size; ++i) - { - size_t j = column_numbers_to_aggregate[i]; - columns_to_aggregate[i]->insertMergeFrom(*cursor->all_columns[j], cursor->pos); - } - - for (auto & desc : columns_to_simple_aggregate) - { - auto & col = cursor->all_columns[desc.column_number]; - desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get()); - } -} - -void AggregatingSortedBlockInputStream::insertSimpleAggregationResult(MutableColumns & merged_columns) -{ - for (auto & desc : columns_to_simple_aggregate) - { - desc.function->insertResultInto(desc.state.data(), *merged_columns[desc.column_number]); - desc.destroyState(); - } -} - -} diff --git a/src/DataStreams/AggregatingSortedBlockInputStream.h b/src/DataStreams/AggregatingSortedBlockInputStream.h deleted file mode 100644 index b0387dbcf2b..00000000000 --- a/src/DataStreams/AggregatingSortedBlockInputStream.h +++ /dev/null @@ -1,123 +0,0 @@ -#pragma once - -#include -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -class Arena; - -/** Merges several sorted streams to one. - * During this for each group of consecutive identical values of the primary key (the columns by which the data is sorted), - * merges them into one row. When merging, the data is pre-aggregated - merge of states of aggregate functions, - * corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type, - * when merged, the first value is selected. - */ -class AggregatingSortedBlockInputStream : public MergingSortedBlockInputStream -{ -public: - AggregatingSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_); - - String getName() const override { return "AggregatingSorted"; } - - bool isSortedOutput() const override { return true; } - - Block getHeader() const override { return result_header; } - -protected: - /// Can return 1 more records than max_block_size. - Block readImpl() override; - -private: - Logger * log = &Logger::get("AggregatingSortedBlockInputStream"); - - /// Read finished. - bool finished = false; - - struct SimpleAggregateDescription; - - /// Columns with which numbers should be aggregated. - ColumnNumbers column_numbers_to_aggregate; - ColumnNumbers column_numbers_not_to_aggregate; - std::vector columns_to_aggregate; - std::vector columns_to_simple_aggregate; - - SharedBlockRowRef current_key; /// The current primary key. - SharedBlockRowRef next_key; /// The primary key of the next row. - - Block result_header; - ColumnNumbers converted_lc_columns; - - /** We support two different cursors - with Collation and without. - * Templates are used instead of polymorphic SortCursor and calls to virtual functions. - */ - void merge(MutableColumns & merged_columns, SortingHeap & queue); - - /** Extract all states of aggregate functions and merge them with the current group. - */ - void addRow(SortCursor & cursor); - - /** Insert all values of current row for simple aggregate functions - */ - void insertSimpleAggregationResult(MutableColumns & merged_columns); - - /// Does SimpleAggregateFunction allocates memory in arena? - bool allocatesMemoryInArena = false; - /// Memory pool for SimpleAggregateFunction - /// (only when allocatesMemoryInArena == true). - std::unique_ptr arena; - - /// Stores information for aggregation of SimpleAggregateFunction columns - struct SimpleAggregateDescription - { - /// An aggregate function 'anyLast', 'sum'... - AggregateFunctionPtr function; - IAggregateFunction::AddFunc add_function; - size_t column_number; - AlignedBuffer state; - bool created = false; - - SimpleAggregateDescription(const AggregateFunctionPtr & function_, const size_t column_number_) : function(function_), column_number(column_number_) - { - add_function = function->getAddressOfAddFunction(); - state.reset(function->sizeOfData(), function->alignOfData()); - } - - void createState() - { - if (created) - return; - function->create(state.data()); - created = true; - } - - void destroyState() - { - if (!created) - return; - function->destroy(state.data()); - created = false; - } - - /// Explicitly destroy aggregation state if the stream is terminated - ~SimpleAggregateDescription() - { - destroyState(); - } - - SimpleAggregateDescription() = default; - SimpleAggregateDescription(SimpleAggregateDescription &&) = default; - SimpleAggregateDescription(const SimpleAggregateDescription &) = delete; - }; -}; - -} diff --git a/src/DataStreams/CollapsingSortedBlockInputStream.cpp b/src/DataStreams/CollapsingSortedBlockInputStream.cpp deleted file mode 100644 index ef82a6d8c5e..00000000000 --- a/src/DataStreams/CollapsingSortedBlockInputStream.cpp +++ /dev/null @@ -1,201 +0,0 @@ -#include -#include -#include -#include - -/// Maximum number of messages about incorrect data in the log. -#define MAX_ERROR_MESSAGES 10 - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_DATA; - extern const int LOGICAL_ERROR; -} - - -void CollapsingSortedBlockInputStream::reportIncorrectData() -{ - std::stringstream s; - s << "Incorrect data: number of rows with sign = 1 (" << count_positive - << ") differs with number of rows with sign = -1 (" << count_negative - << ") by more than one (for key: "; - - for (size_t i = 0, size = current_key.size(); i < size; ++i) - { - if (i != 0) - s << ", "; - s << applyVisitor(FieldVisitorToString(), (*(*current_key.columns)[i])[current_key.row_num]); - } - - s << ")."; - - /** Fow now we limit ourselves to just logging such situations, - * since the data is generated by external programs. - * With inconsistent data, this is an unavoidable error that can not be easily corrected by admins. Therefore Warning. - */ - LOG_WARNING(log, s.rdbuf()); -} - - -void CollapsingSortedBlockInputStream::insertRows(MutableColumns & merged_columns, size_t block_size, MergeStopCondition & condition) -{ - if (count_positive == 0 && count_negative == 0) - { - /// No input rows have been read. - return; - } - - if (last_is_positive || count_positive != count_negative) - { - if (count_positive <= count_negative) - { - condition.addRowWithGranularity(block_size); - for (size_t i = 0; i < num_columns; ++i) - merged_columns[i]->insertFrom(*(*first_negative.columns)[i], first_negative.row_num); - - if (out_row_sources_buf) - current_row_sources[first_negative_pos].setSkipFlag(false); - } - - if (count_positive >= count_negative) - { - condition.addRowWithGranularity(block_size); - for (size_t i = 0; i < num_columns; ++i) - merged_columns[i]->insertFrom(*(*last_positive.columns)[i], last_positive.row_num); - - if (out_row_sources_buf) - current_row_sources[last_positive_pos].setSkipFlag(false); - } - - if (!(count_positive == count_negative || count_positive + 1 == count_negative || count_positive == count_negative + 1)) - { - if (count_incorrect_data < MAX_ERROR_MESSAGES) - reportIncorrectData(); - ++count_incorrect_data; - } - } - - if (out_row_sources_buf) - out_row_sources_buf->write( - reinterpret_cast(current_row_sources.data()), - current_row_sources.size() * sizeof(RowSourcePart)); -} - - -Block CollapsingSortedBlockInputStream::readImpl() -{ - if (finished) - return {}; - - MutableColumns merged_columns; - init(merged_columns); - - if (has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - if (merged_columns.empty()) - return {}; - - merge(merged_columns, queue_without_collation); - return header.cloneWithColumns(std::move(merged_columns)); -} - - -void CollapsingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap & queue) -{ - - MergeStopCondition stop_condition(average_block_sizes, max_block_size); - size_t current_block_granularity; - /// Take rows in correct order and put them into `merged_columns` until the rows no more than `max_block_size` - for (; queue.isValid(); ++current_pos) - { - SortCursor current = queue.current(); - current_block_granularity = current->rows; - - if (current_key.empty()) - setPrimaryKeyRef(current_key, current); - - Int8 sign = assert_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; - setPrimaryKeyRef(next_key, current); - - bool key_differs = next_key != current_key; - - /// if there are enough rows and the last one is calculated completely - if (key_differs && stop_condition.checkStop()) - { - ++blocks_written; - return; - } - - if (key_differs) - { - /// We write data for the previous primary key. - insertRows(merged_columns, current_block_granularity, stop_condition); - - current_key.swap(next_key); - - count_negative = 0; - count_positive = 0; - - current_pos = 0; - first_negative_pos = 0; - last_positive_pos = 0; - last_negative_pos = 0; - current_row_sources.resize(0); - } - - /// Initially, skip all rows. On insert, unskip "corner" rows. - if (out_row_sources_buf) - current_row_sources.emplace_back(current.impl->order, true); - - if (sign == 1) - { - ++count_positive; - last_is_positive = true; - - setRowRef(last_positive, current); - last_positive_pos = current_pos; - } - else if (sign == -1) - { - if (!count_negative) - { - setRowRef(first_negative, current); - first_negative_pos = current_pos; - } - - if (!blocks_written && stop_condition.empty()) - { - setRowRef(last_negative, current); - last_negative_pos = current_pos; - } - - ++count_negative; - last_is_positive = false; - } - else - throw Exception("Incorrect data: Sign = " + toString(sign) + " (must be 1 or -1).", - ErrorCodes::INCORRECT_DATA); - - if (!current->isLast()) - { - queue.next(); - } - else - { - /// We take next block from the corresponding source, if there is one. - fetchNextBlock(current, queue); - } - } - - /// Write data for last primary key. - insertRows(merged_columns, /*some_granularity*/ 0, stop_condition); - - finished = true; -} - -} diff --git a/src/DataStreams/CollapsingSortedBlockInputStream.h b/src/DataStreams/CollapsingSortedBlockInputStream.h deleted file mode 100644 index 2b528d27339..00000000000 --- a/src/DataStreams/CollapsingSortedBlockInputStream.h +++ /dev/null @@ -1,84 +0,0 @@ -#pragma once - -#include - -#include -#include - - -namespace DB -{ - -/** Merges several sorted streams to one. - * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), - * keeps no more than one row with the value of the column `sign_column = -1` ("negative row") - * and no more than a row with the value of the column `sign_column = 1` ("positive row"). - * That is, it collapses the records from the change log. - * - * If the number of positive and negative rows is the same, and the last row is positive, then the first negative and last positive rows are written. - * If the number of positive and negative rows is the same, and the last line is negative, it writes nothing. - * If the positive by 1 is greater than the negative rows, then only the last positive row is written. - * If negative by 1 is greater than positive rows, then only the first negative row is written. - * Otherwise, a logical error. - */ -class CollapsingSortedBlockInputStream : public MergingSortedBlockInputStream -{ -public: - CollapsingSortedBlockInputStream( - BlockInputStreams inputs_, const SortDescription & description_, - const String & sign_column, size_t max_block_size_, - WriteBuffer * out_row_sources_buf_ = nullptr, bool average_block_sizes_ = false) - : MergingSortedBlockInputStream(inputs_, description_, max_block_size_, 0, out_row_sources_buf_, false, average_block_sizes_) - { - sign_column_number = header.getPositionByName(sign_column); - } - - String getName() const override { return "CollapsingSorted"; } - -protected: - /// Can return 1 more records than max_block_size. - Block readImpl() override; - -private: - size_t sign_column_number; - - Logger * log = &Logger::get("CollapsingSortedBlockInputStream"); - - /// Read is finished. - bool finished = false; - - SharedBlockRowRef current_key; /// The current primary key. - SharedBlockRowRef next_key; /// The primary key of the next row. - - SharedBlockRowRef first_negative; /// The first negative row for the current primary key. - SharedBlockRowRef last_positive; /// The last positive row for the current primary key. - SharedBlockRowRef last_negative; /// Last negative row. It is only stored if there is not one row is written to output. - - size_t count_positive = 0; /// The number of positive rows for the current primary key. - size_t count_negative = 0; /// The number of negative rows for the current primary key. - bool last_is_positive = false; /// true if the last row for the current primary key is positive. - - size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log. - - size_t blocks_written = 0; - - /// Fields specific for VERTICAL merge algorithm. - /// Row numbers are relative to the start of current primary key. - size_t current_pos = 0; /// Current row number - size_t first_negative_pos = 0; /// Row number of first_negative - size_t last_positive_pos = 0; /// Row number of last_positive - size_t last_negative_pos = 0; /// Row number of last_negative - PODArray current_row_sources; /// Sources of rows with the current primary key - - /** We support two different cursors - with Collation and without. - * Templates are used instead of polymorphic SortCursors and calls to virtual functions. - */ - void merge(MutableColumns & merged_columns, SortingHeap & queue); - - /// Output to result rows for the current primary key. - void insertRows(MutableColumns & merged_columns, size_t block_size, MergeStopCondition & condition); - - void reportIncorrectData(); -}; - -} diff --git a/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp b/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp deleted file mode 100644 index 1b81881ab1a..00000000000 --- a/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp +++ /dev/null @@ -1,324 +0,0 @@ -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -GraphiteRollupSortedBlockInputStream::GraphiteRollupSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_, - const Graphite::Params & params_, time_t time_of_merge_) - : MergingSortedBlockInputStream(inputs_, description_, max_block_size_), - params(params_), time_of_merge(time_of_merge_) -{ - size_t max_size_of_aggregate_state = 0; - size_t max_alignment_of_aggregate_state = 1; - - for (const auto & pattern : params.patterns) - { - if (pattern.function) - { - max_size_of_aggregate_state = std::max(max_size_of_aggregate_state, pattern.function->sizeOfData()); - max_alignment_of_aggregate_state = std::max(max_alignment_of_aggregate_state, pattern.function->alignOfData()); - } - } - - place_for_aggregate_state.reset(max_size_of_aggregate_state, max_alignment_of_aggregate_state); - - /// Memoize column numbers in block. - path_column_num = header.getPositionByName(params.path_column_name); - time_column_num = header.getPositionByName(params.time_column_name); - value_column_num = header.getPositionByName(params.value_column_name); - version_column_num = header.getPositionByName(params.version_column_name); - - for (size_t i = 0; i < num_columns; ++i) - if (i != time_column_num && i != value_column_num && i != version_column_num) - unmodified_column_numbers.push_back(i); -} - - -Graphite::RollupRule GraphiteRollupSortedBlockInputStream::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 GraphiteRollupSortedBlockInputStream::selectPrecision(const Graphite::Retentions & retentions, time_t time) const -{ - static_assert(is_signed_v, "time_t must be signed type"); - - for (const auto & retention : retentions) - { - if (time_of_merge - time >= static_cast(retention.age)) - return retention.precision; - } - - /// No rounding. - return 1; -} - - -/** Round the unix timestamp to seconds precision. - * In this case, the date should not change. The date is calculated using the local time zone. - * - * If the rounding value is less than an hour, - * then, assuming that time zones that differ from UTC by a non-integer number of hours are not supported, - * just simply round the unix timestamp down to a multiple of 3600. - * And if the rounding value is greater, - * then we will round down the number of seconds from the beginning of the day in the local time zone. - * - * Rounding to more than a day is not supported. - */ -static time_t roundTimeToPrecision(const DateLUTImpl & date_lut, time_t time, UInt32 precision) -{ - if (precision <= 3600) - { - return time / precision * precision; - } - else - { - time_t date = date_lut.toDate(time); - time_t remainder = time - date; - return date + remainder / precision * precision; - } -} - - -Block GraphiteRollupSortedBlockInputStream::readImpl() -{ - if (finished) - return Block(); - - MutableColumns merged_columns; - init(merged_columns); - - if (has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - if (merged_columns.empty()) - return Block(); - - merge(merged_columns, queue_without_collation); - return header.cloneWithColumns(std::move(merged_columns)); -} - - -void GraphiteRollupSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap & queue) -{ - const DateLUTImpl & date_lut = DateLUT::instance(); - - size_t started_rows = 0; /// Number of times startNextRow() has been called. - - /// Take rows in needed order and put them into `merged_columns` until we get `max_block_size` rows. - /// - /// Variables starting with current_* refer to the rows previously popped from the queue that will - /// contribute towards current output row. - /// Variables starting with next_* refer to the row at the top of the queue. - - while (queue.isValid()) - { - SortCursor next_cursor = queue.current(); - - StringRef next_path = next_cursor->all_columns[path_column_num]->getDataAt(next_cursor->pos); - bool new_path = is_first || next_path != current_group_path; - - is_first = false; - - time_t next_row_time = next_cursor->all_columns[time_column_num]->getUInt(next_cursor->pos); - /// Is new key before rounding. - bool is_new_key = new_path || next_row_time != current_time; - - if (is_new_key) - { - /// Accumulate the row that has maximum version in the previous group of rows with the same key: - if (started_rows) - accumulateRow(current_subgroup_newest_row); - - Graphite::RollupRule next_rule = current_rule; - if (new_path) - next_rule = selectPatternForPath(next_path); - - const Graphite::RetentionPattern * retention_pattern = std::get<0>(next_rule); - time_t next_time_rounded; - if (retention_pattern) - { - UInt32 precision = selectPrecision(retention_pattern->retentions, next_row_time); - next_time_rounded = roundTimeToPrecision(date_lut, next_row_time, precision); - } - else - { - /// If no pattern has matched - take the value as-is. - next_time_rounded = next_row_time; - } - - /// Key will be new after rounding. It means new result row. - bool will_be_new_key = new_path || next_time_rounded != current_time_rounded; - - if (will_be_new_key) - { - if (started_rows) - { - finishCurrentGroup(merged_columns); - - /// We have enough rows - return, but don't advance the loop. At the beginning of the - /// next call to merge() the same next_cursor will be processed once more and - /// the next output row will be created from it. - if (started_rows >= max_block_size) - return; - } - - /// At this point previous row has been fully processed, so we can advance the loop - /// (substitute current_* values for next_*, advance the cursor). - - startNextGroup(merged_columns, next_cursor, next_rule); - ++started_rows; - - current_time_rounded = next_time_rounded; - } - - current_time = next_row_time; - } - - /// Within all rows with same key, we should leave only one row with maximum version; - /// and for rows with same maximum version - only last row. - if (is_new_key - || next_cursor->all_columns[version_column_num]->compareAt( - next_cursor->pos, current_subgroup_newest_row.row_num, - *(*current_subgroup_newest_row.columns)[version_column_num], - /* nan_direction_hint = */ 1) >= 0) - { - setRowRef(current_subgroup_newest_row, next_cursor); - - /// Small hack: group and subgroups have the same path, so we can set current_group_path here instead of startNextGroup - /// But since we keep in memory current_subgroup_newest_row's block, we could use StringRef for current_group_path and don't - /// make deep copy of the path. - current_group_path = next_path; - } - - if (!next_cursor->isLast()) - { - queue.next(); - } - else - { - /// We get the next block from the appropriate source, if there is one. - fetchNextBlock(next_cursor, queue); - } - } - - /// Write result row for the last group. - if (started_rows) - { - accumulateRow(current_subgroup_newest_row); - finishCurrentGroup(merged_columns); - } - - finished = true; -} - - -template -void GraphiteRollupSortedBlockInputStream::startNextGroup(MutableColumns & merged_columns, TSortCursor & cursor, - Graphite::RollupRule next_rule) -{ - const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(next_rule); - - /// Copy unmodified column values (including path column). - for (size_t j : unmodified_column_numbers) - merged_columns[j]->insertFrom(*cursor->all_columns[j], cursor->pos); - - if (aggregation_pattern) - { - aggregation_pattern->function->create(place_for_aggregate_state.data()); - aggregate_state_created = true; - } - - current_rule = next_rule; -} - - -void GraphiteRollupSortedBlockInputStream::finishCurrentGroup(MutableColumns & merged_columns) -{ - /// Insert calculated values of the columns `time`, `value`, `version`. - merged_columns[time_column_num]->insert(current_time_rounded); - merged_columns[version_column_num]->insertFrom( - *(*current_subgroup_newest_row.columns)[version_column_num], current_subgroup_newest_row.row_num); - - const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule); - if (aggregate_state_created) - { - aggregation_pattern->function->insertResultInto(place_for_aggregate_state.data(), *merged_columns[value_column_num]); - aggregation_pattern->function->destroy(place_for_aggregate_state.data()); - aggregate_state_created = false; - } - else - merged_columns[value_column_num]->insertFrom( - *(*current_subgroup_newest_row.columns)[value_column_num], current_subgroup_newest_row.row_num); -} - - -void GraphiteRollupSortedBlockInputStream::accumulateRow(SharedBlockRowRef & row) -{ - const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule); - if (aggregate_state_created) - aggregation_pattern->function->add(place_for_aggregate_state.data(), &(*row.columns)[value_column_num], row.row_num, nullptr); -} - -} diff --git a/src/DataStreams/GraphiteRollupSortedBlockInputStream.h b/src/DataStreams/GraphiteRollupSortedBlockInputStream.h deleted file mode 100644 index bfaeff7733b..00000000000 --- a/src/DataStreams/GraphiteRollupSortedBlockInputStream.h +++ /dev/null @@ -1,241 +0,0 @@ -#pragma once - -#include - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -/** Intended for implementation of "rollup" - aggregation (rounding) of older data - * for a table with Graphite data (Graphite is the system for time series monitoring). - * - * Table with graphite data has at least the following columns (accurate to the name): - * Path, Time, Value, Version - * - * Path - name of metric (sensor); - * Time - time of measurement; - * Value - value of measurement; - * Version - a number, that for equal pairs of Path and Time, need to leave only record with maximum version. - * - * Each row in a table correspond to one value of one sensor. - * - * Pattern should contain function, retention scheme, or both of them. The order of patterns does mean as well: - * * Aggregation OR retention patterns should be first - * * Then aggregation AND retention full patterns have to be placed - * * default pattern without regexp must be the last - * - * Rollup rules are specified in the following way: - * - * pattern - * regexp - * function - * pattern - * regexp - * age -> precision - * age -> precision - * ... - * pattern - * regexp - * function - * age -> precision - * age -> precision - * ... - * pattern - * ... - * default - * function - * age -> precision - * ... - * - * regexp - pattern for sensor name - * default - if no pattern has matched - * - * age - minimal data age (in seconds), to start rounding with specified precision. - * precision - rounding precision (in seconds) - * - * function - name of aggregate function to be applied for values, that time was rounded to same. - * - * Example: - * - * - * - * \.max$ - * max - * - * - * click_cost - * any - * - * 0 - * 5 - * - * - * 86400 - * 60 - * - * - * - * max - * - * 0 - * 60 - * - * - * 3600 - * 300 - * - * - * 86400 - * 3600 - * - * - * - */ - -namespace Graphite -{ - struct Retention - { - UInt32 age; - UInt32 precision; - }; - - using Retentions = std::vector; - - struct Pattern - { - std::shared_ptr regexp; - std::string regexp_str; - AggregateFunctionPtr function; - Retentions retentions; /// Must be ordered by 'age' descending. - enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically - }; - - using Patterns = std::vector; - using RetentionPattern = Pattern; - using AggregationPattern = Pattern; - - struct Params - { - String config_name; - String path_column_name; - String time_column_name; - String value_column_name; - String version_column_name; - Graphite::Patterns patterns; - }; - - using RollupRule = std::pair; -} - -/** Merges several sorted streams into one. - * - * For each group of consecutive identical values of the `path` column, - * and the same `time` values, rounded to some precision - * (where rounding accuracy depends on the template set for `path` - * and the amount of time elapsed from `time` to the specified time), - * keeps one line, - * performing the rounding of time, - * merge `value` values using the specified aggregate functions, - * as well as keeping the maximum value of the `version` column. - */ -class GraphiteRollupSortedBlockInputStream : public MergingSortedBlockInputStream -{ -public: - GraphiteRollupSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_, - const Graphite::Params & params_, time_t time_of_merge_); - - String getName() const override { return "GraphiteRollupSorted"; } - - ~GraphiteRollupSortedBlockInputStream() override - { - if (aggregate_state_created) - std::get<1>(current_rule)->function->destroy(place_for_aggregate_state.data()); - } - -protected: - Block readImpl() override; - -private: - Logger * log = &Logger::get("GraphiteRollupSortedBlockInputStream"); - - const Graphite::Params params; - - size_t path_column_num; - size_t time_column_num; - size_t value_column_num; - size_t version_column_num; - - /// All columns other than 'time', 'value', 'version'. They are unmodified during rollup. - ColumnNumbers unmodified_column_numbers; - - time_t time_of_merge; - - /// No data has been read. - bool is_first = true; - - /// All data has been read. - bool finished = false; - - /* | path | time | rounded_time | version | value | unmodified | - * ----------------------------------------------------------------------------------- - * | A | 11 | 10 | 1 | 1 | a | | - * | A | 11 | 10 | 3 | 2 | b |> subgroup(A, 11) | - * | A | 11 | 10 | 2 | 3 | c | |> group(A, 10) - * ----------------------------------------------------------------------------------|> - * | A | 12 | 10 | 0 | 4 | d | |> Outputs (A, 10, avg(2, 5), a) - * | A | 12 | 10 | 1 | 5 | e |> subgroup(A, 12) | - * ----------------------------------------------------------------------------------- - * | A | 21 | 20 | 1 | 6 | f | - * | B | 11 | 10 | 1 | 7 | g | - * ... - */ - - /// Path name of current bucket - StringRef current_group_path; - - /// Last row with maximum version for current primary key (time bucket). - SharedBlockRowRef current_subgroup_newest_row; - - /// Time of last read row - time_t current_time = 0; - time_t current_time_rounded = 0; - - Graphite::RollupRule current_rule = {nullptr, nullptr}; - AlignedBuffer place_for_aggregate_state; - bool aggregate_state_created = false; /// Invariant: if true then current_rule is not NULL. - - const Graphite::Pattern undef_pattern = - { /// temporary empty pattern for selectPatternForPath - nullptr, - "", - nullptr, - DB::Graphite::Retentions(), - undef_pattern.TypeUndef, - }; - Graphite::RollupRule selectPatternForPath(StringRef path) const; - UInt32 selectPrecision(const Graphite::Retentions & retentions, time_t time) const; - - - void merge(MutableColumns & merged_columns, SortingHeap & queue); - - /// Insert the values into the resulting columns, which will not be changed in the future. - template - void startNextGroup(MutableColumns & merged_columns, TSortCursor & cursor, Graphite::RollupRule next_rule); - - /// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows. - void finishCurrentGroup(MutableColumns & merged_columns); - - /// Update the state of the aggregate function with the new `value`. - void accumulateRow(SharedBlockRowRef & row); -}; - -} diff --git a/src/DataStreams/MergingSortedBlockInputStream.h b/src/DataStreams/MergingSortedBlockInputStream.h index d9c9f1c26cc..726d67d432a 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.h +++ b/src/DataStreams/MergingSortedBlockInputStream.h @@ -116,58 +116,6 @@ protected: /// If it is not nullptr then it should be populated during execution WriteBuffer * out_row_sources_buf; - - /// These methods are used in Collapsing/Summing/Aggregating... SortedBlockInputStream-s. - - /// Save the row pointed to by cursor in `row`. - template - void setRow(Row & row, TSortCursor & cursor) - { - for (size_t i = 0; i < num_columns; ++i) - { - try - { - cursor->all_columns[i]->get(cursor->pos, row[i]); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - - /// Find out the name of the column and throw more informative exception. - - String column_name; - for (const auto & block : source_blocks) - { - if (i < block->columns()) - { - column_name = block->safeGetByPosition(i).name; - break; - } - } - - throw Exception("MergingSortedBlockInputStream failed to read row " + toString(cursor->pos) - + " of column " + toString(i) + (column_name.empty() ? "" : " (" + column_name + ")"), - ErrorCodes::CORRUPTED_DATA); - } - } - } - - template - void setRowRef(SharedBlockRowRef & row_ref, TSortCursor & cursor) - { - row_ref.row_num = cursor.impl->pos; - row_ref.shared_block = source_blocks[cursor.impl->order]; - row_ref.columns = &row_ref.shared_block->all_columns; - } - - template - void setPrimaryKeyRef(SharedBlockRowRef & row_ref, TSortCursor & cursor) - { - row_ref.row_num = cursor.impl->pos; - row_ref.shared_block = source_blocks[cursor.impl->order]; - row_ref.columns = &row_ref.shared_block->sort_columns; - } - private: /** We support two different cursors - with Collation and without. diff --git a/src/DataStreams/ReplacingSortedBlockInputStream.cpp b/src/DataStreams/ReplacingSortedBlockInputStream.cpp deleted file mode 100644 index 967b4ebb046..00000000000 --- a/src/DataStreams/ReplacingSortedBlockInputStream.cpp +++ /dev/null @@ -1,116 +0,0 @@ -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -void ReplacingSortedBlockInputStream::insertRow(MutableColumns & merged_columns) -{ - if (out_row_sources_buf) - { - /// true flag value means "skip row" - current_row_sources[max_pos].setSkipFlag(false); - - out_row_sources_buf->write(reinterpret_cast(current_row_sources.data()), - current_row_sources.size() * sizeof(RowSourcePart)); - current_row_sources.resize(0); - } - - for (size_t i = 0; i < num_columns; ++i) - merged_columns[i]->insertFrom(*(*selected_row.columns)[i], selected_row.row_num); -} - - -Block ReplacingSortedBlockInputStream::readImpl() -{ - if (finished) - return Block(); - - MutableColumns merged_columns; - init(merged_columns); - - if (has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - if (merged_columns.empty()) - return Block(); - - merge(merged_columns, queue_without_collation); - return header.cloneWithColumns(std::move(merged_columns)); -} - - -void ReplacingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap & queue) -{ - MergeStopCondition stop_condition(average_block_sizes, max_block_size); - - /// Take the rows in needed order and put them into `merged_columns` until rows no more than `max_block_size` - while (queue.isValid()) - { - SortCursor current = queue.current(); - size_t current_block_granularity = current->rows; - - if (current_key.empty()) - setPrimaryKeyRef(current_key, current); - - setPrimaryKeyRef(next_key, current); - - bool key_differs = next_key != current_key; - - /// if there are enough rows and the last one is calculated completely - if (key_differs && stop_condition.checkStop()) - return; - - if (key_differs) - { - /// Write the data for the previous primary key. - insertRow(merged_columns); - stop_condition.addRowWithGranularity(current_block_granularity); - selected_row.reset(); - current_key.swap(next_key); - } - - /// Initially, skip all rows. Unskip last on insert. - size_t current_pos = current_row_sources.size(); - if (out_row_sources_buf) - current_row_sources.emplace_back(current.impl->order, true); - - /// A non-strict comparison, since we select the last row for the same version values. - if (version_column_number == -1 - || selected_row.empty() - || current->all_columns[version_column_number]->compareAt( - current->pos, selected_row.row_num, - *(*selected_row.columns)[version_column_number], - /* nan_direction_hint = */ 1) >= 0) - { - max_pos = current_pos; - setRowRef(selected_row, current); - } - - if (!current->isLast()) - { - queue.next(); - } - else - { - /// We get the next block from the corresponding source, if there is one. - fetchNextBlock(current, queue); - } - } - - /// We will write the data for the last primary key. - if (!selected_row.empty()) - insertRow(merged_columns); - - finished = true; -} - -} diff --git a/src/DataStreams/ReplacingSortedBlockInputStream.h b/src/DataStreams/ReplacingSortedBlockInputStream.h deleted file mode 100644 index 22920c2eb20..00000000000 --- a/src/DataStreams/ReplacingSortedBlockInputStream.h +++ /dev/null @@ -1,61 +0,0 @@ -#pragma once - -#include - -#include -#include - - -namespace DB -{ - -/** Merges several sorted streams into one. - * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), - * keeps row with max `version` value. - */ -class ReplacingSortedBlockInputStream : public MergingSortedBlockInputStream -{ -public: - ReplacingSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, - const String & version_column, size_t max_block_size_, WriteBuffer * out_row_sources_buf_ = nullptr, - bool average_block_sizes_ = false) - : MergingSortedBlockInputStream(inputs_, description_, max_block_size_, 0, out_row_sources_buf_, false, average_block_sizes_) - { - if (!version_column.empty()) - version_column_number = header.getPositionByName(version_column); - } - - String getName() const override { return "ReplacingSorted"; } - -protected: - /// Can return 1 more records than max_block_size. - Block readImpl() override; - -private: - ssize_t version_column_number = -1; - - Logger * log = &Logger::get("ReplacingSortedBlockInputStream"); - - /// All data has been read. - bool finished = false; - - /// Primary key of current row. - SharedBlockRowRef current_key; - /// Primary key of next row. - SharedBlockRowRef next_key; - /// Last row with maximum version for current primary key. - SharedBlockRowRef selected_row; - /// The position (into current_row_sources) of the row with the highest version. - size_t max_pos = 0; - - /// Sources of rows with the current primary key. - PODArray current_row_sources; - - void merge(MutableColumns & merged_columns, SortingHeap & queue); - - /// Output into result the rows for current primary key. - void insertRow(MutableColumns & merged_columns); -}; - -} diff --git a/src/DataStreams/SummingSortedBlockInputStream.cpp b/src/DataStreams/SummingSortedBlockInputStream.cpp deleted file mode 100644 index ed5b1b820b4..00000000000 --- a/src/DataStreams/SummingSortedBlockInputStream.cpp +++ /dev/null @@ -1,522 +0,0 @@ -#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; -} - - -namespace -{ - bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number) - { - for (auto & desc : description) - if (desc.column_name == name || (desc.column_name.empty() && desc.column_number == number)) - return true; - - return false; - } - - /// Returns true if merge result is not empty - bool mergeMap(const SummingSortedBlockInputStream::MapDescription & desc, Row & row, SortCursor & cursor) - { - /// Strongly non-optimal. - - Row & left = row; - Row right(left.size()); - - for (size_t col_num : desc.key_col_nums) - right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); - - for (size_t col_num : desc.val_col_nums) - right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get(); - - auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & - { - return matrix[i].get()[j]; - }; - - auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array - { - size_t size = col_nums.size(); - Array res(size); - for (size_t col_num_index = 0; col_num_index < size; ++col_num_index) - res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j); - return res; - }; - - std::map merged; - - auto accumulate = [](Array & dst, const Array & src) - { - bool has_non_zero = false; - size_t size = dst.size(); - for (size_t i = 0; i < size; ++i) - if (applyVisitor(FieldVisitorSum(src[i]), dst[i])) - has_non_zero = true; - return has_non_zero; - }; - - auto merge = [&](const Row & matrix) - { - size_t rows = matrix[desc.key_col_nums[0]].get().size(); - - for (size_t j = 0; j < rows; ++j) - { - Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j); - Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j); - - auto it = merged.find(key); - if (merged.end() == it) - merged.emplace(std::move(key), std::move(value)); - else - { - if (!accumulate(it->second, value)) - merged.erase(it); - } - } - }; - - merge(left); - merge(right); - - for (size_t col_num : desc.key_col_nums) - row[col_num] = Array(merged.size()); - for (size_t col_num : desc.val_col_nums) - row[col_num] = Array(merged.size()); - - size_t row_num = 0; - for (const auto & key_value : merged) - { - for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; - - for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; - - ++row_num; - } - - return row_num != 0; - } -} - - -SummingSortedBlockInputStream::SummingSortedBlockInputStream( - const BlockInputStreams & inputs_, - const SortDescription & description_, - /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. - const Names & column_names_to_sum, - size_t max_block_size_) - : MergingSortedBlockInputStream(inputs_, description_, max_block_size_), - log(&Logger::get("SummingSortedBlockInputStream")) -{ - current_row.resize(num_columns); - - /// name of nested structure -> the column numbers that refer to it. - std::unordered_map> discovered_maps; - - /** Fill in the column numbers, which must be summed. - * This can only be numeric columns that are not part of the sort key. - * If a non-empty column_names_to_sum is specified, then we only take these columns. - * Some columns from column_names_to_sum may not be found. This is ignored. - */ - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnWithTypeAndName & column = header.safeGetByPosition(i); - - /// Discover nested Maps and find columns for summation - if (typeid_cast(column.type.get())) - { - const auto map_name = Nested::extractTableName(column.name); - /// if nested table name ends with `Map` it is a possible candidate for special handling - if (map_name == column.name || !endsWith(map_name, "Map")) - { - column_numbers_not_to_aggregate.push_back(i); - continue; - } - - discovered_maps[map_name].emplace_back(i); - } - else - { - bool is_agg_func = WhichDataType(column.type).isAggregateFunction(); - - /// There are special const columns for example after prewere sections. - if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column)) - { - column_numbers_not_to_aggregate.push_back(i); - continue; - } - - /// Are they inside the PK? - if (isInPrimaryKey(description, column.name, i)) - { - column_numbers_not_to_aggregate.push_back(i); - continue; - } - - if (column_names_to_sum.empty() - || column_names_to_sum.end() != - std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name)) - { - // Create aggregator to sum this column - AggregateDescription desc; - desc.is_agg_func_type = is_agg_func; - desc.column_numbers = {i}; - - if (!is_agg_func) - { - desc.init("sumWithOverflow", {column.type}); - } - - columns_to_aggregate.emplace_back(std::move(desc)); - } - else - { - // Column is not going to be summed, use last value - column_numbers_not_to_aggregate.push_back(i); - } - } - } - - /// select actual nested Maps from list of candidates - for (const auto & map : discovered_maps) - { - /// map should contain at least two elements (key -> value) - if (map.second.size() < 2) - { - for (auto col : map.second) - column_numbers_not_to_aggregate.push_back(col); - continue; - } - - /// no elements of map could be in primary key - auto column_num_it = map.second.begin(); - for (; column_num_it != map.second.end(); ++column_num_it) - if (isInPrimaryKey(description, header.safeGetByPosition(*column_num_it).name, *column_num_it)) - break; - if (column_num_it != map.second.end()) - { - for (auto col : map.second) - column_numbers_not_to_aggregate.push_back(col); - continue; - } - - DataTypes argument_types; - AggregateDescription desc; - MapDescription map_desc; - - column_num_it = map.second.begin(); - for (; column_num_it != map.second.end(); ++column_num_it) - { - const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it); - const String & name = key_col.name; - const IDataType & nested_type = *static_cast(key_col.type.get())->getNestedType(); - - if (column_num_it == map.second.begin() - || endsWith(name, "ID") - || endsWith(name, "Key") - || endsWith(name, "Type")) - { - if (!nested_type.isValueRepresentedByInteger() && !isStringOrFixedString(nested_type)) - break; - - map_desc.key_col_nums.push_back(*column_num_it); - } - else - { - if (!nested_type.isSummable()) - break; - - map_desc.val_col_nums.push_back(*column_num_it); - } - - // Add column to function arguments - desc.column_numbers.push_back(*column_num_it); - argument_types.push_back(key_col.type); - } - - if (column_num_it != map.second.end()) - { - for (auto col : map.second) - column_numbers_not_to_aggregate.push_back(col); - continue; - } - - if (map_desc.key_col_nums.size() == 1) - { - // Create summation for all value columns in the map - desc.init("sumMapWithOverflow", argument_types); - columns_to_aggregate.emplace_back(std::move(desc)); - } - else - { - // Fall back to legacy mergeMaps for composite keys - for (auto col : map.second) - column_numbers_not_to_aggregate.push_back(col); - maps_to_sum.emplace_back(std::move(map_desc)); - } - } -} - - -void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns) -{ - /// We have nothing to aggregate. It means that it could be non-zero, because we have columns_not_to_aggregate. - if (columns_to_aggregate.empty()) - current_row_is_zero = false; - - for (auto & desc : columns_to_aggregate) - { - // Do not insert if the aggregation state hasn't been created - if (desc.created) - { - if (desc.is_agg_func_type) - { - current_row_is_zero = false; - } - else - { - try - { - desc.function->insertResultInto(desc.state.data(), *desc.merged_column); - - /// Update zero status of current row - if (desc.column_numbers.size() == 1) - { - // Flag row as non-empty if at least one column number if non-zero - current_row_is_zero = current_row_is_zero && desc.merged_column->isDefaultAt(desc.merged_column->size() - 1); - } - else - { - /// It is sumMapWithOverflow aggregate function. - /// Assume that the row isn't empty in this case (just because it is compatible with previous version) - current_row_is_zero = false; - } - } - catch (...) - { - desc.destroyState(); - throw; - } - } - desc.destroyState(); - } - else - desc.merged_column->insertDefault(); - } - - /// If it is "zero" row, then rollback the insertion - /// (at this moment we need rollback only cols from columns_to_aggregate) - if (current_row_is_zero) - { - for (auto & desc : columns_to_aggregate) - desc.merged_column->popBack(1); - - return; - } - - for (auto i : column_numbers_not_to_aggregate) - merged_columns[i]->insert(current_row[i]); - - /// Update per-block and per-group flags - ++merged_rows; -} - - -Block SummingSortedBlockInputStream::readImpl() -{ - if (finished) - return Block(); - - MutableColumns merged_columns; - init(merged_columns); - - if (has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); - - if (merged_columns.empty()) - return {}; - - /// Update aggregation result columns for current block - for (auto & desc : columns_to_aggregate) - { - // Wrap aggregated columns in a tuple to match function signature - if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) - { - size_t tuple_size = desc.column_numbers.size(); - MutableColumns tuple_columns(tuple_size); - for (size_t i = 0; i < tuple_size; ++i) - tuple_columns[i] = header.safeGetByPosition(desc.column_numbers[i]).column->cloneEmpty(); - - desc.merged_column = ColumnTuple::create(std::move(tuple_columns)); - } - else - desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty(); - } - - merge(merged_columns, queue_without_collation); - Block res = header.cloneWithColumns(std::move(merged_columns)); - - /// Place aggregation results into block. - for (auto & desc : columns_to_aggregate) - { - if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) - { - /// Unpack tuple into block. - size_t tuple_size = desc.column_numbers.size(); - for (size_t i = 0; i < tuple_size; ++i) - res.getByPosition(desc.column_numbers[i]).column = assert_cast(*desc.merged_column).getColumnPtr(i); - } - else - res.getByPosition(desc.column_numbers[0]).column = std::move(desc.merged_column); - } - - return res; -} - - -void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap & queue) -{ - merged_rows = 0; - - /// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size` - while (queue.isValid()) - { - SortCursor current = queue.current(); - - setPrimaryKeyRef(next_key, current); - - bool key_differs; - - if (current_key.empty()) /// The first key encountered. - { - key_differs = true; - current_row_is_zero = true; - } - else - key_differs = next_key != current_key; - - if (key_differs) - { - if (!current_key.empty()) - /// Write the data for the previous group. - insertCurrentRowIfNeeded(merged_columns); - - if (merged_rows >= max_block_size) - { - /// The block is now full and the last row is calculated completely. - current_key.reset(); - return; - } - - current_key.swap(next_key); - - setRow(current_row, current); - - /// Reset aggregation states for next row - for (auto & desc : columns_to_aggregate) - desc.createState(); - - // Start aggregations with current row - addRow(current); - - if (maps_to_sum.empty()) - { - /// We have only columns_to_aggregate. The status of current row will be determined - /// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions. - current_row_is_zero = true; // NOLINT - } - else - { - /// We have complex maps that will be summed with 'mergeMap' method. - /// The single row is considered non zero, and the status after merging with other rows - /// will be determined in the branch below (when key_differs == false). - current_row_is_zero = false; // NOLINT - } - } - else - { - addRow(current); - - // Merge maps only for same rows - for (const auto & desc : maps_to_sum) - if (mergeMap(desc, current_row, current)) - current_row_is_zero = false; - } - - if (!current->isLast()) - { - queue.next(); - } - else - { - /// We get the next block from the corresponding source, if there is one. - fetchNextBlock(current, queue); - } - } - - /// We will write the data for the last group, if it is non-zero. - /// If it is zero, and without it the output stream will be empty, we will write it anyway. - insertCurrentRowIfNeeded(merged_columns); - finished = true; -} - - -void SummingSortedBlockInputStream::addRow(SortCursor & cursor) -{ - for (auto & desc : columns_to_aggregate) - { - if (!desc.created) - throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR); - - if (desc.is_agg_func_type) - { - // desc.state is not used for AggregateFunction types - auto & col = cursor->all_columns[desc.column_numbers[0]]; - assert_cast(*desc.merged_column).insertMergeFrom(*col, cursor->pos); - } - else - { - // Specialized case for unary functions - if (desc.column_numbers.size() == 1) - { - auto & col = cursor->all_columns[desc.column_numbers[0]]; - desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr); - } - else - { - // Gather all source columns into a vector - ColumnRawPtrs columns(desc.column_numbers.size()); - for (size_t i = 0; i < desc.column_numbers.size(); ++i) - columns[i] = cursor->all_columns[desc.column_numbers[i]]; - - desc.add_function(desc.function.get(), desc.state.data(), columns.data(), cursor->pos, nullptr); - } - } - } -} - -} diff --git a/src/DataStreams/SummingSortedBlockInputStream.h b/src/DataStreams/SummingSortedBlockInputStream.h deleted file mode 100644 index 09bced85b1d..00000000000 --- a/src/DataStreams/SummingSortedBlockInputStream.h +++ /dev/null @@ -1,155 +0,0 @@ -#pragma once - -#include - -#include -#include -#include -#include -#include -#include - - -namespace Poco { class Logger; } - -namespace DB -{ - -namespace ErrorCodes -{ -} - - -/** Merges several sorted streams into one. - * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), - * collapses them into one row, summing all the numeric columns except the primary key. - * If in all numeric columns, except for the primary key, the result is zero, it deletes the row. - */ -class SummingSortedBlockInputStream : public MergingSortedBlockInputStream -{ -public: - SummingSortedBlockInputStream( - const BlockInputStreams & inputs_, - const SortDescription & description_, - /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. - const Names & column_names_to_sum_, - size_t max_block_size_); - - String getName() const override { return "SummingSorted"; } - - /// Stores numbers of key-columns and value-columns. - struct MapDescription - { - std::vector key_col_nums; - std::vector val_col_nums; - }; - -protected: - /// Can return 1 more records than max_block_size. - Block readImpl() override; - -private: - Poco::Logger * log; - - /// Read up to the end. - bool finished = false; - - /// Columns with which values should be summed. - ColumnNumbers column_numbers_not_to_aggregate; - - /** A table can have nested tables that are treated in a special way. - * If the name of the nested table ends in `Map` and it contains at least two columns, - * satisfying the following criteria: - * - the first column, as well as all columns whose names end with `ID`, `Key` or `Type` - numeric ((U)IntN, Date, DateTime); - * (a tuple of such columns will be called `keys`) - * - the remaining columns are arithmetic ((U)IntN, Float32/64), called (`values`...). - * This nested table is treated as a mapping (keys...) => (values...) and when merge - * its rows, the merge of the elements of two sets by (keys...) with summing of corresponding (values...). - * - * Example: - * [(1, 100)] + [(2, 150)] -> [(1, 100), (2, 150)] - * [(1, 100)] + [(1, 150)] -> [(1, 250)] - * [(1, 100)] + [(1, 150), (2, 150)] -> [(1, 250), (2, 150)] - * [(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)] - * - * This very unusual functionality is made exclusively for the banner system, - * is not supposed for use by anyone else, - * and can be deleted at any time. - */ - - /// Stores aggregation function, state, and columns to be used as function arguments - struct AggregateDescription - { - /// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing. - AggregateFunctionPtr function; - IAggregateFunction::AddFunc add_function = nullptr; - std::vector column_numbers; - MutableColumnPtr merged_column; - AlignedBuffer state; - bool created = false; - - /// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above. - bool is_agg_func_type = false; - - void init(const char * function_name, const DataTypes & argument_types) - { - function = AggregateFunctionFactory::instance().get(function_name, argument_types); - add_function = function->getAddressOfAddFunction(); - state.reset(function->sizeOfData(), function->alignOfData()); - } - - void createState() - { - if (created) - return; - if (is_agg_func_type) - merged_column->insertDefault(); - else - function->create(state.data()); - created = true; - } - - void destroyState() - { - if (!created) - return; - if (!is_agg_func_type) - function->destroy(state.data()); - created = false; - } - - /// Explicitly destroy aggregation state if the stream is terminated - ~AggregateDescription() - { - destroyState(); - } - - AggregateDescription() = default; - AggregateDescription(AggregateDescription &&) = default; - AggregateDescription(const AggregateDescription &) = delete; - }; - - std::vector columns_to_aggregate; - std::vector maps_to_sum; - - SharedBlockRowRef current_key; /// The current primary key. - SharedBlockRowRef next_key; /// The primary key of the next row. - - Row current_row; - bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. - - size_t merged_rows = 0; /// Number of rows merged into current result block - - /** We support two different cursors - with Collation and without. - * Templates are used instead of polymorphic SortCursor and calls to virtual functions. - */ - void merge(MutableColumns & merged_columns, SortingHeap & queue); - - /// Insert the summed row for the current group into the result and updates some of per-block flags if the row is not "zero". - void insertCurrentRowIfNeeded(MutableColumns & merged_columns); - - // Add the row under the cursor to the `row`. - void addRow(SortCursor & cursor); -}; - -} diff --git a/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp b/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp deleted file mode 100644 index 30fa0460ac5..00000000000 --- a/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp +++ /dev/null @@ -1,181 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - - -VersionedCollapsingSortedBlockInputStream::VersionedCollapsingSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, - const String & sign_column_, size_t max_block_size_, - WriteBuffer * out_row_sources_buf_, bool average_block_sizes_) - : MergingSortedBlockInputStream(inputs_, description_, max_block_size_, 0, out_row_sources_buf_, false, average_block_sizes_) - , max_rows_in_queue(std::min(std::max(3, max_block_size_), MAX_ROWS_IN_MULTIVERSION_QUEUE) - 2) - , current_keys(max_rows_in_queue + 1) -{ - sign_column_number = header.getPositionByName(sign_column_); -} - - -inline ALWAYS_INLINE static void writeRowSourcePart(WriteBuffer & buffer, RowSourcePart row_source) -{ - if constexpr (sizeof(RowSourcePart) == 1) - buffer.write(*reinterpret_cast(&row_source)); - else - buffer.write(reinterpret_cast(&row_source), sizeof(RowSourcePart)); -} - -void VersionedCollapsingSortedBlockInputStream::insertGap(size_t gap_size) -{ - if (out_row_sources_buf) - { - for (size_t i = 0; i < gap_size; ++i) - { - writeRowSourcePart(*out_row_sources_buf, current_row_sources.front()); - current_row_sources.pop(); - } - } -} - -void VersionedCollapsingSortedBlockInputStream::insertRow(size_t skip_rows, const SharedBlockRowRef & row, MutableColumns & merged_columns) -{ - const auto & columns = row.shared_block->all_columns; - for (size_t i = 0; i < num_columns; ++i) - merged_columns[i]->insertFrom(*columns[i], row.row_num); - - insertGap(skip_rows); - - if (out_row_sources_buf) - { - current_row_sources.front().setSkipFlag(false); - writeRowSourcePart(*out_row_sources_buf, current_row_sources.front()); - current_row_sources.pop(); - } -} - -Block VersionedCollapsingSortedBlockInputStream::readImpl() -{ - if (finished) - return {}; - - MutableColumns merged_columns; - init(merged_columns); - - if (has_collation) - throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::NOT_IMPLEMENTED); - - if (merged_columns.empty()) - return {}; - - merge(merged_columns, queue_without_collation); - return header.cloneWithColumns(std::move(merged_columns)); -} - - -void VersionedCollapsingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap & queue) -{ - MergeStopCondition stop_condition(average_block_sizes, max_block_size); - - auto update_queue = [this, & queue](SortCursor & cursor) - { - if (out_row_sources_buf) - current_row_sources.emplace(cursor->order, true); - - if (!cursor->isLast()) - { - queue.next(); - } - else - { - /// We take next block from the corresponding source, if there is one. - fetchNextBlock(cursor, queue); - } - }; - - /// Take rows in correct order and put them into `merged_columns` until the rows no more than `max_block_size` - while (queue.isValid()) - { - SortCursor current = queue.current(); - size_t current_block_granularity = current->rows; - - SharedBlockRowRef next_key; - - Int8 sign = assert_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; - - setPrimaryKeyRef(next_key, current); - - size_t rows_to_merge = 0; - - /// Each branch either updates queue or increases rows_to_merge. - if (current_keys.empty()) - { - sign_in_queue = sign; - current_keys.pushBack(next_key); - update_queue(current); - } - else - { - if (current_keys.back() == next_key) - { - update_queue(current); - - if (sign == sign_in_queue) - current_keys.pushBack(next_key); - else - { - current_keys.popBack(); - current_keys.pushGap(2); - } - } - else - rows_to_merge = current_keys.size(); - } - - if (current_keys.size() > max_rows_in_queue) - rows_to_merge = std::max(rows_to_merge, current_keys.size() - max_rows_in_queue); - - while (rows_to_merge) - { - const auto & row = current_keys.front(); - auto gap = current_keys.frontGap(); - - insertRow(gap, row, merged_columns); - - current_keys.popFront(); - - stop_condition.addRowWithGranularity(current_block_granularity); - --rows_to_merge; - - if (stop_condition.checkStop()) - { - ++blocks_written; - return; - } - } - } - - while (!current_keys.empty()) - { - const auto & row = current_keys.front(); - auto gap = current_keys.frontGap(); - - insertRow(gap, row, merged_columns); - - current_keys.popFront(); - } - - /// Write information about last collapsed rows. - insertGap(current_keys.frontGap()); - - finished = true; -} - -} diff --git a/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h b/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h deleted file mode 100644 index c64972d9266..00000000000 --- a/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h +++ /dev/null @@ -1,215 +0,0 @@ -#pragma once - -#include - -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -static const size_t MAX_ROWS_IN_MULTIVERSION_QUEUE = 8192; - - -/* Deque with fixed memory size. Allows pushing gaps. - * frontGap() returns the number of gaps were inserted before front. - * - * This structure may be implemented via std::deque, but - * - Deque uses fixed amount of memory which is allocated in constructor. No more allocations are performed. - * - Gaps are not stored as separate values in queue, which is more memory efficient. - * - Deque is responsible for gaps invariant: after removing element, moves gaps into neighbor cell. - * - * Note: empty deque may have non-zero front gap. - */ -template -class FixedSizeDequeWithGaps -{ -public: - - struct ValueWithGap - { - /// The number of gaps before current element. The number of gaps after last element stores into end cell. - size_t gap; - /// Store char[] instead of T in order to make ValueWithGap POD. - /// Call placement constructors after push and and destructors after pop. - char value[sizeof(T)]; - }; - - explicit FixedSizeDequeWithGaps(size_t size) - { - container.resize_fill(size + 1); - } - - ~FixedSizeDequeWithGaps() - { - auto destruct_range = [this](size_t from, size_t to) - { - for (size_t i = from; i < to; ++i) - destructValue(i); - }; - - if (begin <= end) - destruct_range(begin, end); - else - { - destruct_range(0, end); - destruct_range(begin, container.size()); - } - } - - void pushBack(const T & value) - { - checkEnoughSpaceToInsert(); - constructValue(end, value); - moveRight(end); - container[end].gap = 0; - } - - void pushGap(size_t count) { container[end].gap += count; } - - void popBack() - { - checkHasValuesToRemove(); - size_t curr_gap = container[end].gap; - moveLeft(end); - destructValue(end); - container[end].gap += curr_gap; - } - - void popFront() - { - checkHasValuesToRemove(); - destructValue(begin); - moveRight(begin); - } - - T & front() - { - checkHasValuesToGet(); - return getValue(begin); - } - const T & front() const - { - checkHasValuesToGet(); - return getValue(begin); - } - - const T & back() const - { - size_t ps = end; - moveLeft(ps); - return getValue(ps); - } - - size_t & frontGap() { return container[begin].gap; } - const size_t & frontGap() const { return container[begin].gap; } - - size_t size() const - { - if (begin <= end) - return end - begin; - return end + (container.size() - begin); - } - - bool empty() const { return begin == end; } - -private: - PODArray container; - - size_t gap_before_first = 0; - size_t begin = 0; - size_t end = 0; - - void constructValue(size_t index, const T & value) { new (container[index].value) T(value); } - void destructValue(size_t index) { reinterpret_cast(container[index].value)->~T(); } - - T & getValue(size_t index) { return *reinterpret_cast(container[index].value); } - const T & getValue(size_t index) const { return *reinterpret_cast(container[index].value); } - - void moveRight(size_t & index) const - { - ++index; - - if (index == container.size()) - index = 0; - } - - void moveLeft(size_t & index) const - { - if (index == 0) - index = container.size(); - - --index; - } - - void checkEnoughSpaceToInsert() const - { - if (size() + 1 == container.size()) - throw Exception("Not enough space to insert into FixedSizeDequeWithGaps with capacity " - + toString(container.size() - 1), ErrorCodes::LOGICAL_ERROR); - } - - void checkHasValuesToRemove() const - { - if (empty()) - throw Exception("Cannot remove from empty FixedSizeDequeWithGaps", ErrorCodes::LOGICAL_ERROR); - } - - void checkHasValuesToGet() const - { - if (empty()) - throw Exception("Cannot get value from empty FixedSizeDequeWithGaps", ErrorCodes::LOGICAL_ERROR); - } -}; - -class VersionedCollapsingSortedBlockInputStream : public MergingSortedBlockInputStream -{ -public: - /// Don't need version column. It's in primary key. - /// max_rows_in_queue should be about max_block_size_ if we won't store a lot of extra blocks (RowRef holds SharedBlockPtr). - VersionedCollapsingSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, - const String & sign_column_, size_t max_block_size_, - WriteBuffer * out_row_sources_buf_ = nullptr, bool average_block_sizes_ = false); - - String getName() const override { return "VersionedCollapsingSorted"; } - -protected: - /// Can return 1 more records than max_block_size. - Block readImpl() override; - -private: - size_t sign_column_number = 0; - - Logger * log = &Logger::get("VersionedCollapsingSortedBlockInputStream"); - - /// Read is finished. - bool finished = false; - - Int8 sign_in_queue = 0; - const size_t max_rows_in_queue; - /// Rows with the same primary key and sign. - FixedSizeDequeWithGaps current_keys; - - size_t blocks_written = 0; - - /// Sources of rows for VERTICAL merge algorithm. Size equals to (size + number of gaps) in current_keys. - std::queue current_row_sources; - - void merge(MutableColumns & merged_columns, SortingHeap & queue); - - /// Output to result row for the current primary key. - void insertRow(size_t skip_rows, const SharedBlockRowRef & row, MutableColumns & merged_columns); - - void insertGap(size_t gap_size); -}; - -} From c0a8def386317346a8d88b037c0b93fb7de19b66 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 13:35:17 +0300 Subject: [PATCH 227/743] Fix build. --- src/DataStreams/tests/collapsing_sorted_stream.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataStreams/tests/collapsing_sorted_stream.cpp b/src/DataStreams/tests/collapsing_sorted_stream.cpp index a34d6a2fbd2..346c05b48ea 100644 --- a/src/DataStreams/tests/collapsing_sorted_stream.cpp +++ b/src/DataStreams/tests/collapsing_sorted_stream.cpp @@ -7,7 +7,6 @@ #include #include -#include #include #include #include From 1de7191aa3df947adcdf064b1b9e4e9faa948035 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 13:49:46 +0300 Subject: [PATCH 228/743] U[date test --- .../gtest_blocks_size_merging_streams.cpp | 55 +++++++++++-------- 1 file changed, 32 insertions(+), 23 deletions(-) diff --git a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp index ff89cebd156..513a3ef16dc 100644 --- a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp @@ -1,10 +1,13 @@ #include #include #include -#include #include #include #include +#include +#include +#include +#include using namespace DB; @@ -27,25 +30,25 @@ static Block getBlockWithSize(const std::vector & columns, size_t r } -static BlockInputStreams getInputStreams(const std::vector & column_names, const std::vector> & block_sizes) +static Pipes getInputStreams(const std::vector & column_names, const std::vector> & block_sizes) { - BlockInputStreams result; + Pipes pipes; for (auto [block_size_in_bytes, blocks_count, stride] : block_sizes) { BlocksList blocks; size_t start = stride; while (blocks_count--) blocks.push_back(getBlockWithSize(column_names, block_size_in_bytes, stride, start)); - result.push_back(std::make_shared(std::move(blocks))); + pipes.emplace_back(std::make_shared(std::make_shared(std::move(blocks)))); } - return result; + return pipes; } -static BlockInputStreams getInputStreamsEqualStride(const std::vector & column_names, const std::vector> & block_sizes) +static Pipes getInputStreamsEqualStride(const std::vector & column_names, const std::vector> & block_sizes) { - BlockInputStreams result; + Pipes pipes; size_t i = 0; for (auto [block_size_in_bytes, blocks_count, stride] : block_sizes) { @@ -53,10 +56,10 @@ static BlockInputStreams getInputStreamsEqualStride(const std::vector(std::move(blocks))); + pipes.emplace_back(std::make_shared(std::make_shared(std::move(blocks)))); i++; } - return result; + return pipes; } @@ -75,18 +78,21 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) { std::vector key_columns{"K1", "K2", "K3"}; auto sort_description = getSortDescription(key_columns); - auto streams = getInputStreams(key_columns, {{5, 1, 1}, {10, 1, 2}, {21, 1, 3}}); + auto pipes = getInputStreams(key_columns, {{5, 1, 1}, {10, 1, 2}, {21, 1, 3}}); - EXPECT_EQ(streams.size(), 3); + EXPECT_EQ(pipes.size(), 3); - MergingSortedBlockInputStream stream(streams, sort_description, DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true); + auto transform = std::make_shared(pipes.front().getHeader(), pipes.size(), sort_description, + DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true); + + auto stream = std::make_shared(Pipe(std::move(pipes), std::move(transform))); size_t total_rows = 0; - auto block1 = stream.read(); - auto block2 = stream.read(); - auto block3 = stream.read(); + auto block1 = stream->read(); + auto block2 = stream->read(); + auto block3 = stream->read(); - EXPECT_EQ(stream.read(), Block()); + EXPECT_EQ(stream->read(), Block()); for (auto & block : {block1, block2, block3}) total_rows += block.rows(); @@ -112,17 +118,20 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) { std::vector key_columns{"K1", "K2", "K3"}; auto sort_description = getSortDescription(key_columns); - auto streams = getInputStreamsEqualStride(key_columns, {{1000, 1, 3}, {1500, 1, 3}, {1400, 1, 3}}); + auto pipes = getInputStreamsEqualStride(key_columns, {{1000, 1, 3}, {1500, 1, 3}, {1400, 1, 3}}); - EXPECT_EQ(streams.size(), 3); + EXPECT_EQ(pipes.size(), 3); - MergingSortedBlockInputStream stream(streams, sort_description, DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true); + auto transform = std::make_shared(pipes.front().getHeader(), pipes.size(), sort_description, + DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true); - auto block1 = stream.read(); - auto block2 = stream.read(); - auto block3 = stream.read(); + auto stream = std::make_shared(Pipe(std::move(pipes), std::move(transform))); - EXPECT_EQ(stream.read(), Block()); + auto block1 = stream->read(); + auto block2 = stream->read(); + auto block3 = stream->read(); + + EXPECT_EQ(stream->read(), Block()); EXPECT_EQ(block1.rows(), (1000 + 1500 + 1400) / 3); EXPECT_EQ(block2.rows(), (1000 + 1500 + 1400) / 3); From 2da9f889cda94f3745394e3ff9c66bded779bdc4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 14:09:23 +0300 Subject: [PATCH 229/743] Fix unit test. --- src/Processors/Merges/Algorithms/MergedData.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Merges/Algorithms/MergedData.h b/src/Processors/Merges/Algorithms/MergedData.h index 37dd3c62587..c96cc9ad6f4 100644 --- a/src/Processors/Merges/Algorithms/MergedData.h +++ b/src/Processors/Merges/Algorithms/MergedData.h @@ -93,7 +93,8 @@ public: if (merged_rows == 0) return false; - return merged_rows * merged_rows >= sum_blocks_granularity; + size_t average = sum_blocks_granularity / merged_rows; + return merged_rows >= average; } UInt64 mergedRows() const { return merged_rows; } From 440e4ecfca309a232bcb4cb4e41c91bb4c6bdbc5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 14 Apr 2020 14:15:35 +0300 Subject: [PATCH 230/743] [draft] changelog for 20.3.6 --- CHANGELOG.md | 39 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2ab006bcdd3..66e850d9d9e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,44 @@ ## ClickHouse release v20.3 +### ClickHouse release origin/20.3 FIXME as compared to v20.3.5.21-stable + +#### Bug Fix + +* Convert types if the table definition of a View does not correspond to the SELECT query. This fixes [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022). [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible inifinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix using the current database for access checking when the database isn't specified. [#10192](https://github.com/ClickHouse/ClickHouse/pull/10192) ([Vitaly Baranov](https://github.com/vitlibar)). +* Convert blocks if structure does not match on INSERT into Distributed(). [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible incorrect result for extremes in processors pipeline. [#10131](https://github.com/ClickHouse/ClickHouse/pull/10131) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix some kinds of alters with compact parts. [#10130](https://github.com/ClickHouse/ClickHouse/pull/10130) ([Anton Popov](https://github.com/CurtizJ)). +* Fix incorrect `index_granularity_bytes` check while creating new replica. Fixes [#10098](https://github.com/ClickHouse/ClickHouse/issues/10098). [#10121](https://github.com/ClickHouse/ClickHouse/pull/10121) ([alesapin](https://github.com/alesapin)). +* Fix SIGSEGV on INSERT into Distributed table when its structure differs from the underlying tables. [#10105](https://github.com/ClickHouse/ClickHouse/pull/10105) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed replicated tables startup when updating from an old ClickHouse version where `/table/replicas/replica_name/metadata` node doesn't exist. Fixes [#10037](https://github.com/ClickHouse/ClickHouse/issues/10037). [#10095](https://github.com/ClickHouse/ClickHouse/pull/10095) ([alesapin](https://github.com/alesapin)). +* Add some arguments check and support identifier arguments for MySQL Database Engine. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)). +* Fix bug in clickhouse dictionary source from localhost clickhouse server. The bug may lead to memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)). +* Fix bug in `CHECK TABLE` query when table contain skip indices. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)). +* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with different level from different shards (mixed single and two level aggregation). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a segmentation fault that could occur in GROUP BY over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix parallel distributed INSERT SELECT for remote table. This PR fixes the solution provided in [#9759](https://github.com/ClickHouse/ClickHouse/pull/9759). [#9999](https://github.com/ClickHouse/ClickHouse/pull/9999) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965). [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). +* Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix a race between DDLWorker and DDLWorkerClnr during startup that can cause the DDLWorker to exit and all future DDL queries to hang. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). +* Fix parsing multiple hosts set in the CREATE USER command, e.g. `CREATE USER user6 HOST NAME REGEXP 'lo.?*host', NAME REGEXP 'lo*host'`. [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). +* Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). +* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). +* Fix error with qualified names in `distributed_product_mode='local'`. Fixes [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756). [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix calculating grants for introspection functions from the setting 'allow_introspection_functions'. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Build/Testing/Packaging Improvement + +* Fix integration test `test_settings_constraints`. [#9962](https://github.com/ClickHouse/ClickHouse/pull/9962) ([Vitaly Baranov](https://github.com/vitlibar)). +* Removed dependency on `clock_getres`. [#9833](https://github.com/ClickHouse/ClickHouse/pull/9833) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + ### ClickHouse release v20.3.5.21, 2020-03-27 #### Bug Fix From e92827b95428b1f3ab95f02b8b9c1249facb3803 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 14:25:01 +0300 Subject: [PATCH 231/743] Remove StopCondition from MergingSortedBlockInputStream. --- .../MergingSortedBlockInputStream.cpp | 24 +++---------- .../MergingSortedBlockInputStream.h | 36 +------------------ 2 files changed, 5 insertions(+), 55 deletions(-) diff --git a/src/DataStreams/MergingSortedBlockInputStream.cpp b/src/DataStreams/MergingSortedBlockInputStream.cpp index 10822485d52..47a9bbeb7e0 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -19,9 +19,9 @@ namespace ErrorCodes MergingSortedBlockInputStream::MergingSortedBlockInputStream( const BlockInputStreams & inputs_, const SortDescription & description_, - size_t max_block_size_, UInt64 limit_, WriteBuffer * out_row_sources_buf_, bool quiet_, bool average_block_sizes_) + size_t max_block_size_, UInt64 limit_, WriteBuffer * out_row_sources_buf_, bool quiet_) : description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) - , average_block_sizes(average_block_sizes_), source_blocks(inputs_.size()) + , source_blocks(inputs_.size()) , cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_) , log(&Logger::get("MergingSortedBlockInputStream")) { @@ -139,30 +139,15 @@ void MergingSortedBlockInputStream::fetchNextBlock(const TSortCursor & current, } -bool MergingSortedBlockInputStream::MergeStopCondition::checkStop() const -{ - if (!count_average) - return sum_rows_count == max_block_size; - - if (sum_rows_count == 0) - return false; - - size_t average = sum_blocks_granularity / sum_rows_count; - return sum_rows_count >= average; -} - - template void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSortingHeap & queue) { size_t merged_rows = 0; - MergeStopCondition stop_condition(average_block_sizes, max_block_size); - /** Increase row counters. * Return true if it's time to finish generating the current data block. */ - auto count_row_and_check_limit = [&, this](size_t current_granularity) + auto count_row_and_check_limit = [&, this]() { ++total_merged_rows; if (limit && total_merged_rows == limit) @@ -174,8 +159,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort } ++merged_rows; - stop_condition.addRowWithGranularity(current_granularity); - return stop_condition.checkStop(); + return merged_rows >= max_block_size; }; /// Take rows in required order and put them into `merged_columns`, while the number of rows are no more than `max_block_size` diff --git a/src/DataStreams/MergingSortedBlockInputStream.h b/src/DataStreams/MergingSortedBlockInputStream.h index 726d67d432a..bbf6833ef26 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.h +++ b/src/DataStreams/MergingSortedBlockInputStream.h @@ -34,7 +34,7 @@ public: */ MergingSortedBlockInputStream( const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_, - UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false, bool average_block_sizes_ = false); + UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false); String getName() const override { return "MergingSorted"; } @@ -44,38 +44,6 @@ public: Block getHeader() const override { return header; } protected: - /// Simple class, which allows to check stop condition during merge process - /// in simple case it just compare amount of merged rows with max_block_size - /// in `count_average` case it compares amount of merged rows with linear combination - /// of block sizes from which these rows were taken. - struct MergeStopCondition - { - size_t sum_blocks_granularity = 0; - size_t sum_rows_count = 0; - bool count_average; - size_t max_block_size; - - MergeStopCondition(bool count_average_, size_t max_block_size_) - : count_average(count_average_) - , max_block_size(max_block_size_) - {} - - /// add single row from block size `granularity` - void addRowWithGranularity(size_t granularity) - { - sum_blocks_granularity += granularity; - sum_rows_count++; - } - - /// check that sum_rows_count is enough - bool checkStop() const; - - bool empty() const - { - return sum_blocks_granularity == 0; - } - }; - Block readImpl() override; void readSuffixImpl() override; @@ -87,7 +55,6 @@ protected: template void fetchNextBlock(const TSortCursor & current, SortingHeap & queue); - Block header; const SortDescription description; @@ -98,7 +65,6 @@ protected: bool first = true; bool has_collation = false; bool quiet = false; - bool average_block_sizes = false; /// May be smaller or equal to max_block_size. To do 'reserve' for columns. size_t expected_block_size = 0; From ada3d4e67c799e41bbdd9be8f83392b2c7618d45 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 14:26:03 +0300 Subject: [PATCH 232/743] Remove StopCondition from MergingSortedBlockInputStream. --- src/DataStreams/MergingSortedBlockInputStream.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/DataStreams/MergingSortedBlockInputStream.cpp b/src/DataStreams/MergingSortedBlockInputStream.cpp index 47a9bbeb7e0..47610d3ab1b 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -166,7 +166,6 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort while (queue.isValid()) { auto current = queue.current(); - size_t current_block_granularity = current->rows; /** And what if the block is totally less or equal than the rest for the current cursor? * Or is there only one data source left in the queue? Then you can take the entire block on current cursor. @@ -251,7 +250,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort fetchNextBlock(current, queue); } - if (count_row_and_check_limit(current_block_granularity)) + if (count_row_and_check_limit()) return; } From 3617b8eb7b37a00f327a8e5aab00d24a67eff87a Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 14 Apr 2020 14:27:34 +0300 Subject: [PATCH 233/743] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 66e850d9d9e..5331fa0ec98 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -25,7 +25,7 @@ * Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965). [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). * Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix a race between DDLWorker and DDLWorkerClnr during startup that can cause the DDLWorker to exit and all future DDL queries to hang. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). +* Fix a bug with ON CLUSTER DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). * Fix parsing multiple hosts set in the CREATE USER command, e.g. `CREATE USER user6 HOST NAME REGEXP 'lo.?*host', NAME REGEXP 'lo*host'`. [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). * Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). From 40a0f53a422bd5e1277f9986954a7c8de8e23cf2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 14:32:38 +0300 Subject: [PATCH 234/743] Remove SharedBlockPtr from MergingSortedBlockInputStream. --- .../MergingSortedBlockInputStream.cpp | 35 ++++++++----------- .../MergingSortedBlockInputStream.h | 6 ++-- 2 files changed, 17 insertions(+), 24 deletions(-) diff --git a/src/DataStreams/MergingSortedBlockInputStream.cpp b/src/DataStreams/MergingSortedBlockInputStream.cpp index 47610d3ab1b..1ed7c6dff22 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -18,9 +18,9 @@ namespace ErrorCodes MergingSortedBlockInputStream::MergingSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, + const BlockInputStreams & inputs_, SortDescription description_, size_t max_block_size_, UInt64 limit_, WriteBuffer * out_row_sources_buf_, bool quiet_) - : description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) + : description(std::move(description_)), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) , source_blocks(inputs_.size()) , cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_) , log(&Logger::get("MergingSortedBlockInputStream")) @@ -39,14 +39,14 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns) for (size_t i = 0; i < source_blocks.size(); ++i) { - SharedBlockPtr & shared_block_ptr = source_blocks[i]; + Block & block = source_blocks[i]; - if (shared_block_ptr.get()) + if (block) continue; - shared_block_ptr = new detail::SharedBlock(children[i]->read()); + block = children[i]->read(); - const size_t rows = shared_block_ptr->rows(); + const size_t rows = block.rows(); if (rows == 0) continue; @@ -54,9 +54,7 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns) if (expected_block_size < rows) expected_block_size = std::min(rows, max_block_size); - cursors[i] = SortCursorImpl(*shared_block_ptr, description, i); - shared_block_ptr->all_columns = cursors[i].all_columns; - shared_block_ptr->sort_columns = cursors[i].sort_columns; + cursors[i] = SortCursorImpl(block, description, i); has_collation |= cursors[i].has_collation; } @@ -67,12 +65,12 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns) } /// Let's check that all source blocks have the same structure. - for (const SharedBlockPtr & shared_block_ptr : source_blocks) + for (const auto & block : source_blocks) { - if (!*shared_block_ptr) + if (!block) continue; - assertBlocksHaveEqualStructure(*shared_block_ptr, header, getName()); + assertBlocksHaveEqualStructure(block, header, getName()); } merged_columns.resize(num_columns); @@ -118,21 +116,18 @@ void MergingSortedBlockInputStream::fetchNextBlock(const TSortCursor & current, while (true) { - source_blocks[order] = new detail::SharedBlock(children[order]->read()); /// intrusive ptr + source_blocks[order] = children[order]->read(); - if (!*source_blocks[order]) + if (!source_blocks[order]) { queue.removeTop(); break; } - if (source_blocks[order]->rows()) + if (source_blocks[order].rows()) { - cursors[order].reset(*source_blocks[order]); + cursors[order].reset(source_blocks[order]); queue.replaceTop(&cursors[order]); - - source_blocks[order]->all_columns = cursors[order].all_columns; - source_blocks[order]->sort_columns = cursors[order].sort_columns; break; } } @@ -190,7 +185,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort throw Exception("Logical error in MergingSortedBlockInputStream", ErrorCodes::LOGICAL_ERROR); for (size_t i = 0; i < num_columns; ++i) - merged_columns[i] = (*std::move(source_blocks[source_num]->getByPosition(i).column)).mutate(); + merged_columns[i] = (*std::move(source_blocks[source_num].getByPosition(i).column)).mutate(); // std::cerr << "copied columns\n"; diff --git a/src/DataStreams/MergingSortedBlockInputStream.h b/src/DataStreams/MergingSortedBlockInputStream.h index bbf6833ef26..468ac2dcfdc 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.h +++ b/src/DataStreams/MergingSortedBlockInputStream.h @@ -1,7 +1,5 @@ #pragma once -#include - #include #include #include @@ -33,7 +31,7 @@ public: * quiet - don't log profiling info */ MergingSortedBlockInputStream( - const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_, + const BlockInputStreams & inputs_, SortDescription description_, size_t max_block_size_, UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false); String getName() const override { return "MergingSorted"; } @@ -71,7 +69,7 @@ protected: /// Blocks currently being merged. size_t num_columns = 0; - std::vector source_blocks; + Blocks source_blocks; SortCursorImpls cursors; From 0f0d9b22d8c51a8189f20ce34e8686cd94d2e1ed Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 14 Apr 2020 14:51:43 +0300 Subject: [PATCH 235/743] trigger From 37a971fbe58046827273ccc8edbf5b104a4a8c30 Mon Sep 17 00:00:00 2001 From: Sergei Shtykov Date: Tue, 14 Apr 2020 15:56:34 +0300 Subject: [PATCH 236/743] CLICKHOUSEDOCS-580: Fixed select.md and some of the broken links. --- .../data_types/special_data_types/set.md | 2 +- docs/en/sql_reference/statements/select.md | 778 +++++++++++++++++- 2 files changed, 775 insertions(+), 5 deletions(-) diff --git a/docs/en/sql_reference/data_types/special_data_types/set.md b/docs/en/sql_reference/data_types/special_data_types/set.md index 61f3c302ce4..05ca97910b6 100644 --- a/docs/en/sql_reference/data_types/special_data_types/set.md +++ b/docs/en/sql_reference/data_types/special_data_types/set.md @@ -5,6 +5,6 @@ toc_title: Set # Set {#set} -Used for the right half of an [IN](../../../sql_reference/statements/select.md#select-in-operators) expression. +Used for the right half of an [IN](../../statements/select.md#select-in-operators) expression. [Original article](https://clickhouse.tech/docs/en/data_types/special_data_types/set/) diff --git a/docs/en/sql_reference/statements/select.md b/docs/en/sql_reference/statements/select.md index 83ead5b26e1..9232011a8e2 100644 --- a/docs/en/sql_reference/statements/select.md +++ b/docs/en/sql_reference/statements/select.md @@ -248,7 +248,7 @@ Here, a sample of 10% is taken from the second half of the data. ### ARRAY JOIN Clause {#select-array-join-clause} -Allows executing `JOIN` with an array or nested data structure. The intent is similar to the [arrayJoin](../../sql_reference/functions/array_join.md#functions_arrayjoin) function, but its functionality is broader. +Allows executing `JOIN` with an array or nested data structure. The intent is similar to the [arrayJoin](../functions/array_join.md#functions_arrayjoin) function, but its functionality is broader. ``` sql SELECT @@ -602,7 +602,777 @@ USING (equi_column1, ... equi_columnN, asof_column) For example, consider the following tables: -\`\`\` text -table\_1 table\_2 + table_1 table_2 + event | ev_time | user_id event | ev_time | user_id + ----------|---------|---------- ----------|---------|---------- + ... ... + event_1_1 | 12:00 | 42 event_2_1 | 11:59 | 42 + ... event_2_2 | 12:30 | 42 + event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 + ... ... -event \| ev\_time \| user\_id event \| ev\_time \| user\_id + +`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can’t be joined. + +!!! note "Note" + `ASOF` join is **not** supported in the [Join](../../engines/table_engines/special/join.md) table engine. + +To set the default strictness value, use the session configuration parameter [join\_default\_strictness](../../operations/settings/settings.md#settings-join_default_strictness). + +#### GLOBAL JOIN {#global-join} + +When using a normal `JOIN`, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately. + +When using `GLOBAL ... JOIN`, first the requestor server runs a subquery to calculate the right table. This temporary table is passed to each remote server, and queries are run on them using the temporary data that was transmitted. + +Be careful when using `GLOBAL`. For more information, see the section [Distributed subqueries](#select-distributed-subqueries). + +#### Usage Recommendations {#usage-recommendations} + +When running a `JOIN`, there is no optimization of the order of execution in relation to other stages of the query. The join (a search in the right table) is run before filtering in `WHERE` and before aggregation. In order to explicitly set the processing order, we recommend running a `JOIN` subquery with a subquery. + +Example: + +``` sql +SELECT + CounterID, + hits, + visits +FROM +( + SELECT + CounterID, + count() AS hits + FROM test.hits + GROUP BY CounterID +) ANY LEFT JOIN +( + SELECT + CounterID, + sum(Sign) AS visits + FROM test.visits + GROUP BY CounterID +) USING CounterID +ORDER BY hits DESC +LIMIT 10 +``` + +``` text +┌─CounterID─┬───hits─┬─visits─┐ +│ 1143050 │ 523264 │ 13665 │ +│ 731962 │ 475698 │ 102716 │ +│ 722545 │ 337212 │ 108187 │ +│ 722889 │ 252197 │ 10547 │ +│ 2237260 │ 196036 │ 9522 │ +│ 23057320 │ 147211 │ 7689 │ +│ 722818 │ 90109 │ 17847 │ +│ 48221 │ 85379 │ 4652 │ +│ 19762435 │ 77807 │ 7026 │ +│ 722884 │ 77492 │ 11056 │ +└───────────┴────────┴────────┘ +``` + +Subqueries don’t allow you to set names or use them for referencing a column from a specific subquery. +The columns specified in `USING` must have the same names in both subqueries, and the other columns must be named differently. You can use aliases to change the names of columns in subqueries (the example uses the aliases `hits` and `visits`). + +The `USING` clause specifies one or more columns to join, which establishes the equality of these columns. The list of columns is set without brackets. More complex join conditions are not supported. + +The right table (the subquery result) resides in RAM. If there isn’t enough memory, you can’t run a `JOIN`. + +Each time a query is run with the same `JOIN`, the subquery is run again because the result is not cached. To avoid this, use the special [Join](../../engines/table_engines/special/join.md) table engine, which is a prepared array for joining that is always in RAM. + +In some cases, it is more efficient to use `IN` instead of `JOIN`. +Among the various types of `JOIN`, the most efficient is `ANY LEFT JOIN`, then `ANY INNER JOIN`. The least efficient are `ALL LEFT JOIN` and `ALL INNER JOIN`. + +If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is an “external dictionaries” feature that you should use instead of `JOIN`. For more information, see the section [External dictionaries](../dictionaries/external_dictionaries/external_dicts.md). + +**Memory Limitations** + +ClickHouse uses the [hash join](https://en.wikipedia.org/wiki/Hash_join) algorithm. ClickHouse takes the `` and creates a hash table for it in RAM. If you need to restrict join operation memory consumption use the following settings: + +- [max\_rows\_in\_join](../../operations/settings/query_complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. +- [max\_bytes\_in\_join](../../operations/settings/query_complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. + +When any of these limits is reached, ClickHouse acts as the [join\_overflow\_mode](../../operations/settings/query_complexity.md#settings-join_overflow_mode) setting instructs. + +#### Processing of Empty or NULL Cells {#processing-of-empty-or-null-cells} + +While joining tables, the empty cells may appear. The setting [join\_use\_nulls](../../operations/settings/settings.md#join_use_nulls) define how ClickHouse fills these cells. + +If the `JOIN` keys are [Nullable](../data_types/nullable.md) fields, the rows where at least one of the keys has the value [NULL](../syntax.md#null-literal) are not joined. + +#### Syntax Limitations {#syntax-limitations} + +For multiple `JOIN` clauses in a single `SELECT` query: + +- Taking all the columns via `*` is available only if tables are joined, not subqueries. +- The `PREWHERE` clause is not available. + +For `ON`, `WHERE`, and `GROUP BY` clauses: + +- Arbitrary expressions cannot be used in `ON`, `WHERE`, and `GROUP BY` clauses, but you can define an expression in a `SELECT` clause and then use it in these clauses via an alias. + +### WHERE Clause {#select-where} + +If there is a WHERE clause, it must contain an expression with the UInt8 type. This is usually an expression with comparison and logical operators. +This expression will be used for filtering data before all other transformations. + +If indexes are supported by the database table engine, the expression is evaluated on the ability to use indexes. + +### PREWHERE Clause {#prewhere-clause} + +This clause has the same meaning as the WHERE clause. The difference is in which data is read from the table. +When using PREWHERE, first only the columns necessary for executing PREWHERE are read. Then the other columns are read that are needed for running the query, but only those blocks where the PREWHERE expression is true. + +It makes sense to use PREWHERE if there are filtration conditions that are used by a minority of the columns in the query, but that provide strong data filtration. This reduces the volume of data to read. + +For example, it is useful to write PREWHERE for queries that extract a large number of columns, but that only have filtration for a few columns. + +PREWHERE is only supported by tables from the `*MergeTree` family. + +A query may simultaneously specify PREWHERE and WHERE. In this case, PREWHERE precedes WHERE. + +If the ‘optimize\_move\_to\_prewhere’ setting is set to 1 and PREWHERE is omitted, the system uses heuristics to automatically move parts of expressions from WHERE to PREWHERE. + +### GROUP BY Clause {#select-group-by-clause} + +This is one of the most important parts of a column-oriented DBMS. + +If there is a GROUP BY clause, it must contain a list of expressions. Each expression will be referred to here as a “key”. +All the expressions in the SELECT, HAVING, and ORDER BY clauses must be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. + +If a query contains only table columns inside aggregate functions, the GROUP BY clause can be omitted, and aggregation by an empty set of keys is assumed. + +Example: + +``` sql +SELECT + count(), + median(FetchTiming > 60 ? 60 : FetchTiming), + count() - sum(Refresh) +FROM hits +``` + +However, in contrast to standard SQL, if the table doesn’t have any rows (either there aren’t any at all, or there aren’t any after using WHERE to filter), an empty result is returned, and not the result from one of the rows containing the initial values of aggregate functions. + +As opposed to MySQL (and conforming to standard SQL), you can’t get some value of some column that is not in a key or aggregate function (except constant expressions). To work around this, you can use the ‘any’ aggregate function (get the first encountered value) or ‘min/max’. + +Example: + +``` sql +SELECT + domainWithoutWWW(URL) AS domain, + count(), + any(Title) AS title -- getting the first occurred page header for each domain. +FROM hits +GROUP BY domain +``` + +For every different key value encountered, GROUP BY calculates a set of aggregate function values. + +GROUP BY is not supported for array columns. + +A constant can’t be specified as arguments for aggregate functions. Example: sum(1). Instead of this, you can get rid of the constant. Example: `count()`. + +#### NULL processing {#null-processing} + +For grouping, ClickHouse interprets [NULL](../syntax.md) as a value, and `NULL=NULL`. + +Here’s an example to show what this means. + +Assume you have this table: + +``` text +┌─x─┬────y─┐ +│ 1 │ 2 │ +│ 2 │ ᴺᵁᴸᴸ │ +│ 3 │ 2 │ +│ 3 │ 3 │ +│ 3 │ ᴺᵁᴸᴸ │ +└───┴──────┘ +``` + +The query `SELECT sum(x), y FROM t_null_big GROUP BY y` results in: + +``` text +┌─sum(x)─┬────y─┐ +│ 4 │ 2 │ +│ 3 │ 3 │ +│ 5 │ ᴺᵁᴸᴸ │ +└────────┴──────┘ +``` + +You can see that `GROUP BY` for `y = NULL` summed up `x`, as if `NULL` is this value. + +If you pass several keys to `GROUP BY`, the result will give you all the combinations of the selection, as if `NULL` were a specific value. + +#### WITH TOTALS Modifier {#with-totals-modifier} + +If the WITH TOTALS modifier is specified, another row will be calculated. This row will have key columns containing default values (zeros or empty lines), and columns of aggregate functions with the values calculated across all the rows (the “total” values). + +This extra row is output in JSON\*, TabSeparated\*, and Pretty\* formats, separately from the other rows. In the other formats, this row is not output. + +In JSON\* formats, this row is output as a separate ‘totals’ field. In TabSeparated\* formats, the row comes after the main result, preceded by an empty row (after the other data). In Pretty\* formats, the row is output as a separate table after the main result. + +`WITH TOTALS` can be run in different ways when HAVING is present. The behavior depends on the ‘totals\_mode’ setting. +By default, `totals_mode = 'before_having'`. In this case, ‘totals’ is calculated across all rows, including the ones that don’t pass through HAVING and ‘max\_rows\_to\_group\_by’. + +The other alternatives include only the rows that pass through HAVING in ‘totals’, and behave differently with the setting `max_rows_to_group_by` and `group_by_overflow_mode = 'any'`. + +`after_having_exclusive` – Don’t include rows that didn’t pass through `max_rows_to_group_by`. In other words, ‘totals’ will have less than or the same number of rows as it would if `max_rows_to_group_by` were omitted. + +`after_having_inclusive` – Include all the rows that didn’t pass through ‘max\_rows\_to\_group\_by’ in ‘totals’. In other words, ‘totals’ will have more than or the same number of rows as it would if `max_rows_to_group_by` were omitted. + +`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn’t pass through ‘max\_rows\_to\_group\_by’ in ‘totals’. Otherwise, do not include them. + +`totals_auto_threshold` – By default, 0.5. The coefficient for `after_having_auto`. + +If `max_rows_to_group_by` and `group_by_overflow_mode = 'any'` are not used, all variations of `after_having` are the same, and you can use any of them (for example, `after_having_auto`). + +You can use WITH TOTALS in subqueries, including subqueries in the JOIN clause (in this case, the respective total values are combined). + +#### GROUP BY in External Memory {#select-group-by-in-external-memory} + +You can enable dumping temporary data to the disk to restrict memory usage during `GROUP BY`. +The [max\_bytes\_before\_external\_group\_by](../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) setting determines the threshold RAM consumption for dumping `GROUP BY` temporary data to the file system. If set to 0 (the default), it is disabled. + +When using `max_bytes_before_external_group_by`, we recommend that you set `max_memory_usage` about twice as high. This is necessary because there are two stages to aggregation: reading the date and forming intermediate data (1) and merging the intermediate data (2). Dumping data to the file system can only occur during stage 1. If the temporary data wasn’t dumped, then stage 2 might require up to the same amount of memory as in stage 1. + +For example, if [max\_memory\_usage](../../operations/settings/settings.md#settings_max_memory_usage) was set to 10000000000 and you want to use external aggregation, it makes sense to set `max_bytes_before_external_group_by` to 10000000000, and max\_memory\_usage to 20000000000. When external aggregation is triggered (if there was at least one dump of temporary data), maximum consumption of RAM is only slightly more than `max_bytes_before_external_group_by`. + +With distributed query processing, external aggregation is performed on remote servers. In order for the requester server to use only a small amount of RAM, set `distributed_aggregation_memory_efficient` to 1. + +When merging data flushed to the disk, as well as when merging results from remote servers when the `distributed_aggregation_memory_efficient` setting is enabled, consumes up to `1/256 * the_number_of_threads` from the total amount of RAM. + +When external aggregation is enabled, if there was less than `max_bytes_before_external_group_by` of data (i.e. data was not flushed), the query runs just as fast as without external aggregation. If any temporary data was flushed, the run time will be several times longer (approximately three times). + +If you have an `ORDER BY` with a `LIMIT` after `GROUP BY`, then the amount of used RAM depends on the amount of data in `LIMIT`, not in the whole table. But if the `ORDER BY` doesn’t have `LIMIT`, don’t forget to enable external sorting (`max_bytes_before_external_sort`). + +### LIMIT BY Clause {#limit-by-clause} + +A query with the `LIMIT n BY expressions` clause selects the first `n` rows for each distinct value of `expressions`. The key for `LIMIT BY` can contain any number of [expressions](../syntax.md#syntax-expressions). + +ClickHouse supports the following syntax: + +- `LIMIT [offset_value, ]n BY expressions` +- `LIMIT n OFFSET offset_value BY expressions` + +During query processing, ClickHouse selects data ordered by sorting key. The sorting key is set explicitly using an [ORDER BY](#select-order-by) clause or implicitly as a property of the table engine. Then ClickHouse applies `LIMIT n BY expressions` and returns the first `n` rows for each distinct combination of `expressions`. If `OFFSET` is specified, then for each data block that belongs to a distinct combination of `expressions`, ClickHouse skips `offset_value` number of rows from the beginning of the block and returns a maximum of `n` rows as a result. If `offset_value` is bigger than the number of rows in the data block, ClickHouse returns zero rows from the block. + +`LIMIT BY` is not related to `LIMIT`. They can both be used in the same query. + +**Examples** + +Sample table: + +``` sql +CREATE TABLE limit_by(id Int, val Int) ENGINE = Memory; +INSERT INTO limit_by values(1, 10), (1, 11), (1, 12), (2, 20), (2, 21); +``` + +Queries: + +``` sql +SELECT * FROM limit_by ORDER BY id, val LIMIT 2 BY id +``` + +``` text +┌─id─┬─val─┐ +│ 1 │ 10 │ +│ 1 │ 11 │ +│ 2 │ 20 │ +│ 2 │ 21 │ +└────┴─────┘ +``` + +``` sql +SELECT * FROM limit_by ORDER BY id, val LIMIT 1, 2 BY id +``` + +``` text +┌─id─┬─val─┐ +│ 1 │ 11 │ +│ 1 │ 12 │ +│ 2 │ 21 │ +└────┴─────┘ +``` + +The `SELECT * FROM limit_by ORDER BY id, val LIMIT 2 OFFSET 1 BY id` query returns the same result. + +The following query returns the top 5 referrers for each `domain, device_type` pair with a maximum of 100 rows in total (`LIMIT n BY + LIMIT`). + +``` sql +SELECT + domainWithoutWWW(URL) AS domain, + domainWithoutWWW(REFERRER_URL) AS referrer, + device_type, + count() cnt +FROM hits +GROUP BY domain, referrer, device_type +ORDER BY cnt DESC +LIMIT 5 BY domain, device_type +LIMIT 100 +``` + +### HAVING Clause {#having-clause} + +Allows filtering the result received after GROUP BY, similar to the WHERE clause. +WHERE and HAVING differ in that WHERE is performed before aggregation (GROUP BY), while HAVING is performed after it. +If aggregation is not performed, HAVING can’t be used. + +### ORDER BY Clause {#select-order-by} + +The ORDER BY clause contains a list of expressions, which can each be assigned DESC or ASC (the sorting direction). If the direction is not specified, ASC is assumed. ASC is sorted in ascending order, and DESC in descending order. The sorting direction applies to a single expression, not to the entire list. Example: `ORDER BY Visits DESC, SearchPhrase` + +For sorting by String values, you can specify collation (comparison). Example: `ORDER BY SearchPhrase COLLATE 'tr'` - for sorting by keyword in ascending order, using the Turkish alphabet, case insensitive, assuming that strings are UTF-8 encoded. COLLATE can be specified or not for each expression in ORDER BY independently. If ASC or DESC is specified, COLLATE is specified after it. When using COLLATE, sorting is always case-insensitive. + +We only recommend using COLLATE for final sorting of a small number of rows, since sorting with COLLATE is less efficient than normal sorting by bytes. + +Rows that have identical values for the list of sorting expressions are output in an arbitrary order, which can also be nondeterministic (different each time). +If the ORDER BY clause is omitted, the order of the rows is also undefined, and may be nondeterministic as well. + +`NaN` and `NULL` sorting order: + +- With the modifier `NULLS FIRST` — First `NULL`, then `NaN`, then other values. +- With the modifier `NULLS LAST` — First the values, then `NaN`, then `NULL`. +- Default — The same as with the `NULLS LAST` modifier. + +Example: + +For the table + +``` text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 2 │ 2 │ +│ 1 │ nan │ +│ 2 │ 2 │ +│ 3 │ 4 │ +│ 5 │ 6 │ +│ 6 │ nan │ +│ 7 │ ᴺᵁᴸᴸ │ +│ 6 │ 7 │ +│ 8 │ 9 │ +└───┴──────┘ +``` + +Run the query `SELECT * FROM t_null_nan ORDER BY y NULLS FIRST` to get: + +``` text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 7 │ ᴺᵁᴸᴸ │ +│ 1 │ nan │ +│ 6 │ nan │ +│ 2 │ 2 │ +│ 2 │ 2 │ +│ 3 │ 4 │ +│ 5 │ 6 │ +│ 6 │ 7 │ +│ 8 │ 9 │ +└───┴──────┘ +``` + +When floating point numbers are sorted, NaNs are separate from the other values. Regardless of the sorting order, NaNs come at the end. In other words, for ascending sorting they are placed as if they are larger than all the other numbers, while for descending sorting they are placed as if they are smaller than the rest. + +Less RAM is used if a small enough LIMIT is specified in addition to ORDER BY. Otherwise, the amount of memory spent is proportional to the volume of data for sorting. For distributed query processing, if GROUP BY is omitted, sorting is partially done on remote servers, and the results are merged on the requestor server. This means that for distributed sorting, the volume of data to sort can be greater than the amount of memory on a single server. + +If there is not enough RAM, it is possible to perform sorting in external memory (creating temporary files on a disk). Use the setting `max_bytes_before_external_sort` for this purpose. If it is set to 0 (the default), external sorting is disabled. If it is enabled, when the volume of data to sort reaches the specified number of bytes, the collected data is sorted and dumped into a temporary file. After all data is read, all the sorted files are merged and the results are output. Files are written to the /var/lib/clickhouse/tmp/ directory in the config (by default, but you can use the ‘tmp\_path’ parameter to change this setting). + +Running a query may use more memory than ‘max\_bytes\_before\_external\_sort’. For this reason, this setting must have a value significantly smaller than ‘max\_memory\_usage’. As an example, if your server has 128 GB of RAM and you need to run a single query, set ‘max\_memory\_usage’ to 100 GB, and ‘max\_bytes\_before\_external\_sort’ to 80 GB. + +External sorting works much less effectively than sorting in RAM. + +### SELECT Clause {#select-select} + +[Expressions](../syntax.md#syntax-expressions) specified in the `SELECT` clause are calculated after all the operations in the clauses described above are finished. These expressions work as if they apply to separate rows in the result. If expressions in the `SELECT` clause contain aggregate functions, then ClickHouse processes aggregate functions and expressions used as their arguments during the [GROUP BY](#select-group-by-clause) aggregation. + +If you want to include all columns in the result, use the asterisk (`*`) symbol. For example, `SELECT * FROM ...`. + +To match some columns in the result with a [re2](https://en.wikipedia.org/wiki/RE2_(software)) regular expression, you can use the `COLUMNS` expression. + +``` sql +COLUMNS('regexp') +``` + +For example, consider the table: + +``` sql +CREATE TABLE default.col_names (aa Int8, ab Int8, bc Int8) ENGINE = TinyLog +``` + +The following query selects data from all the columns containing the `a` symbol in their name. + +``` sql +SELECT COLUMNS('a') FROM col_names +``` + +``` text +┌─aa─┬─ab─┐ +│ 1 │ 1 │ +└────┴────┘ +``` + +The selected columns are returned not in the alphabetical order. + +You can use multiple `COLUMNS` expressions in a query and apply functions to them. + +For example: + +``` sql +SELECT COLUMNS('a'), COLUMNS('c'), toTypeName(COLUMNS('c')) FROM col_names +``` + +``` text +┌─aa─┬─ab─┬─bc─┬─toTypeName(bc)─┐ +│ 1 │ 1 │ 1 │ Int8 │ +└────┴────┴────┴────────────────┘ +``` + +Each column returned by the `COLUMNS` expression is passed to the function as a separate argument. Also you can pass other arguments to the function if it supports them. Be careful when using functions. If a function doesn’t support the number of arguments you have passed to it, ClickHouse throws an exception. + +For example: + +``` sql +SELECT COLUMNS('a') + COLUMNS('c') FROM col_names +``` + +``` text +Received exception from server (version 19.14.1): +Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of arguments for function plus doesn't match: passed 3, should be 2. +``` + +In this example, `COLUMNS('a')` returns two columns: `aa` and `ab`. `COLUMNS('c')` returns the `bc` column. The `+` operator can’t apply to 3 arguments, so ClickHouse throws an exception with the relevant message. + +Columns that matched the `COLUMNS` expression can have different data types. If `COLUMNS` doesn’t match any columns and is the only expression in `SELECT`, ClickHouse throws an exception. + +### DISTINCT Clause {#select-distinct} + +If DISTINCT is specified, only a single row will remain out of all the sets of fully matching rows in the result. +The result will be the same as if GROUP BY were specified across all the fields specified in SELECT without aggregate functions. But there are several differences from GROUP BY: + +- DISTINCT can be applied together with GROUP BY. +- When ORDER BY is omitted and LIMIT is defined, the query stops running immediately after the required number of different rows has been read. +- Data blocks are output as they are processed, without waiting for the entire query to finish running. + +DISTINCT is not supported if SELECT has at least one array column. + +`DISTINCT` works with [NULL](../syntax.md) as if `NULL` were a specific value, and `NULL=NULL`. In other words, in the `DISTINCT` results, different combinations with `NULL` only occur once. + +ClickHouse supports using the `DISTINCT` and `ORDER BY` clauses for different columns in one query. The `DISTINCT` clause is executed before the `ORDER BY` clause. + +Example table: + +``` text +┌─a─┬─b─┐ +│ 2 │ 1 │ +│ 1 │ 2 │ +│ 3 │ 3 │ +│ 2 │ 4 │ +└───┴───┘ +``` + +When selecting data with the `SELECT DISTINCT a FROM t1 ORDER BY b ASC` query, we get the following result: + +``` text +┌─a─┐ +│ 2 │ +│ 1 │ +│ 3 │ +└───┘ +``` + +If we change the sorting direction `SELECT DISTINCT a FROM t1 ORDER BY b DESC`, we get the following result: + +``` text +┌─a─┐ +│ 3 │ +│ 1 │ +│ 2 │ +└───┘ +``` + +Row `2, 4` was cut before sorting. + +Take this implementation specificity into account when programming queries. + +### LIMIT Clause {#limit-clause} + +`LIMIT m` allows you to select the first `m` rows from the result. + +`LIMIT n, m` allows you to select the first `m` rows from the result after skipping the first `n` rows. The `LIMIT m OFFSET n` syntax is also supported. + +`n` and `m` must be non-negative integers. + +If there isn’t an `ORDER BY` clause that explicitly sorts results, the result may be arbitrary and nondeterministic. + +### UNION ALL Clause {#union-all-clause} + +You can use UNION ALL to combine any number of queries. Example: + +``` sql +SELECT CounterID, 1 AS table, toInt64(count()) AS c + FROM test.hits + GROUP BY CounterID + +UNION ALL + +SELECT CounterID, 2 AS table, sum(Sign) AS c + FROM test.visits + GROUP BY CounterID + HAVING c > 0 +``` + +Only UNION ALL is supported. The regular UNION (UNION DISTINCT) is not supported. If you need UNION DISTINCT, you can write SELECT DISTINCT from a subquery containing UNION ALL. + +Queries that are parts of UNION ALL can be run simultaneously, and their results can be mixed together. + +The structure of results (the number and type of columns) must match for the queries. But the column names can differ. In this case, the column names for the final result will be taken from the first query. Type casting is performed for unions. For example, if two queries being combined have the same field with non-`Nullable` and `Nullable` types from a compatible type, the resulting `UNION ALL` has a `Nullable` type field. + +Queries that are parts of UNION ALL can’t be enclosed in brackets. ORDER BY and LIMIT are applied to separate queries, not to the final result. If you need to apply a conversion to the final result, you can put all the queries with UNION ALL in a subquery in the FROM clause. + +### INTO OUTFILE Clause {#into-outfile-clause} + +Add the `INTO OUTFILE filename` clause (where filename is a string literal) to redirect query output to the specified file. +In contrast to MySQL, the file is created on the client side. The query will fail if a file with the same filename already exists. +This functionality is available in the command-line client and clickhouse-local (a query sent via HTTP interface will fail). + +The default output format is TabSeparated (the same as in the command-line client batch mode). + +### FORMAT Clause {#format-clause} + +Specify ‘FORMAT format’ to get data in any specified format. +You can use this for convenience, or for creating dumps. +For more information, see the section “Formats”. +If the FORMAT clause is omitted, the default format is used, which depends on both the settings and the interface used for accessing the DB. For the HTTP interface and the command-line client in batch mode, the default format is TabSeparated. For the command-line client in interactive mode, the default format is PrettyCompact (it has attractive and compact tables). + +When using the command-line client, data is passed to the client in an internal efficient format. The client independently interprets the FORMAT clause of the query and formats the data itself (thus relieving the network and the server from the load). + +### IN Operators {#select-in-operators} + +The `IN`, `NOT IN`, `GLOBAL IN`, and `GLOBAL NOT IN` operators are covered separately, since their functionality is quite rich. + +The left side of the operator is either a single column or a tuple. + +Examples: + +``` sql +SELECT UserID IN (123, 456) FROM ... +SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... +``` + +If the left side is a single column that is in the index, and the right side is a set of constants, the system uses the index for processing the query. + +Don’t list too many values explicitly (i.e. millions). If a data set is large, put it in a temporary table (for example, see the section “External data for query processing”), then use a subquery. + +The right side of the operator can be a set of constant expressions, a set of tuples with constant expressions (shown in the examples above), or the name of a database table or SELECT subquery in brackets. + +If the right side of the operator is the name of a table (for example, `UserID IN users`), this is equivalent to the subquery `UserID IN (SELECT * FROM users)`. Use this when working with external data that is sent along with the query. For example, the query can be sent together with a set of user IDs loaded to the ‘users’ temporary table, which should be filtered. + +If the right side of the operator is a table name that has the Set engine (a prepared data set that is always in RAM), the data set will not be created over again for each query. + +The subquery may specify more than one column for filtering tuples. +Example: + +``` sql +SELECT (CounterID, UserID) IN (SELECT CounterID, UserID FROM ...) FROM ... +``` + +The columns to the left and right of the IN operator should have the same type. + +The IN operator and subquery may occur in any part of the query, including in aggregate functions and lambda functions. +Example: + +``` sql +SELECT + EventDate, + avg(UserID IN + ( + SELECT UserID + FROM test.hits + WHERE EventDate = toDate('2014-03-17') + )) AS ratio +FROM test.hits +GROUP BY EventDate +ORDER BY EventDate ASC +``` + +``` text +┌──EventDate─┬────ratio─┐ +│ 2014-03-17 │ 1 │ +│ 2014-03-18 │ 0.807696 │ +│ 2014-03-19 │ 0.755406 │ +│ 2014-03-20 │ 0.723218 │ +│ 2014-03-21 │ 0.697021 │ +│ 2014-03-22 │ 0.647851 │ +│ 2014-03-23 │ 0.648416 │ +└────────────┴──────────┘ +``` + +For each day after March 17th, count the percentage of pageviews made by users who visited the site on March 17th. +A subquery in the IN clause is always run just one time on a single server. There are no dependent subqueries. + +#### NULL processing {#null-processing-1} + +During request processing, the IN operator assumes that the result of an operation with [NULL](../syntax.md) is always equal to `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared. + +Here is an example with the `t_null` table: + +``` text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 2 │ 3 │ +└───┴──────┘ +``` + +Running the query `SELECT x FROM t_null WHERE y IN (NULL,3)` gives you the following result: + +``` text +┌─x─┐ +│ 2 │ +└───┘ +``` + +You can see that the row in which `y = NULL` is thrown out of the query results. This is because ClickHouse can’t decide whether `NULL` is included in the `(NULL,3)` set, returns `0` as the result of the operation, and `SELECT` excludes this row from the final output. + +``` sql +SELECT y IN (NULL, 3) +FROM t_null +``` + +``` text +┌─in(y, tuple(NULL, 3))─┐ +│ 0 │ +│ 1 │ +└───────────────────────┘ +``` + +#### Distributed Subqueries {#select-distributed-subqueries} + +There are two options for IN-s with subqueries (similar to JOINs): normal `IN` / `JOIN` and `GLOBAL IN` / `GLOBAL JOIN`. They differ in how they are run for distributed query processing. + +!!! attention "Attention" + Remember that the algorithms described below may work differently depending on the [settings](../../operations/settings/settings.md) `distributed_product_mode` setting. + +When using the regular IN, the query is sent to remote servers, and each of them runs the subqueries in the `IN` or `JOIN` clause. + +When using `GLOBAL IN` / `GLOBAL JOINs`, first all the subqueries are run for `GLOBAL IN` / `GLOBAL JOINs`, and the results are collected in temporary tables. Then the temporary tables are sent to each remote server, where the queries are run using this temporary data. + +For a non-distributed query, use the regular `IN` / `JOIN`. + +Be careful when using subqueries in the `IN` / `JOIN` clauses for distributed query processing. + +Let’s look at some examples. Assume that each server in the cluster has a normal **local\_table**. Each server also has a **distributed\_table** table with the **Distributed** type, which looks at all the servers in the cluster. + +For a query to the **distributed\_table**, the query will be sent to all the remote servers and run on them using the **local\_table**. + +For example, the query + +``` sql +SELECT uniq(UserID) FROM distributed_table +``` + +will be sent to all remote servers as + +``` sql +SELECT uniq(UserID) FROM local_table +``` + +and run on each of them in parallel, until it reaches the stage where intermediate results can be combined. Then the intermediate results will be returned to the requestor server and merged on it, and the final result will be sent to the client. + +Now let’s examine a query with IN: + +``` sql +SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) +``` + +- Calculation of the intersection of audiences of two sites. + +This query will be sent to all remote servers as + +``` sql +SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) +``` + +In other words, the data set in the IN clause will be collected on each server independently, only across the data that is stored locally on each of the servers. + +This will work correctly and optimally if you are prepared for this case and have spread data across the cluster servers such that the data for a single UserID resides entirely on a single server. In this case, all the necessary data will be available locally on each server. Otherwise, the result will be inaccurate. We refer to this variation of the query as “local IN”. + +To correct how the query works when data is spread randomly across the cluster servers, you could specify **distributed\_table** inside a subquery. The query would look like this: + +``` sql +SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) +``` + +This query will be sent to all remote servers as + +``` sql +SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) +``` + +The subquery will begin running on each remote server. Since the subquery uses a distributed table, the subquery that is on each remote server will be resent to every remote server as + +``` sql +SELECT UserID FROM local_table WHERE CounterID = 34 +``` + +For example, if you have a cluster of 100 servers, executing the entire query will require 10,000 elementary requests, which is generally considered unacceptable. + +In such cases, you should always use GLOBAL IN instead of IN. Let’s look at how it works for the query + +``` sql +SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID GLOBAL IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) +``` + +The requestor server will run the subquery + +``` sql +SELECT UserID FROM distributed_table WHERE CounterID = 34 +``` + +and the result will be put in a temporary table in RAM. Then the request will be sent to each remote server as + +``` sql +SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL IN _data1 +``` + +and the temporary table `_data1` will be sent to every remote server with the query (the name of the temporary table is implementation-defined). + +This is more optimal than using the normal IN. However, keep the following points in mind: + +1. When creating a temporary table, data is not made unique. To reduce the volume of data transmitted over the network, specify DISTINCT in the subquery. (You don’t need to do this for a normal IN.) +2. The temporary table will be sent to all the remote servers. Transmission does not account for network topology. For example, if 10 remote servers reside in a datacenter that is very remote in relation to the requestor server, the data will be sent 10 times over the channel to the remote datacenter. Try to avoid large data sets when using GLOBAL IN. +3. When transmitting data to remote servers, restrictions on network bandwidth are not configurable. You might overload the network. +4. Try to distribute data across servers so that you don’t need to use GLOBAL IN on a regular basis. +5. If you need to use GLOBAL IN often, plan the location of the ClickHouse cluster so that a single group of replicas resides in no more than one data center with a fast network between them, so that a query can be processed entirely within a single data center. + +It also makes sense to specify a local table in the `GLOBAL IN` clause, in case this local table is only available on the requestor server and you want to use data from it on remote servers. + +### Extreme Values {#extreme-values} + +In addition to results, you can also get minimum and maximum values for the results columns. To do this, set the **extremes** setting to 1. Minimums and maximums are calculated for numeric types, dates, and dates with times. For other columns, the default values are output. + +An extra two rows are calculated – the minimums and maximums, respectively. These extra two rows are output in `JSON*`, `TabSeparated*`, and `Pretty*` [formats](../../interfaces/formats.md), separate from the other rows. They are not output for other formats. + +In `JSON*` formats, the extreme values are output in a separate ‘extremes’ field. In `TabSeparated*` formats, the row comes after the main result, and after ‘totals’ if present. It is preceded by an empty row (after the other data). In `Pretty*` formats, the row is output as a separate table after the main result, and after `totals` if present. + +Extreme values are calculated for rows before `LIMIT`, but after `LIMIT BY`. However, when using `LIMIT offset, size`, the rows before `offset` are included in `extremes`. In stream requests, the result may also include a small number of rows that passed through `LIMIT`. + +### Notes {#notes} + +The `GROUP BY` and `ORDER BY` clauses do not support positional arguments. This contradicts MySQL, but conforms to standard SQL. +For example, `GROUP BY 1, 2` will be interpreted as grouping by constants (i.e. aggregation of all rows into one). + +You can use synonyms (`AS` aliases) in any part of a query. + +You can put an asterisk in any part of a query instead of an expression. When the query is analyzed, the asterisk is expanded to a list of all table columns (excluding the `MATERIALIZED` and `ALIAS` columns). There are only a few cases when using an asterisk is justified: + +- When creating a table dump. +- For tables containing just a few columns, such as system tables. +- For getting information about what columns are in a table. In this case, set `LIMIT 1`. But it is better to use the `DESC TABLE` query. +- When there is strong filtration on a small number of columns using `PREWHERE`. +- In subqueries (since columns that aren’t needed for the external query are excluded from subqueries). + +In all other cases, we don’t recommend using the asterisk, since it only gives you the drawbacks of a columnar DBMS instead of the advantages. In other words using the asterisk is not recommended. + +[Original article](https://clickhouse.tech/docs/en/query_language/select/) From 01945bf5d9d5098b211dc181de5d7098b14a4add Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 16:27:33 +0300 Subject: [PATCH 237/743] Fix style --- src/DataStreams/MergingSortedBlockInputStream.h | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/DataStreams/MergingSortedBlockInputStream.h b/src/DataStreams/MergingSortedBlockInputStream.h index 468ac2dcfdc..9a732fff947 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.h +++ b/src/DataStreams/MergingSortedBlockInputStream.h @@ -15,12 +15,6 @@ namespace Poco { class Logger; } namespace DB { -namespace ErrorCodes -{ - extern const int CORRUPTED_DATA; -} - - /** Merges several sorted streams into one sorted stream. */ class MergingSortedBlockInputStream : public IBlockInputStream From 24f715ad87785e0c55b62f6b3392f7772ae9205b Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Tue, 14 Apr 2020 17:43:09 +0300 Subject: [PATCH 238/743] tech debt --- src/IO/readDecimalText.h | 5 +- src/Interpreters/ColumnNamesContext.h | 89 ------------------- src/Interpreters/DatabaseAndTableWithAlias.h | 44 ++++----- src/Interpreters/JoinedTables.h | 2 + ...text.cpp => RequiredSourceColumnsData.cpp} | 43 +++------ src/Interpreters/RequiredSourceColumnsData.h | 51 +++++++++++ .../RequiredSourceColumnsVisitor.cpp | 13 +-- .../RequiredSourceColumnsVisitor.h | 6 +- src/Parsers/ASTIdentifier.h | 2 + 9 files changed, 90 insertions(+), 165 deletions(-) delete mode 100644 src/Interpreters/ColumnNamesContext.h rename src/Interpreters/{ColumnNamesContext.cpp => RequiredSourceColumnsData.cpp} (62%) create mode 100644 src/Interpreters/RequiredSourceColumnsData.h diff --git a/src/IO/readDecimalText.h b/src/IO/readDecimalText.h index 5b2a3f76481..6edc300eac8 100644 --- a/src/IO/readDecimalText.h +++ b/src/IO/readDecimalText.h @@ -10,7 +10,10 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } - +/// Try to read Decimal into underlying type T from ReadBuffer. Throws if 'digits_only' is set and there's unexpected symbol in input. +/// Returns integer 'exponent' factor that x should be muntiplyed by to get correct Decimal value: result = x * 10^exponent. +/// Use 'digits' input as max allowed meaning decimal digits in result. Place actual meanin digits in 'digits' output. +/// Do not care about decimal scale, only about meaning digits in decimal text representation. template inline bool readDigits(ReadBuffer & buf, T & x, unsigned int & digits, int & exponent, bool digits_only = false) { diff --git a/src/Interpreters/ColumnNamesContext.h b/src/Interpreters/ColumnNamesContext.h deleted file mode 100644 index c30102cf8d7..00000000000 --- a/src/Interpreters/ColumnNamesContext.h +++ /dev/null @@ -1,89 +0,0 @@ -#pragma once - -#include -#include - -#include -#include -#include -#include - -namespace DB -{ - -/// Information about table and column names extracted from ASTSelectQuery block. Do not include info from subselects. -struct ColumnNamesContext -{ - struct JoinedTable - { - const ASTTableExpression * expr = nullptr; - const ASTTableJoin * join = nullptr; - - std::optional alias() const - { - String alias; - if (expr) - { - if (expr->database_and_table_name) - alias = expr->database_and_table_name->tryGetAlias(); - else if (expr->table_function) - alias = expr->table_function->tryGetAlias(); - else if (expr->subquery) - alias = expr->subquery->tryGetAlias(); - } - if (!alias.empty()) - return alias; - return {}; - } - - std::optional name() const - { - if (expr) - return tryGetIdentifierName(expr->database_and_table_name); - return {}; - } - - std::optional joinKind() const - { - if (join) - return join->kind; - return {}; - } - }; - - struct NameInfo - { - std::set aliases; - size_t appears = 0; - - void addInclusion(const String & alias) - { - if (!alias.empty()) - aliases.insert(alias); - ++appears; - } - }; - - std::unordered_map required_names; - NameSet table_aliases; - NameSet private_aliases; - NameSet complex_aliases; - NameSet masked_columns; - NameSet array_join_columns; - std::vector tables; /// ordered list of visited tables in FROM section with joins - bool has_table_join = false; - bool has_array_join = false; - - bool addTableAliasIfAny(const IAST & ast); - bool addColumnAliasIfAny(const IAST & ast); - void addColumnIdentifier(const ASTIdentifier & node); - bool addArrayJoinAliasIfAny(const IAST & ast); - void addArrayJoinIdentifier(const ASTIdentifier & node); - - NameSet requiredColumns() const; - size_t nameInclusion(const String & name) const; -}; - -std::ostream & operator << (std::ostream & os, const ColumnNamesContext & cols); - -} diff --git a/src/Interpreters/DatabaseAndTableWithAlias.h b/src/Interpreters/DatabaseAndTableWithAlias.h index 92d6d40b455..e28d76b12f3 100644 --- a/src/Interpreters/DatabaseAndTableWithAlias.h +++ b/src/Interpreters/DatabaseAndTableWithAlias.h @@ -49,61 +49,51 @@ struct TableWithColumnNames { DatabaseAndTableWithAlias table; Names columns; - Names hidden_columns; + Names hidden_columns; /// Not general columns like MATERIALIZED and ALIAS. They are omitted in * and t.* results. TableWithColumnNames(const DatabaseAndTableWithAlias & table_, const Names & columns_) : table(table_) , columns(columns_) - {} + { + columns_set.insert(columns.begin(), columns.end()); + } TableWithColumnNames(const DatabaseAndTableWithAlias table_, Names && columns_, Names && hidden_columns_) : table(table_) , columns(columns_) , hidden_columns(hidden_columns_) - {} - - bool hasColumn(const String & name) const { - if (columns_set.empty()) - { - columns_set.insert(columns.begin(), columns.end()); - columns_set.insert(hidden_columns.begin(), hidden_columns.end()); - } - - return columns_set.count(name); + columns_set.insert(columns.begin(), columns.end()); + columns_set.insert(hidden_columns.begin(), hidden_columns.end()); } + bool hasColumn(const String & name) const { return columns_set.count(name); } + private: - mutable NameSet columns_set; + NameSet columns_set; }; struct TableWithColumnNamesAndTypes { DatabaseAndTableWithAlias table; NamesAndTypesList columns; - NamesAndTypesList hidden_columns; + NamesAndTypesList hidden_columns; /// Not general columns like MATERIALIZED and ALIAS. They are omitted in * and t.* results. TableWithColumnNamesAndTypes(const DatabaseAndTableWithAlias & table_, const NamesAndTypesList & columns_) : table(table_) , columns(columns_) - {} - - bool hasColumn(const String & name) const { - if (names.empty()) - { - for (auto & col : columns) - names.insert(col.name); - for (auto & col : hidden_columns) - names.insert(col.name); - } - - return names.count(name); + for (auto & col : columns) + names.insert(col.name); } + bool hasColumn(const String & name) const { return names.count(name); } + void addHiddenColumns(const NamesAndTypesList & addition) { hidden_columns.insert(hidden_columns.end(), addition.begin(), addition.end()); + for (auto & col : addition) + names.insert(col.name); } TableWithColumnNames removeTypes() const @@ -122,7 +112,7 @@ struct TableWithColumnNamesAndTypes } private: - mutable NameSet names; + NameSet names; }; std::vector getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database); diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index 66b3c8de609..3bcec883f30 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -27,6 +27,8 @@ public: StoragePtr getLeftTableStorage(); bool resolveTables(); + + /// Make fake tables_with_columns[0] in case we have predefined input in InterpreterSelectQuery void makeFakeTable(StoragePtr storage, const Block & source_header); const std::vector & tablesWithColumns() const { return tables_with_columns; } diff --git a/src/Interpreters/ColumnNamesContext.cpp b/src/Interpreters/RequiredSourceColumnsData.cpp similarity index 62% rename from src/Interpreters/ColumnNamesContext.cpp rename to src/Interpreters/RequiredSourceColumnsData.cpp index c8fde183d96..b5a3544f22f 100644 --- a/src/Interpreters/ColumnNamesContext.cpp +++ b/src/Interpreters/RequiredSourceColumnsData.cpp @@ -1,21 +1,13 @@ -#include +#include +#include #include #include +#include namespace DB { -bool ColumnNamesContext::addTableAliasIfAny(const IAST & ast) -{ - String alias = ast.tryGetAlias(); - if (alias.empty()) - return false; - - table_aliases.insert(alias); - return true; -} - -bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast) +bool RequiredSourceColumnsData::addColumnAliasIfAny(const IAST & ast) { String alias = ast.tryGetAlias(); if (alias.empty()) @@ -28,7 +20,7 @@ bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast) return true; } -void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node) +void RequiredSourceColumnsData::addColumnIdentifier(const ASTIdentifier & node) { if (!IdentifierSemantic::getColumnName(node)) return; @@ -38,7 +30,7 @@ void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node) required_names[node.name].addInclusion(alias); } -bool ColumnNamesContext::addArrayJoinAliasIfAny(const IAST & ast) +bool RequiredSourceColumnsData::addArrayJoinAliasIfAny(const IAST & ast) { String alias = ast.tryGetAlias(); if (alias.empty()) @@ -48,12 +40,12 @@ bool ColumnNamesContext::addArrayJoinAliasIfAny(const IAST & ast) return true; } -void ColumnNamesContext::addArrayJoinIdentifier(const ASTIdentifier & node) +void RequiredSourceColumnsData::addArrayJoinIdentifier(const ASTIdentifier & node) { array_join_columns.insert(node.name); } -size_t ColumnNamesContext::nameInclusion(const String & name) const +size_t RequiredSourceColumnsData::nameInclusion(const String & name) const { auto it = required_names.find(name); if (it != required_names.end()) @@ -61,7 +53,7 @@ size_t ColumnNamesContext::nameInclusion(const String & name) const return 0; } -NameSet ColumnNamesContext::requiredColumns() const +NameSet RequiredSourceColumnsData::requiredColumns() const { NameSet required; for (const auto & pr : required_names) @@ -79,7 +71,7 @@ NameSet ColumnNamesContext::requiredColumns() const return required; } -std::ostream & operator << (std::ostream & os, const ColumnNamesContext & cols) +std::ostream & operator << (std::ostream & os, const RequiredSourceColumnsData & cols) { os << "required_names: "; for (const auto & pr : cols.required_names) @@ -89,21 +81,6 @@ std::ostream & operator << (std::ostream & os, const ColumnNamesContext & cols) os << "/'" << alias << "'"; os << ", "; } - os << "source_tables: "; - for (const auto & x : cols.tables) - { - auto alias = x.alias(); - auto name = x.name(); - if (alias && name) - os << "'" << *alias << "'/'" << *name << "', "; - else if (alias) - os << "'" << *alias << "', "; - else if (name) - os << "'" << *name << "', "; - } - os << "table_aliases: "; - for (const auto & x : cols.table_aliases) - os << "'" << x << "', "; os << "complex_aliases: "; for (const auto & x : cols.complex_aliases) os << "'" << x << "', "; diff --git a/src/Interpreters/RequiredSourceColumnsData.h b/src/Interpreters/RequiredSourceColumnsData.h new file mode 100644 index 00000000000..de1f3bc2721 --- /dev/null +++ b/src/Interpreters/RequiredSourceColumnsData.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include + +#include +#include +#include + +namespace DB +{ + +class ASTIdentifier; + +/// Information about table and column names extracted from ASTSelectQuery block. Do not include info from subselects. +struct RequiredSourceColumnsData +{ + struct NameInfo + { + std::set aliases; + size_t appears = 0; + + void addInclusion(const String & alias) + { + if (!alias.empty()) + aliases.insert(alias); + ++appears; + } + }; + + std::unordered_map required_names; + NameSet private_aliases; /// lambda aliases that should not be interpreted as required columns + NameSet complex_aliases; /// aliases to functions results: they are not required cause calculated by query itself + NameSet masked_columns; /// columns names masked by function aliases: we still need them in required columns + NameSet array_join_columns; /// Tech debt: we exclude ArrayJoin columns from general logic cause they have own logic outside + + bool has_table_join = false; + bool has_array_join = false; + + bool addColumnAliasIfAny(const IAST & ast); + void addColumnIdentifier(const ASTIdentifier & node); + bool addArrayJoinAliasIfAny(const IAST & ast); + void addArrayJoinIdentifier(const ASTIdentifier & node); + + NameSet requiredColumns() const; + size_t nameInclusion(const String & name) const; +}; + +std::ostream & operator << (std::ostream & os, const RequiredSourceColumnsData & cols); + +} diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 469a5852fa5..08bfbf30f7f 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -91,14 +91,12 @@ void RequiredSourceColumnsMatcher::visit(const ASTPtr & ast, Data & data) if (auto * t = ast->as()) { - data.addTableAliasIfAny(*ast); visit(*t, ast, data); return; } if (ast->as()) { - data.addTableAliasIfAny(*ast); return; } @@ -174,20 +172,11 @@ void RequiredSourceColumnsMatcher::visit(const ASTTablesInSelectQueryElement & n if (join) data.has_table_join = true; - data.tables.emplace_back(ColumnNamesContext::JoinedTable{expr, join}); } /// ASTIdentifiers here are tables. Do not visit them as generic ones. -void RequiredSourceColumnsMatcher::visit(const ASTTableExpression & node, const ASTPtr &, Data & data) +void RequiredSourceColumnsMatcher::visit(const ASTTableExpression &, const ASTPtr &, Data &) { - if (node.database_and_table_name) - data.addTableAliasIfAny(*node.database_and_table_name); - - if (node.table_function) - data.addTableAliasIfAny(*node.table_function); - - if (node.subquery) - data.addTableAliasIfAny(*node.subquery); } void RequiredSourceColumnsMatcher::visit(const ASTArrayJoin & node, const ASTPtr &, Data & data) diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.h b/src/Interpreters/RequiredSourceColumnsVisitor.h index 1f6ff482e3a..53decb3a849 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.h +++ b/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -21,7 +21,7 @@ class RequiredSourceColumnsMatcher { public: using Visitor = ConstInDepthNodeVisitor; - using Data = ColumnNamesContext; + using Data = RequiredSourceColumnsData; static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); static void visit(const ASTPtr & ast, Data & data); @@ -35,7 +35,7 @@ private: static void visit(const ASTSelectQuery & select, const ASTPtr &, Data & data); }; -/// Extracts all the information about columns and tables from ASTSelectQuery block into ColumnNamesContext object. +/// Extracts all the information about columns and tables from ASTSelectQuery block into Data object. /// It doesn't use anything but AST. It visits nodes from bottom to top except ASTFunction content to get aliases in right manner. /// @note There's some ambiguousness with nested columns names that can't be solved without schema. using RequiredSourceColumnsVisitor = RequiredSourceColumnsMatcher::Visitor; diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 9e28a1461ca..c13c2c3f977 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -40,6 +40,8 @@ public: bool isShort() const { return name_parts.empty() || name == name_parts.back(); } void setShortName(const String & new_name); + + /// Restore name field from name_parts in case it was cropped by analyzer but we need a full form for future (re)analyze. void restoreCompoundName(); const String & shortName() const From 1d3fe23b3b9215070c41166701750288ad350f13 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 14 Apr 2020 20:41:06 +0300 Subject: [PATCH 239/743] review fixes --- src/Interpreters/ActionsVisitor.cpp | 24 ++++--------------- src/Interpreters/ActionsVisitor.h | 21 +++++++++++++++- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Parsers/ASTLiteral.h | 13 ++++++++-- .../01101_literal_column_clash.reference | 2 ++ .../01101_literal_column_clash.sql | 1 + 6 files changed, 40 insertions(+), 23 deletions(-) rename {dbms/tests => tests}/queries/0_stateless/01101_literal_column_clash.reference (77%) rename {dbms/tests => tests}/queries/0_stateless/01101_literal_column_clash.sql (85%) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 317ef5494e9..6bb0589b4b3 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -195,20 +195,6 @@ SetPtr makeExplicitSet( return set; } -static String getUniqueName(ActionsVisitor::Data & data, const String & prefix) -{ - auto & block = data.getSampleBlock(); - auto result = prefix; - - while (block.has(result)) - { - result = prefix + "_" + toString(data.next_unique_suffix); - ++data.next_unique_suffix; - } - - return result; -} - ScopeStack::ScopeStack(const ExpressionActionsPtr & actions, const Context & context_) : context(context_) { @@ -464,7 +450,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). if (!prepared_set->empty()) - column.name = getUniqueName(data, "__set"); + column.name = data.getUniqueName("__set"); else column.name = child->getColumnName(); @@ -492,7 +478,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & ColumnWithTypeAndName column( ColumnConst::create(std::move(column_string), 1), std::make_shared(), - getUniqueName(data, "__joinGet")); + data.getUniqueName("__joinGet")); data.addAction(ExpressionAction::addColumn(column)); argument_types.push_back(column.type); argument_names.push_back(column.name); @@ -506,7 +492,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & // generated a unique column name for it. Use it instead of a generic // display name. auto child_column_name = child->getColumnName(); - auto as_literal = dynamic_cast(child.get()); + auto as_literal = child->as(); if (as_literal) { assert(!as_literal->unique_column_name.empty()); @@ -573,7 +559,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// We can not name `getColumnName()`, /// because it does not uniquely define the expression (the types of arguments can be different). - String lambda_name = getUniqueName(data, "__lambda"); + String lambda_name = data.getUniqueName("__lambda"); auto function_capture = std::make_unique( lambda_actions, captured, lambda_arguments, result_type, result_name); @@ -638,7 +624,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, else { const_cast(literal).unique_column_name - = getUniqueName(data, default_name); + = data.getUniqueName(default_name); } } diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index e67e181e009..dbcc54c01d6 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -42,7 +42,6 @@ struct ScopeStack const Context & context; -public: ScopeStack(const ExpressionActionsPtr & actions, const Context & context_); void pushLevel(const NamesAndTypesList & input_columns); @@ -127,6 +126,26 @@ public: { return actions_stack.getSampleBlock().has(columnName); } + + /* + * Generate a column name that is not present in the sample block, using + * the given prefix and an optional numeric suffix. + */ + String getUniqueName(const String & prefix) + { + const auto & block = getSampleBlock(); + auto result = prefix; + + // First, try the name without any suffix, because it is currently + // used both as a display name and a column id. + while (block.has(result)) + { + result = prefix + "_" + toString(next_unique_suffix); + ++next_unique_suffix; + } + + return result; + } }; static void visit(const ASTPtr & ast, Data & data); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 3aa1ad6676c..af11098b883 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -875,7 +875,7 @@ void SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & * names and identifiers for columns. This code is a workaround for * a particular subclass of problems, and not a proper solution. */ - if (auto as_literal = dynamic_cast(ast.get()); + if (auto as_literal = ast->as(); as_literal) { source_name = as_literal->unique_column_name; diff --git a/src/Parsers/ASTLiteral.h b/src/Parsers/ASTLiteral.h index e9fb3d3b8ec..18f440a81a4 100644 --- a/src/Parsers/ASTLiteral.h +++ b/src/Parsers/ASTLiteral.h @@ -17,12 +17,21 @@ class ASTLiteral : public ASTWithAlias public: Field value; - String unique_column_name; - /// For ConstantExpressionTemplate std::optional begin; std::optional end; + /* + * The name of the column corresponding to this literal. Only used to + * disambiguate the literal columns with the same display name that are + * created at the expression analyzer stage. In the future, we might want to + * have a full separation between display names and column identifiers. For + * now, this field is effectively just some private EA data. + */ + String unique_column_name; + + +public: ASTLiteral(Field && value_) : value(value_) {} ASTLiteral(const Field & value_) : value(value_) {} diff --git a/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference b/tests/queries/0_stateless/01101_literal_column_clash.reference similarity index 77% rename from dbms/tests/queries/0_stateless/01101_literal_column_clash.reference rename to tests/queries/0_stateless/01101_literal_column_clash.reference index 5cbe1f1eea3..094b70dd0b7 100644 --- a/dbms/tests/queries/0_stateless/01101_literal_column_clash.reference +++ b/tests/queries/0_stateless/01101_literal_column_clash.reference @@ -1,5 +1,7 @@ 1 7 3 +7 0 +7 1 xyzabc 2 1 3 1 2 0 0 diff --git a/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql b/tests/queries/0_stateless/01101_literal_column_clash.sql similarity index 85% rename from dbms/tests/queries/0_stateless/01101_literal_column_clash.sql rename to tests/queries/0_stateless/01101_literal_column_clash.sql index 3a2a71f126a..75d3ffa693c 100644 --- a/dbms/tests/queries/0_stateless/01101_literal_column_clash.sql +++ b/tests/queries/0_stateless/01101_literal_column_clash.sql @@ -5,6 +5,7 @@ join (select '1' as sid) as t2 on t2.sid = cast(t1.iid as String); -- even simpler cases select cast(7 as String), * from (select 3 "'String'"); +select cast(7 as String), * from (select number "'String'" FROM numbers(2)); SELECT concat('xyz', 'abc'), * FROM (SELECT 2 AS "'xyz'"); with 3 as "1" select 1, "1"; From 92692aff5e829e8b52d9acd3f751b1b415422776 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 21:03:29 +0300 Subject: [PATCH 240/743] Fix build. --- .../Algorithms/AggregatingSortedAlgorithm.cpp | 6 +++--- .../Merges/Algorithms/SummingSortedAlgorithm.cpp | 16 ++++++++-------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index ff9368ab21e..6247e016b5e 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -175,7 +175,7 @@ static void preprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::Col static void postprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::ColumnsDefinition & def) { size_t num_rows = chunk.getNumRows(); - auto columns_ = chunk.detachColumns(); + auto columns = chunk.detachColumns(); for (auto & desc : def.columns_to_simple_aggregate) { @@ -183,11 +183,11 @@ static void postprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::Co { auto & from_type = desc.nested_type; auto & to_type = desc.real_type; - columns_[desc.column_number] = recursiveTypeConversion(columns_[desc.column_number], from_type, to_type); + columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type); } } - chunk.setColumns(std::move(columns_), num_rows); + chunk.setColumns(std::move(columns), num_rows); } diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index d275ce51cba..c93e3cf221b 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -95,7 +95,7 @@ static bool isInPrimaryKey(const SortDescription & description, const std::strin /// Returns true if merge result is not empty static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, - Row & row, const ColumnRawPtrs & raw_columns, size_t row_num) + Row & row, const ColumnRawPtrs & raw_columns, size_t row_number) { /// Strongly non-optimal. @@ -103,10 +103,10 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row right(left.size()); for (size_t col_num : desc.key_col_nums) - right[col_num] = (*raw_columns[col_num])[row_num].template get(); + right[col_num] = (*raw_columns[col_num])[row_number].template get(); for (size_t col_num : desc.val_col_nums) - right[col_num] = (*raw_columns[col_num])[row_num].template get(); + right[col_num] = (*raw_columns[col_num])[row_number].template get(); auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & { @@ -162,19 +162,19 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, for (size_t col_num : desc.val_col_nums) row[col_num] = Array(merged.size()); - size_t row_num_ = 0; + size_t row_num = 0; for (const auto & key_value : merged) { for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.key_col_nums[col_num_index]].get()[row_num_] = key_value.first[col_num_index]; + row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.val_col_nums[col_num_index]].get()[row_num_] = key_value.second[col_num_index]; + row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; - ++row_num_; + ++row_num; } - return row_num_ != 0; + return row_num != 0; } static SummingSortedAlgorithm::ColumnsDefinition defineColumns( From 89dc40d4926c63eb9c36dc9c0a0186b8c9671c1b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 14 Apr 2020 21:05:10 +0300 Subject: [PATCH 241/743] Update test.py --- tests/integration/test_quorum_inserts/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index c85a1825b94..27901842692 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -57,7 +57,7 @@ def test_simple_add_replica(started_cluster): first.query("SYSTEM START FETCHES test_simple") - zero.query("SYSTEM SYNC REPLICA test_simple", timeout=20) + first.query("SYSTEM SYNC REPLICA test_simple", timeout=20) assert '1\t2011-01-01\n' == zero.query("SELECT * from test_simple") assert '1\t2011-01-01\n' == first.query("SELECT * from test_simple") From 80c27689ff7746edd00a88e5cac867bf5aa89863 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 21:22:14 +0300 Subject: [PATCH 242/743] Add debug output. --- src/Processors/Merges/IMergingTransform.h | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index b5e97efaca6..f9c2dba8271 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -82,21 +82,32 @@ public: algorithm.initialize(std::move(state.init_chunks)); if (state.input_chunk) + { + // std::cerr << "Consume chunk with " << state.input_chunk.getNumRows() + // << " for input " << state.next_input_to_read << std::endl; algorithm.consume(std::move(state.input_chunk), state.next_input_to_read); + } IMergingAlgorithm::Status status = algorithm.merge(); if (status.chunk && status.chunk.hasRows()) + { + // std::cerr << "Got chunk with " << status.chunk.getNumRows() << " rows" << std::endl; state.output_chunk = std::move(status.chunk); + } if (status.required_source >= 0) { + // std::cerr << "Required data for input " << status.required_source << std::endl; state.next_input_to_read = status.required_source; state.need_data = true; } if (status.is_finished) + { + // std::cerr << "Finished" << std::endl; state.is_finished = true; + } } protected: From 7def7b021e3fcaefc9acca5bd136a1707bd6c43c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 21:58:22 +0300 Subject: [PATCH 243/743] Fix test with vesrioned collapsing. --- .../Merges/Algorithms/VersionedCollapsingAlgorithm.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp index 508c4b43949..e039c383995 100644 --- a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp @@ -16,7 +16,8 @@ VersionedCollapsingAlgorithm::VersionedCollapsingAlgorithm( : IMergingAlgorithmWithSharedChunks( num_inputs, std::move(description_), out_row_sources_buf_, MAX_ROWS_IN_MULTIVERSION_QUEUE) , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) - , max_rows_in_queue(MAX_ROWS_IN_MULTIVERSION_QUEUE - 1) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer + /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer. 3 is a reasonable minimum size to collapse anything. + , max_rows_in_queue(std::min(std::max(3, max_block_size), MAX_ROWS_IN_MULTIVERSION_QUEUE) - 1) , current_keys(max_rows_in_queue) { sign_column_number = header.getPositionByName(sign_column_); From 15a8ac324516068a12a1e4a1b29d85ad0dd2108a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Apr 2020 22:31:49 +0300 Subject: [PATCH 244/743] Fix tests. --- src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index 6247e016b5e..63d15addf4f 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -322,7 +322,7 @@ IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge() if (merged_data.hasEnoughRows()) { last_key.reset(); - Status(merged_data.pull()); + return Status(merged_data.pull()); } merged_data.startGroup(current->all_columns, current->pos); From c454c5ae6210edcfc5ae9b7198977081b00edd87 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 15 Apr 2020 01:39:36 +0300 Subject: [PATCH 245/743] Trigger CI --- tests/queries/0_stateless/01098_msgpack_format.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01098_msgpack_format.sh b/tests/queries/0_stateless/01098_msgpack_format.sh index 233399570bb..4cb0731604d 100755 --- a/tests/queries/0_stateless/01098_msgpack_format.sh +++ b/tests/queries/0_stateless/01098_msgpack_format.sh @@ -8,7 +8,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS msgpack"; $CLICKHOUSE_CLIENT --query="CREATE TABLE msgpack (uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, int8 Int8, int16 Int16, int32 Int32, int64 Int64, float Float32, double Float64, string String, date Date, datetime DateTime, datetime64 DateTime64, array Array(UInt32)) ENGINE = Memory"; -$CLICKHOUSE_CLIENT --query="INSERT INTO msgpack VALUES (255, 65535, 4294967295, 100000000000, -128, -32768, -2147483648, -100000000000, 2.02, 10000.0000001, 'String', 18980, 1639872000, 1639872000000, [1,2,3,4,5]), (4, 1234, 3244467295, 500000000000, -1, -256, -14741221, -7000000000, 100.1, 14321.032141201, 'Another string', 20000, 1839882000, 1639872891123, [5,4,3,2,1]),(42, 42, 42, 42, 42, 42, 42, 42, 42.42, 42.42, '42', 42, 42, 42, [42])"; +$CLICKHOUSE_CLIENT --query="INSERT INTO msgpack VALUES (255, 65535, 4294967295, 100000000000, -128, -32768, -2147483648, -100000000000, 2.02, 10000.0000001, 'String', 18980, 1639872000, 1639872000000, [1,2,3,4,5]), (4, 1234, 3244467295, 500000000000, -1, -256, -14741221, -7000000000, 100.1, 14321.032141201, 'Another string', 20000, 1839882000, 1639872891123, [5,4,3,2,1]), (42, 42, 42, 42, 42, 42, 42, 42, 42.42, 42.42, '42', 42, 42, 42, [42])"; $CLICKHOUSE_CLIENT --query="SELECT * FROM msgpack FORMAT MsgPack" > $CURDIR/tmp_msgpac_test_all_types.msgpk; From 104c8877c383801e938f59672b2ed81b97dd7487 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Wed, 15 Apr 2020 01:54:35 +0300 Subject: [PATCH 246/743] small documentation fix --- .../external_dictionaries/external_dicts_dict_sources.md | 2 +- .../external_dictionaries/external_dicts_dict_sources.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md b/docs/en/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md index 7accdb5e77e..ca8aef24ea1 100644 --- a/docs/en/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md +++ b/docs/en/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md @@ -40,7 +40,7 @@ For source types [Executable file](#dicts-external_dicts_dict_sources-executable), [HTTP(s)](#dicts-external_dicts_dict_sources-http), [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) -optional format settings are available: +optional settings are available: ``` xml diff --git a/docs/ru/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md b/docs/ru/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md index 9949dca2e5d..102a7cc0e7f 100644 --- a/docs/ru/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md +++ b/docs/ru/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md @@ -35,7 +35,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration [Исполняемый файл](#dicts-external_dicts_dict_sources-executable), [HTTP(s)](#dicts-external_dicts_dict_sources-http), [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) -доступны дополнительные настройки форматирования: +доступны дополнительные настройки: ``` xml From 91011bc700ce9c9e1f20c1b1109b06e0d4fb9bd9 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 15 Apr 2020 03:04:12 +0400 Subject: [PATCH 247/743] bump --- .../test_quorum_inserts/configs/users.d/settings.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_quorum_inserts/configs/users.d/settings.xml b/tests/integration/test_quorum_inserts/configs/users.d/settings.xml index 0dbee66ff44..29b762656aa 100644 --- a/tests/integration/test_quorum_inserts/configs/users.d/settings.xml +++ b/tests/integration/test_quorum_inserts/configs/users.d/settings.xml @@ -4,9 +4,8 @@ 10000000000 0 - 2 1 - \ No newline at end of file + From b8061356b0cb443d77d404828b5c93e51cf16a58 Mon Sep 17 00:00:00 2001 From: Avogar Date: Wed, 15 Apr 2020 02:08:55 +0300 Subject: [PATCH 248/743] Update parsing method in MsgPackRowInputFormat --- .../Formats/Impl/MsgPackRowInputFormat.cpp | 39 ++++++++++--------- .../Formats/Impl/MsgPackRowInputFormat.h | 7 +++- 2 files changed, 26 insertions(+), 20 deletions(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 53c5a623a35..9b24978cf52 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -24,34 +24,35 @@ namespace ErrorCodes } MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) - : IRowInputFormat(header_, in_, std::move(params_)), buf(in_), data_types(header_.getDataTypes()) {} + : IRowInputFormat(header_, in_, std::move(params_)), buf(in), ctx(&reference_func, nullptr, msgpack::unpack_limit()), data_types(header_.getDataTypes()) {} + +int MsgPackRowInputFormat::unpack(msgpack::zone & zone, size_t & offset) +{ + offset = 0; + ctx.init(); + ctx.user().set_zone(zone); + return ctx.execute(buf.position(), buf.buffer().end() - buf.position(), offset); +} bool MsgPackRowInputFormat::readObject() { if (buf.eof()) return false; + PeekableReadBufferCheckpoint checkpoint{buf}; - size_t offset = 0; - bool need_more_data = true; - while (need_more_data) + std::unique_ptr zone(new msgpack::zone); + size_t offset; + while(!unpack(*zone, offset)) { - offset = 0; - try - { - object_handle = msgpack::unpack(buf.position(), buf.buffer().end() - buf.position(), offset); - need_more_data = false; - } - catch (msgpack::insufficient_bytes &) - { - buf.position() = buf.buffer().end(); - if (buf.eof()) - throw Exception("Unexpected end of file while parsing msgpack object.", ErrorCodes::INCORRECT_DATA); - buf.position() = buf.buffer().end(); - buf.makeContinuousMemoryFromCheckpointToPos(); - buf.rollbackToCheckpoint(); - } + buf.position() = buf.buffer().end(); + if (buf.eof()) + throw Exception("Unexpected end of file while parsing msgpack object.", ErrorCodes::INCORRECT_DATA); + buf.position() = buf.buffer().end(); + buf.makeContinuousMemoryFromCheckpointToPos(); + buf.rollbackToCheckpoint(); } buf.position() += offset; + object_handle = msgpack::object_handle(ctx.data(), std::move(zone)); return true; } diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index 8ed23a1e0f4..a426dc4950c 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -20,10 +20,15 @@ public: private: bool readObject(); void insertObject(IColumn & column, DataTypePtr type, const msgpack::object & object); + int unpack(msgpack::zone & zone, size_t & offset); + + // msgpack makes a copy of object by default, this function tells unpacker not to copy. + static bool reference_func(msgpack::type::object_type, size_t, void *) { return true; } PeekableReadBuffer buf; - DataTypes data_types; msgpack::object_handle object_handle; + msgpack::v1::detail::context ctx; + DataTypes data_types; }; } From 93c322bc30c7122c300b849a1c03c092eb3c1254 Mon Sep 17 00:00:00 2001 From: Avogar Date: Wed, 15 Apr 2020 02:58:36 +0300 Subject: [PATCH 249/743] Fix style error --- src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 9b24978cf52..b7da335f0c5 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -42,7 +42,7 @@ bool MsgPackRowInputFormat::readObject() PeekableReadBufferCheckpoint checkpoint{buf}; std::unique_ptr zone(new msgpack::zone); size_t offset; - while(!unpack(*zone, offset)) + while (!unpack(*zone, offset)) { buf.position() = buf.buffer().end(); if (buf.eof()) From 3d8562452d9818c9679374bc1cc65c29b66fca13 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 15 Apr 2020 13:07:16 +0300 Subject: [PATCH 250/743] remove more code --- src/Interpreters/RequiredSourceColumnsVisitor.cpp | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 08bfbf30f7f..9542b2882c5 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -159,19 +159,9 @@ void RequiredSourceColumnsMatcher::visit(const ASTFunction & node, const ASTPtr void RequiredSourceColumnsMatcher::visit(const ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data) { - ASTTableExpression * expr = nullptr; - ASTTableJoin * join = nullptr; - for (auto & child : node.children) - { - if (auto * e = child->as()) - expr = e; - if (auto * j = child->as()) - join = j; - } - - if (join) - data.has_table_join = true; + if (child->as()) + data.has_table_join = true; } /// ASTIdentifiers here are tables. Do not visit them as generic ones. From 01e495d78312358dcaaaa6d951fd998f17f76399 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 15 Apr 2020 13:55:34 +0300 Subject: [PATCH 251/743] dded more comments to AggregatingTransform. --- .../Transforms/AggregatingTransform.cpp | 3 +++ src/Processors/Transforms/AggregatingTransform.h | 15 +++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 4a76e0f2273..1a79955a9c7 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -413,6 +413,9 @@ AggregatingTransform::~AggregatingTransform() = default; IProcessor::Status AggregatingTransform::prepare() { + /// There are one or two input ports. + /// The first one is used at aggregation step, the second one - while reading merged data from ConvertingAggregated + auto & output = outputs.front(); /// Last output is current. All other outputs should already be closed. auto & input = inputs.back(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 469392e5840..9c1e9d4e2db 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -49,6 +49,21 @@ struct ManyAggregatedData using AggregatingTransformParamsPtr = std::shared_ptr; using ManyAggregatedDataPtr = std::shared_ptr; +/** Aggregates the stream of blocks using the specified key columns and aggregate functions. + * Columns with aggregate functions adds to the end of the block. + * If final = false, the aggregate functions are not finalized, that is, they are not replaced by their value, but contain an intermediate state of calculations. + * This is necessary so that aggregation can continue (for example, by combining streams of partially aggregated data). + * + * For every separate stream of data separate AggregatingTransform is created. + * Every AggregatingTransform reads data from the first port till is is not run out, or max_rows_to_group_by reached. + * When the last AggregatingTransform finish reading, the result of aggregation is needed to be merged together. + * This task is performed by ConvertingAggregatedToChunksTransform. + * Last AggregatingTransform expands pipeline and adds second input port, which reads from ConvertingAggregated. + * + * Aggregation data is passed by ManyAggregatedData structure, which is shared between all aggregating transforms. + * At aggregation step, every transform uses it's own AggregatedDataVariants structure. + * At merging step, all structures pass to ConvertingAggregatedToChunksTransform. + */ class AggregatingTransform : public IProcessor { public: From 43e178abfbb4046fa3e95655464202561a239ab2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 15 Apr 2020 14:04:38 +0300 Subject: [PATCH 252/743] Revert "dded more comments to AggregatingTransform." This reverts commit 01e495d78312358dcaaaa6d951fd998f17f76399. --- .../Transforms/AggregatingTransform.cpp | 3 --- src/Processors/Transforms/AggregatingTransform.h | 15 --------------- 2 files changed, 18 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 1a79955a9c7..4a76e0f2273 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -413,9 +413,6 @@ AggregatingTransform::~AggregatingTransform() = default; IProcessor::Status AggregatingTransform::prepare() { - /// There are one or two input ports. - /// The first one is used at aggregation step, the second one - while reading merged data from ConvertingAggregated - auto & output = outputs.front(); /// Last output is current. All other outputs should already be closed. auto & input = inputs.back(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 9c1e9d4e2db..469392e5840 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -49,21 +49,6 @@ struct ManyAggregatedData using AggregatingTransformParamsPtr = std::shared_ptr; using ManyAggregatedDataPtr = std::shared_ptr; -/** Aggregates the stream of blocks using the specified key columns and aggregate functions. - * Columns with aggregate functions adds to the end of the block. - * If final = false, the aggregate functions are not finalized, that is, they are not replaced by their value, but contain an intermediate state of calculations. - * This is necessary so that aggregation can continue (for example, by combining streams of partially aggregated data). - * - * For every separate stream of data separate AggregatingTransform is created. - * Every AggregatingTransform reads data from the first port till is is not run out, or max_rows_to_group_by reached. - * When the last AggregatingTransform finish reading, the result of aggregation is needed to be merged together. - * This task is performed by ConvertingAggregatedToChunksTransform. - * Last AggregatingTransform expands pipeline and adds second input port, which reads from ConvertingAggregated. - * - * Aggregation data is passed by ManyAggregatedData structure, which is shared between all aggregating transforms. - * At aggregation step, every transform uses it's own AggregatedDataVariants structure. - * At merging step, all structures pass to ConvertingAggregatedToChunksTransform. - */ class AggregatingTransform : public IProcessor { public: From d15d6e16610fb683c1df7200d880a9eaea470fc5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 15 Apr 2020 17:10:33 +0300 Subject: [PATCH 253/743] review fixes --- src/Columns/IColumn.cpp | 5 ----- src/Columns/IColumn.h | 1 - src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 3 +-- 4 files changed, 2 insertions(+), 9 deletions(-) diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index df56823b6aa..a3ed0885651 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -9,11 +9,6 @@ namespace DB { -Field IColumn::get(size_t n) const -{ - return (*this)[n]; -} - String IColumn::dumpStructure() const { WriteBufferFromOwnString res; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index aa9455fc6cc..090537d6770 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -70,7 +70,6 @@ public: /// Returns value of n-th element in universal Field representation. /// Is used in rare cases, since creation of Field instance is expensive usually. virtual Field operator[](size_t n) const = 0; - Field get(size_t n) const; /// Like the previous one, but avoids extra copying if Field is in a container, for example. virtual void get(size_t n, Field & res) const = 0; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 6bb0589b4b3..b810182eb48 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -617,7 +617,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, && existing_column->column && isColumnConst(*existing_column->column) && existing_column->column->size() == 1 - && existing_column->column->get(0) == value) + && existing_column->column->operator[](0) == value) { const_cast(literal).unique_column_name = default_name; } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index af11098b883..bb33aca7655 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -875,8 +875,7 @@ void SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & * names and identifiers for columns. This code is a workaround for * a particular subclass of problems, and not a proper solution. */ - if (auto as_literal = ast->as(); - as_literal) + if (auto as_literal = ast->as()) { source_name = as_literal->unique_column_name; assert(!source_name.empty()); From 16537bbac873218715086ab898656701c648e7e3 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 15 Apr 2020 17:59:44 +0300 Subject: [PATCH 254/743] Update release.sh --- docs/tools/release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/release.sh b/docs/tools/release.sh index faca0e8ec17..8c61d16966f 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -44,7 +44,7 @@ then if [[ ! -z "${CLOUDFLARE_TOKEN}" ]] then sleep 1m - git diff --stat="9999,9999" --diff-filter=M HEAD~1 | grep '|' | awk '$1 ~ /\.html$/ { if ($3>4) { url="https://'${BASE_DOMAIN}'/"$1; sub(/\/index.html/, "/", url); print "\""url"\""; }}' | split -l 25 /dev/stdin PURGE + git diff --stat="9999,9999" --diff-filter=M HEAD~1 | grep '|' | awk '$1 ~ /\.html$/ { if ($3>4) { url="https://clickhouse.tech/"$1; sub(/\/index.html/, "/", url); print "\""url"\""; }}' | split -l 25 /dev/stdin PURGE for FILENAME in $(ls PURGE*) do POST_DATA=$(cat "${FILENAME}" | sed -n -e 'H;${x;s/\n/,/g;s/^,//;p;}' | awk '{print "{\"files\":["$0"]}";}') From 4d5b170480ff7413570d4b6bb73acd494cecfccf Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Wed, 15 Apr 2020 23:53:15 +0800 Subject: [PATCH 255/743] update documentation-zh (#10260) * update documentation-zh * Update index.md * Update metrica.md * Update playground.md * Update adopters.md Co-authored-by: Ivan Blinkov --- .../example_datasets/amplab_benchmark.md | 2 +- .../getting_started/example_datasets/index.md | 14 ++++----- .../example_datasets/metrica.md | 4 +-- .../example_datasets/star_schema.md | 2 +- docs/zh/getting_started/install.md | 1 + docs/zh/getting_started/playground.md | 29 +++++++++---------- docs/zh/getting_started/tutorial.md | 2 +- docs/zh/introduction/adopters.md | 6 ++-- docs/zh/introduction/distinctive_features.md | 4 +-- 9 files changed, 29 insertions(+), 35 deletions(-) diff --git a/docs/zh/getting_started/example_datasets/amplab_benchmark.md b/docs/zh/getting_started/example_datasets/amplab_benchmark.md index 4c3b26819b1..30d55f8b28d 100644 --- a/docs/zh/getting_started/example_datasets/amplab_benchmark.md +++ b/docs/zh/getting_started/example_datasets/amplab_benchmark.md @@ -1,5 +1,5 @@ -# AMPLab 大数据基准测试 {#amplab-da-shu-ju-ji-zhun-ce-shi} +# AMPLab大数据基准测试 {#amplab-da-shu-ju-ji-zhun-ce-shi} 参考 https://amplab.cs.berkeley.edu/benchmark/ diff --git a/docs/zh/getting_started/example_datasets/index.md b/docs/zh/getting_started/example_datasets/index.md index 4faf3b0ecfc..c610af8a269 100644 --- a/docs/zh/getting_started/example_datasets/index.md +++ b/docs/zh/getting_started/example_datasets/index.md @@ -1,6 +1,4 @@ --- -machine_translated: true -machine_translated_rev: b111334d6614a02564cf32f379679e9ff970d9b1 toc_folder_title: "\u793A\u4F8B\u6570\u636E\u96C6" toc_priority: 12 toc_title: "\u5BFC\u8A00" @@ -11,12 +9,12 @@ toc_title: "\u5BFC\u8A00" 本节介绍如何获取示例数据集并将其导入ClickHouse。 对于某些数据集示例查询也可用。 -- [匿名Yandex的。梅里卡数据集](metrica.md) -- [星型架构基准测试](star_schema.md) -- [WikiStat](wikistat.md) -- [来自Criteo的万兆字节点击日志](criteo.md) -- [AMPLab大数据基准](amplab_benchmark.md) +- [脱敏的Yandex.Metrica数据集](metrica.md) +- [星型基准测试](star_schema.md) +- [维基访问数据](wikistat.md) +- [Criteo TB级别点击日志](criteo.md) +- [AMPLab大数据基准测试](amplab_benchmark.md) - [纽约出租车数据](nyc_taxi.md) -- [时间](ontime.md) +- [航班飞行数据](ontime.md) [原始文章](https://clickhouse.tech/docs/en/getting_started/example_datasets) diff --git a/docs/zh/getting_started/example_datasets/metrica.md b/docs/zh/getting_started/example_datasets/metrica.md index f7e0c86d324..6e349a1135d 100644 --- a/docs/zh/getting_started/example_datasets/metrica.md +++ b/docs/zh/getting_started/example_datasets/metrica.md @@ -1,11 +1,9 @@ --- -machine_translated: true -machine_translated_rev: b111334d6614a02564cf32f379679e9ff970d9b1 toc_priority: 21 toc_title: "Yandex\u6885\u7279\u91CC\u5361\u6570\u636E" --- -# 匿名Yandex的。梅特里卡数据 {#anonymized-yandex-metrica-data} +# 脱敏的Yandex.Metrica数据集 {#anonymized-yandex-metrica-data} Dataset由两个表组成,其中包含有关命中的匿名数据 (`hits_v1`)和访问 (`visits_v1`)的Yandex的。梅特里卡 你可以阅读更多关于Yandex的。梅特里卡 [ClickHouse历史](../../introduction/history.md) 科。 diff --git a/docs/zh/getting_started/example_datasets/star_schema.md b/docs/zh/getting_started/example_datasets/star_schema.md index 4fed13923ff..b575abe63fa 100644 --- a/docs/zh/getting_started/example_datasets/star_schema.md +++ b/docs/zh/getting_started/example_datasets/star_schema.md @@ -1,5 +1,5 @@ -# 星型架构基准测试 {#star-schema-benchmark} +# 星型基准测试 {#star-schema-benchmark} 编译 dbgen: diff --git a/docs/zh/getting_started/install.md b/docs/zh/getting_started/install.md index 6a9aae286ad..111c362caf7 100644 --- a/docs/zh/getting_started/install.md +++ b/docs/zh/getting_started/install.md @@ -1,3 +1,4 @@ +# 安装 {#clickhouse-an-zhuang} ## 系统要求 {#xi-tong-yao-qiu} diff --git a/docs/zh/getting_started/playground.md b/docs/zh/getting_started/playground.md index a09d615ba21..252b2eae611 100644 --- a/docs/zh/getting_started/playground.md +++ b/docs/zh/getting_started/playground.md @@ -1,14 +1,12 @@ --- -machine_translated: true -machine_translated_rev: b111334d6614a02564cf32f379679e9ff970d9b1 toc_priority: 14 toc_title: "\u266A\u64CD\u573A\u266A" --- -# ツ环板Playgroundョツ嘉ッ {#clickhouse-playground} +# ClickHouse体验平台 {#clickhouse-playground} -[ツ环板Playgroundョツ嘉ッ](https://play.clickhouse.tech?file=welcome) 允许人们通过即时运行查询来尝试ClickHouse,而无需设置他们的服务器或集群。 -Playground中提供了几个示例数据集以及显示ClickHouse要素的示例查询。 +[ClickHouse体验平台](https://play.clickhouse.tech?file=welcome) 允许人们通过即时运行查询来尝试ClickHouse,而无需设置他们的服务器或集群。 +体验平台中提供了几个示例数据集以及显示ClickHouse特性的示例查询。 查询以只读用户身份执行。 这意味着一些局限性: @@ -21,25 +19,26 @@ Playground中提供了几个示例数据集以及显示ClickHouse要素的示例 - [`result_overflow_mode=break`](../operations/settings/query_complexity/#result-overflow-mode) - [`max_execution_time=60000`](../operations/settings/query_complexity/#max-execution-time) -ClickHouse游乐场给m2的经验。小 -[管理服务ClickHouse](https://cloud.yandex.com/services/managed-clickhouse) -实例托管在 [Yandex云](https://cloud.yandex.com/). +ClickHouse体验还有如下: +[ClickHouse管理服务](https://cloud.yandex.com/services/managed-clickhouse) +实例托管 [Yandex云](https://cloud.yandex.com/). 更多信息 [云提供商](../commercial/cloud.md). -ClickHouse游乐场网的界面使请求通过ClickHouse [HTTP API](../interfaces/http.md). -Playground后端只是一个ClickHouse集群,没有任何额外的服务器端应用程序。 -隆隆隆隆路虏脢..陇.貌.垄拢卢虏禄.陇.貌路.隆拢脳枚脢虏 +ClickHouse体验平台界面实际上是通过ClickHouse [HTTP API](../interfaces/http.md)接口实现的. +体验平台后端只是一个ClickHouse集群,没有任何额外的服务器端应用程序。 +体验平台也同样提供了ClickHouse HTTPS服务端口。 -您可以使用任何HTTP客户端向playground进行查询,例如 [卷曲的](https://curl.haxx.se) 或 [wget](https://www.gnu.org/software/wget/),或使用以下方式建立连接 [JDBC](../interfaces/jdbc.md) 或 [ODBC](../interfaces/odbc.md) 司机 + +您可以使用任何HTTP客户端向体验平台进行查询,例如 [curl](https://curl.haxx.se) 或 [wget](https://www.gnu.org/software/wget/),或使用以下方式建立连接 [JDBC](../interfaces/jdbc.md) 或 [ODBC](../interfaces/odbc.md) 司机 有关支持ClickHouse的软件产品的更多信息,请访问 [这里](../interfaces/index.md). -| 参数 | 价值 | +| 参数 | 值 | |:-----|:--------------------------------------| -| 端点 | https://play-api.克莱克豪斯技术:8443 | +| 服务端口 | https://play-api.clickhouse.tech:8443 | | 用户 | `playground` | | 密码 | `clickhouse` | -请注意,此端点需要安全连接。 +请注意,此服务端口需要安全连接。 示例: diff --git a/docs/zh/getting_started/tutorial.md b/docs/zh/getting_started/tutorial.md index 4fc2fbdf290..bd86e19a037 100644 --- a/docs/zh/getting_started/tutorial.md +++ b/docs/zh/getting_started/tutorial.md @@ -5,7 +5,7 @@ toc_priority: 12 toc_title: "\u6559\u7A0B" --- -# 点击教程 {#clickhouse-tutorial} +# 教程 {#clickhouse-tutorial} ## 从本教程中可以期待什么? {#what-to-expect-from-this-tutorial} diff --git a/docs/zh/introduction/adopters.md b/docs/zh/introduction/adopters.md index 8a69e67264e..f550fd6e473 100644 --- a/docs/zh/introduction/adopters.md +++ b/docs/zh/introduction/adopters.md @@ -1,14 +1,12 @@ --- -machine_translated: true -machine_translated_rev: b111334d6614a02564cf32f379679e9ff970d9b1 toc_priority: 8 toc_title: "\u91C7\u7528\u8005" --- -# ツ环板tersョツ嘉ッツ偲 {#clickhouse-adopters} +# ClickHouse用户 {#clickhouse-adopters} !!! warning "免责声明" - 使用ClickHouse的公司和他们的成功故事下面的名单是从公共来源组装,因此可能不同于当前的现实. 如果您分享您公司采用ClickHouse的故事,我们将不胜感激 [将其添加到列表](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/introduction/adopters.md),但请确保你不会有任何保密协议的问题,这样做。 提供来自其他公司的出版物的更新也很有用。 + 如下使用ClickHouse的公司和他们的成功案例来源于公开资源,因此和实际情况可能有所出入。如果您分享您公司使用ClickHouse的故事,我们将不胜感激 [将其添加到列表](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/introduction/adopters.md),但请确保你这样做不会有任何保密协议的问题。也欢迎提供来自其他公司的出版物的更新。 | 公司简介 | 行业 | 用例 | 群集大小 | (Un)压缩数据大小\* | 参考资料 | |-----------------------------------------------------------------|-------------------|----------------|---------------------------------------------------|----------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| diff --git a/docs/zh/introduction/distinctive_features.md b/docs/zh/introduction/distinctive_features.md index a267a49bf8a..e27f3317bf0 100644 --- a/docs/zh/introduction/distinctive_features.md +++ b/docs/zh/introduction/distinctive_features.md @@ -1,5 +1,5 @@ -# ClickHouse的独特功能 {#clickhousede-du-te-gong-neng} +# ClickHouse的特性 {#clickhouse-de-te-xing} ## 真正的列式数据库管理系统 {#zhen-zheng-de-lie-shi-shu-ju-ku-guan-li-xi-tong} @@ -62,7 +62,7 @@ ClickHouse使用异步的多主复制技术。当数据被写入任何一个可 更多信息,参见 [数据复制](../engines/table_engines/mergetree_family/replication.md)。 -# 的限制 {#clickhouseke-yi-ren-wei-shi-que-dian-de-gong-neng} +# 限制 {#clickhouseke-xian-zhi} 1. 没有完整的事务支持。 2. 缺少高频率,低延迟的修改或删除已存在数据的能力。仅能用于批量删除或修改数据,但这符合 [GDPR](https://gdpr-info.eu)。 From 7974fcb9d7c033734a5a171d09b61782ab3a7e7a Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 15 Apr 2020 19:08:16 +0300 Subject: [PATCH 256/743] CLICKHOUSEDOCS-582: Column timestamp_ns in system.trace_log (#10287) Co-authored-by: Sergei Shtykov --- docs/en/operations/system_tables.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 8905ca14569..22e688c59cd 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -709,7 +709,7 @@ When the table is deleted manually, it will be automatically created on the fly. You can specify an arbitrary partitioning key for the `system.query_thread_log` table in the [query\_thread\_log](server_configuration_parameters/settings.md#server_configuration_parameters-query-thread-log) server setting (see the `partition_by` parameter). -## system.trace\_log {#system_tables-trace_log} +## system.trace_log {#system_tables-trace_log} Contains stack traces collected by the sampling query profiler. @@ -719,24 +719,26 @@ To analyze logs, use the `addressToLine`, `addressToSymbol` and `demangle` intro Columns: -- `event_date`([Date](../sql_reference/data_types/date.md)) — Date of sampling moment. +- `event_date` ([Date](../sql_reference/data_types/date.md)) — Date of sampling moment. -- `event_time`([DateTime](../sql_reference/data_types/datetime.md)) — Timestamp of sampling moment. +- `event_time` ([DateTime](../sql_reference/data_types/datetime.md)) — Timestamp of the sampling moment. -- `revision`([UInt32](../sql_reference/data_types/int_uint.md)) — ClickHouse server build revision. +- `timestamp_ns` ([UInt64](../sql_reference/data_types/int_uint.md)) — Timestamp of the sampling moment in nanoseconds. + +- `revision` ([UInt32](../sql_reference/data_types/int_uint.md)) — ClickHouse server build revision. When connecting to server by `clickhouse-client`, you see the string similar to `Connected to ClickHouse server version 19.18.1 revision 54429.`. This field contains the `revision`, but not the `version` of a server. -- `timer_type`([Enum8](../sql_reference/data_types/enum.md)) — Timer type: +- `timer_type` ([Enum8](../sql_reference/data_types/enum.md)) — Timer type: - `Real` represents wall-clock time. - `CPU` represents CPU time. -- `thread_number`([UInt32](../sql_reference/data_types/int_uint.md)) — Thread identifier. +- `thread_number` ([UInt32](../sql_reference/data_types/int_uint.md)) — Thread identifier. -- `query_id`([String](../sql_reference/data_types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) system table. +- `query_id` ([String](../sql_reference/data_types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query\_log](#system_tables-query_log) system table. -- `trace`([Array(UInt64)](../sql_reference/data_types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. +- `trace` ([Array(UInt64)](../sql_reference/data_types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. **Example** From 41489e57ac063df7d416b7b2a72c9ac47b654ebe Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 15 Apr 2020 20:25:26 +0300 Subject: [PATCH 257/743] Update CHANGELOG.md --- CHANGELOG.md | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5331fa0ec98..e01b36ecb00 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,8 +2,17 @@ ### ClickHouse release origin/20.3 FIXME as compared to v20.3.5.21-stable +#### New Feature + +* Added function `isConstant`. This function checks whether its argument is constant expression and returns 1 or 0. It is intended for development, debugging and demonstration purposes. [#10198](https://github.com/ClickHouse/ClickHouse/pull/10198) ([alexey-milovidov](https://github.com/alexey-milovidov)). + #### Bug Fix +* Fix error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug where ClickHouse would throw "Unknown function lambda." error message when user tries to run ALTER UPDATE/DELETE on tables with ENGINE = Replicated*. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). +* Fix `'Cannot add column'` error while creating `range_hashed` dictionary using DDL query. Fixes [#10093](https://github.com/ClickHouse/ClickHouse/issues/10093). [#10235](https://github.com/ClickHouse/ClickHouse/pull/10235) ([alesapin](https://github.com/alesapin)). +* Fixed "generateRandom" function for Date type. This fixes [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). Fix an edge case when dates with year 2106 are inserted to MergeTree tables with old-style partitioning but partitions are named with year 1970. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Convert types if the table definition of a View does not correspond to the SELECT query. This fixes [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022). [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -25,12 +34,13 @@ * Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965). [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). * Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix a bug with ON CLUSTER DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). +* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). * Fix parsing multiple hosts set in the CREATE USER command, e.g. `CREATE USER user6 HOST NAME REGEXP 'lo.?*host', NAME REGEXP 'lo*host'`. [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). * Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). * Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). * Fix error with qualified names in `distributed_product_mode='local'`. Fixes [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756). [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)). +* For INSERT queries shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send INSERT queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix calculating grants for introspection functions from the setting 'allow_introspection_functions'. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)). #### Build/Testing/Packaging Improvement From f08cdfcc4c7628011d4a529647b04e1e74338efc Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 15 Apr 2020 20:47:08 +0300 Subject: [PATCH 258/743] fix count(*) with multiple_joins_rewriter_version = 1 --- .../JoinToSubqueryTransformVisitor.cpp | 25 ++++++++++++++-- .../01116_cross_count_asterisks.reference | 4 +++ .../01116_cross_count_asterisks.sql | 29 +++++++++++++++++++ 3 files changed, 56 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01116_cross_count_asterisks.reference create mode 100644 tests/queries/0_stateless/01116_cross_count_asterisks.sql diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index ca21a53b5b0..331c364c5fa 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -129,6 +130,8 @@ private: /// Make aliases maps (alias -> column_name, column_name -> alias) struct ColumnAliasesMatcher { + using Visitor = ConstInDepthNodeVisitor; + struct Data { const std::vector tables; @@ -137,6 +140,7 @@ struct ColumnAliasesMatcher std::unordered_map aliases; /// alias -> long_name std::vector> compound_identifiers; std::set allowed_long_names; /// original names allowed as aliases '--t.x as t.x' (select expressions only). + bool inside_function = false; explicit Data(const std::vector && tables_) : tables(tables_) @@ -192,6 +196,10 @@ struct ColumnAliasesMatcher static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { + /// Do not go into subqueries. Function visits children itself. + if (node->as() || + node->as()) + return false; return !node->as(); } @@ -199,11 +207,24 @@ struct ColumnAliasesMatcher { if (auto * t = ast->as()) visit(*t, ast, data); + else if (auto * f = ast->as()) + visit(*f, ast, data); - if (ast->as() || ast->as()) + /// Do not allow asterisks but ignore them inside functions. I.e. allow 'count(*)'. + if (!data.inside_function && (ast->as() || ast->as())) throw Exception("Multiple JOIN do not support asterisks for complex queries yet", ErrorCodes::NOT_IMPLEMENTED); } + static void visit(const ASTFunction &, const ASTPtr & ast, Data & data) + { + /// Grandchild case: Function -> (ExpressionList) -> Asterisk + data.inside_function = true; + Visitor visitor(data); + for (auto & child : ast->children) + visitor.visit(child); + data.inside_function = false; + } + static void visit(const ASTIdentifier & const_node, const ASTPtr &, Data & data) { ASTIdentifier & node = const_cast(const_node); /// we know it's not const @@ -348,7 +369,7 @@ bool needRewrite(ASTSelectQuery & select, std::vector; using RewriteVisitor = InDepthNodeVisitor; using ExtractAsterisksVisitor = ConstInDepthNodeVisitor; -using ColumnAliasesVisitor = ConstInDepthNodeVisitor; +using ColumnAliasesVisitor = ColumnAliasesMatcher::Visitor; using AppendSemanticMatcher = OneTypeMatcher; using AppendSemanticVisitor = InDepthNodeVisitor; diff --git a/tests/queries/0_stateless/01116_cross_count_asterisks.reference b/tests/queries/0_stateless/01116_cross_count_asterisks.reference new file mode 100644 index 00000000000..8347b144a35 --- /dev/null +++ b/tests/queries/0_stateless/01116_cross_count_asterisks.reference @@ -0,0 +1,4 @@ +2 +1 +2 +1 diff --git a/tests/queries/0_stateless/01116_cross_count_asterisks.sql b/tests/queries/0_stateless/01116_cross_count_asterisks.sql new file mode 100644 index 00000000000..1fb8b0b0e66 --- /dev/null +++ b/tests/queries/0_stateless/01116_cross_count_asterisks.sql @@ -0,0 +1,29 @@ +SET multiple_joins_rewriter_version = 2; + +SELECT count(*) +FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 +WHERE (n1.number = n2.number) AND (n2.number = n3.number); + +SELECT count(*) c FROM ( + SELECT count(*), count(*) as c + FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 + WHERE (n1.number = n2.number) AND (n2.number = n3.number) + AND (SELECT count(*) FROM numbers(1)) = 1 +) +WHERE (SELECT count(*) FROM numbers(2)) = 2 +HAVING c IN(SELECT count(*) c FROM numbers(1)); + +SET multiple_joins_rewriter_version = 1; + +SELECT count(*) +FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 +WHERE (n1.number = n2.number) AND (n2.number = n3.number); + +SELECT count(*) c FROM ( + SELECT count(*), count(*) as c + FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 + WHERE (n1.number = n2.number) AND (n2.number = n3.number) + AND (SELECT count(*) FROM numbers(1)) = 1 +) +WHERE (SELECT count(*) FROM numbers(2)) = 2 +HAVING c IN(SELECT count(*) c FROM numbers(1)); From 56e4089f8d23d64516fac05981ca61ce47df3eec Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 15 Apr 2020 20:50:41 +0300 Subject: [PATCH 259/743] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e01b36ecb00..a865f351af6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -40,7 +40,6 @@ * Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). * Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). * Fix error with qualified names in `distributed_product_mode='local'`. Fixes [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756). [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)). -* For INSERT queries shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send INSERT queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix calculating grants for introspection functions from the setting 'allow_introspection_functions'. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)). #### Build/Testing/Packaging Improvement @@ -55,6 +54,7 @@ * Fix 'Different expressions with the same alias' error when query has PREWHERE and WHERE on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)). * Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)). +* For INSERT queries shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send INSERT queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix 'COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query' error in case of subqueries with COMMA JOIN out of tables lists (i.e. in WHERE). Fixes [#9782](https://github.com/ClickHouse/ClickHouse/issues/9782). [#9830](https://github.com/ClickHouse/ClickHouse/pull/9830) ([Artem Zuikov](https://github.com/4ertus2)). * Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix SIGSEGV with optimize_skip_unused_shards when type cannot be converted. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)). From 9bf8f00c3d98cf390a0a2b66cd8ed443a7e24446 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Apr 2020 20:52:05 +0300 Subject: [PATCH 260/743] Fix test --- .../test_dictionaries_mysql/test.py | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 80424a3471a..647e36c71b3 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -23,7 +23,7 @@ create_table_mysql_template = """ """ create_clickhouse_dictionary_table_template = """ - CREATE TABLE IF NOT EXISTS `test`.`dict_table_{}` (`id` Int32, `value` String) ENGINE = Dictionary({}) + CREATE TABLE IF NOT EXISTS `test`.`dict_table_{}` (`id` UInt64, `value` String) ENGINE = Dictionary({}) """ @pytest.fixture(scope="module") @@ -31,18 +31,18 @@ def started_cluster(): try: #time.sleep(30) cluster.start() - + # Create a MySQL database mysql_connection = get_mysql_conn() create_mysql_db(mysql_connection, 'test') mysql_connection.close() - + # Create database in ClickHouse instance.query("CREATE DATABASE IF NOT EXISTS test") - + # Create database in ClickChouse using MySQL protocol (will be used for data insertion) instance.query("CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql1:3306', 'test', 'root', 'clickhouse')") - + yield cluster finally: @@ -53,11 +53,11 @@ def test_load_mysql_dictionaries(started_cluster): # Load dictionaries query = instance.query query("SYSTEM RELOAD DICTIONARIES") - + for n in range(0, 5): # Create MySQL tables, fill them and create CH dict tables prepare_mysql_table('test', str(n)) - + # Check dictionaries are loaded and have correct number of elements for n in range(0, 100): # Force reload of dictionaries (each 10 iteration) @@ -73,7 +73,7 @@ def create_mysql_db(mysql_connection, name): def prepare_mysql_table(table_name, index): mysql_connection = get_mysql_conn() - + # Create table create_mysql_table(mysql_connection, table_name + str(index)) @@ -82,8 +82,8 @@ def prepare_mysql_table(table_name, index): query("INSERT INTO `clickhouse_mysql`.{}(id, value) select number, concat('{} value ', toString(number)) from numbers(10000) ".format(table_name + str(index), table_name + str(index))) assert query("SELECT count() FROM `clickhouse_mysql`.{}".format(table_name + str(index))).rstrip() == '10000' mysql_connection.close() - - #Create CH Dictionary tables based on MySQL tables + + #Create CH Dictionary tables based on MySQL tables query(create_clickhouse_dictionary_table_template.format(table_name + str(index), 'dict' + str(index))) def get_mysql_conn(): From d0eb40e4ca6d543049b18d1c14c411fcb81877a2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 15 Apr 2020 22:03:16 +0300 Subject: [PATCH 261/743] simple backport script --- utils/simple-backport/backport.sh | 56 +++++++++++++++++++++--------- utils/simple-backport/changelog.sh | 21 ++++++----- 2 files changed, 50 insertions(+), 27 deletions(-) diff --git a/utils/simple-backport/backport.sh b/utils/simple-backport/backport.sh index 7d05f6902d0..7d5b12d6f7f 100755 --- a/utils/simple-backport/backport.sh +++ b/utils/simple-backport/backport.sh @@ -3,12 +3,29 @@ set -e branch="$1" merge_base=$(git merge-base origin/master "origin/$branch") +master_git_cmd=(git log "$merge_base..origin/master" --first-parent) +branch_git_cmd=(git log "$merge_base..origin/$branch" --first-parent) # Make lists of PRs that were merged into each branch. Use first parent here, or else # we'll get weird things like seeing older master that was merged into a PR branch # that was then merged into master. -git log "$merge_base..origin/master" --first-parent > master-log.txt -git log "$merge_base..origin/$branch" --first-parent > "$branch-log.txt" +"${master_git_cmd[@]}" > master-log.txt +"${branch_git_cmd[@]}" > "$branch-log.txt" + +# Check for diamond merges. +"${master_git_cmd[@]}" --oneline --grep "Merge branch '" | grep '' +diamonds_in_master=$? + +"${branch_git_cmd[@]}" --oneline --grep "Merge branch '" | grep '' +diamonds_in_branch=$? + +if [ "$diamonds_in_master" -eq 0 ] || [ "$diamonds_in_branch" -eq 0 ] +then + # DO NOT ADD automated handling of diamond merges to this script. + # It is an unsustainable way to work with git, and it MUST be visible. + echo Warning: suspected diamond merges above. + echo Some commits will be missed, review these manually. +fi # NOTE keep in sync with ./changelog.sh. # Search for PR numbers in commit messages. First variant is normal merge, and second @@ -24,26 +41,32 @@ find_prs=(sed -n "s/^.*Merge pull request #\([[:digit:]]\+\).*$/\1/p; # Find all master PRs that are not in branch by calculating differences of two PR lists. grep -f "$branch-prs.txt" -F -x -v master-prs.txt > "$branch-diff-prs.txt" -rm "$branch-report.tsv" ||: - echo "$(wc -l < "$branch-diff-prs".txt) PRs differ between $branch and master." +function github_download() +{ + local url=${1} + local file=${2} + if ! [ -f "$file" ] + then + if ! curl -H "Authorization: token $GITHUB_TOKEN" \ + -sSf "$url" \ + > "$file" + then + >&2 echo "Failed to download '$url' to '$file'. Contents: '$(cat "$file")'." + rm "$file" + return 1 + fi + sleep 0.1 + fi +} + +rm "$branch-report.tsv" &> /dev/null ||: for pr in $(cat "$branch-diff-prs.txt") do # Download PR info from github. file="pr$pr.json" - if ! [ -f "$file" ] - then - if ! curl -H "Authorization: token $GITHUB_TOKEN" \ - -sSf "https://api.github.com/repos/ClickHouse/ClickHouse/pulls/$pr" \ - > "$file" - then - >&2 cat "$file" - rm "$file" - break - fi - sleep 0.1 - fi + github_download "https://api.github.com/repos/ClickHouse/ClickHouse/pulls/$pr" "$file" || continue if ! [ "$pr" == "$(jq -r .number "$file")" ] then @@ -82,3 +105,4 @@ do fi done +echo "Done." diff --git a/utils/simple-backport/changelog.sh b/utils/simple-backport/changelog.sh index d5a4e68900a..75a54a50b92 100755 --- a/utils/simple-backport/changelog.sh +++ b/utils/simple-backport/changelog.sh @@ -7,6 +7,15 @@ log_command=(git log "$from..$to" --first-parent) "${log_command[@]}" > "changelog-log.txt" +# Check for diamond merges. +if "${log_command[@]}" --oneline --grep "Merge branch '" | grep '' +then + # DO NOT ADD automated handling of diamond merges to this script. + # It is an unsustainable way to work with git, and it MUST be visible. + echo Warning: suspected diamond merges above. + echo Some commits will be missed, review these manually. +fi + # NOTE keep in sync with ./backport.sh. # Search for PR numbers in commit messages. First variant is normal merge, and second # variant is squashed. Next are some backport message variants. @@ -19,14 +28,6 @@ find_prs=(sed -n "s/^.*Merge pull request #\([[:digit:]]\+\).*$/\1/p; echo "$(wc -l < "changelog-prs.txt") PRs added between $from and $to." -if "${log_command[@]}" --oneline --grep "Merge branch '" | grep '' -then - # DO NOT ADD automated handling of diamond merges to this script. - # It is an unsustainable way to work with git, and it MUST be visible. - echo Warning: suspected diamond merges above. - echo Some commits will be missed, review these manually. -fi - function github_download() { local url=${1} @@ -37,9 +38,7 @@ function github_download() -sSf "$url" \ > "$file" then - >&2 echo "Failed to download '$url' to '$file'. Contents: '" - >&2 cat "$file" - >&2 echo "'." + >&2 echo "Failed to download '$url' to '$file'. Contents: '$(cat "$file")'." rm "$file" return 1 fi From dc152c8a7499d9ce1b5520684784bdb5ce2b491a Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 15 Apr 2020 22:07:50 +0300 Subject: [PATCH 262/743] Update parse_engine_file.xml --- tests/performance/parse_engine_file.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/parse_engine_file.xml b/tests/performance/parse_engine_file.xml index c96f4e537ff..a88945125b3 100644 --- a/tests/performance/parse_engine_file.xml +++ b/tests/performance/parse_engine_file.xml @@ -43,7 +43,7 @@ test.hits -SELECT * FROM table_{format} +SELECT * FROM table_{format} FORMAT Null DROP TABLE IF EXISTS table_{format} From f4d80e81c3ebaefeca28bddbbf056dd13f0bfa6e Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 15 Apr 2020 23:18:24 +0300 Subject: [PATCH 263/743] Remove extra slashes --- docs/en/development/browse_code.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/browse_code.md b/docs/en/development/browse_code.md index 8e98e3f5f0f..64c064b9ba9 100644 --- a/docs/en/development/browse_code.md +++ b/docs/en/development/browse_code.md @@ -5,7 +5,7 @@ toc_title: Browse ClickHouse Source Code # Browse ClickHouse Source Code {#browse-clickhouse-source-code} -You can use **Woboq** online code browser available [here](https://clickhouse.tech/codebrowser/html_report///ClickHouse/src/index.html). It provides code navigation and semantic highlighting, search and indexing. The code snapshot is updated daily. +You can use **Woboq** online code browser available [here](https://clickhouse.tech/codebrowser/html_report/ClickHouse/src/index.html). It provides code navigation and semantic highlighting, search and indexing. The code snapshot is updated daily. Also, you can browse sources on [GitHub](https://github.com/ClickHouse/ClickHouse) as usual. From 23ca04ee097c2ba75cbcf39b63b252a7ec976613 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 15 Apr 2020 23:18:52 +0300 Subject: [PATCH 264/743] Update browse_code.md --- docs/en/development/browse_code.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/browse_code.md b/docs/en/development/browse_code.md index 64c064b9ba9..3e7b259bb39 100644 --- a/docs/en/development/browse_code.md +++ b/docs/en/development/browse_code.md @@ -1,6 +1,6 @@ --- toc_priority: 63 -toc_title: Browse ClickHouse Source Code +toc_title: Browse Source Code --- # Browse ClickHouse Source Code {#browse-clickhouse-source-code} From 33dde399369d35a5c0bf6c03097b3996de007163 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Apr 2020 23:38:44 +0300 Subject: [PATCH 265/743] Added results for Intel NUC --- website/benchmark_hardware.html | 52 +++++++++++++++++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/website/benchmark_hardware.html b/website/benchmark_hardware.html index 36f034457f8..b45ccb4875c 100644 --- a/website/benchmark_hardware.html +++ b/website/benchmark_hardware.html @@ -2478,6 +2478,57 @@ var results = [0.024, 0.007, 0.007] ] }, + + { + "system": "Intel NUC, 4 cores (Intel i7-6770HQ), 32 GiB RAM, 1 TB NVMe SSD", + "time": "2020-04-15 00:00:00", + "result": + [ +[0.003, 0.002, 0.001], +[0.025, 0.016, 0.018], +[0.084, 0.058, 0.057], +[0.158, 0.092, 0.085], +[0.273, 0.211, 0.190], +[0.671, 0.555, 0.539], +[0.031, 0.033, 0.033], +[0.026, 0.019, 0.017], +[1.183, 1.110, 1.090], +[1.330, 1.246, 1.254], +[0.352, 0.297, 0.296], +[0.441, 0.375, 0.352], +[1.611, 1.491, 1.439], +[2.130, 2.022, 1.976], +[1.903, 1.795, 1.819], +[1.927, 1.851, 1.861], +[5.282, 5.155, 5.172], +[3.246, 3.313, 3.189], +[12.059, 11.378, 10.562], +[0.146, 0.092, 0.090], +[2.103, 1.496, 1.477], +[2.447, 1.777, 1.734], +[5.123, 3.999, 3.955], +[3.733, 1.808, 1.775], +[0.685, 0.530, 0.523], +[0.525, 0.446, 0.438], +[0.755, 0.545, 0.547], +[2.052, 1.416, 1.403], +[2.976, 2.441, 2.423], +[2.197, 2.189, 2.164], +[1.748, 1.596, 1.607], +[2.773, 2.481, 2.466], +[18.903, 19.166, 16.563], +[7.457, 7.116, 6.943], +[7.311, 6.957, 6.958], +[3.036, 3.005, 2.991], +[0.247, 0.186, 0.162], +[0.100, 0.063, 0.065], +[0.098, 0.061, 0.056], +[0.434, 0.344, 0.331], +[0.040, 0.025, 0.025], +[0.049, 0.026, 0.026], +[0.022, 0.008, 0.006] + ] + }, ]; @@ -2914,6 +2965,7 @@ Results for AMD Ryzen are from Alexey Milovidov. Firefox was running in b Results for Azure E32s are from Piotr Maśko.
Results for MacBook Pro are from Denis Glazachev. MacOS Catalina Version 10.15.4 (19E266). For "drop caches", the "Free Up RAM" in CleanMyMac is used.
Results for AMD EPYC 7702 are from Peng Gao in sina.com.
+Results for Intel NUC are from Alexander Zaitsev, Altinity.
Xeon Gold 6230 server is using 4 x SAMSUNG datacenter class SSD in RAID-10.
Results for Yandex Managed ClickHouse for "cold cache" are biased and should not be compared, because cache was not flushed for every next query.
From 23ae3f4bffbb04b3a348d0553b831d4dc012215b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 00:43:58 +0300 Subject: [PATCH 266/743] Update select_format.xml --- tests/performance/select_format.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/select_format.xml b/tests/performance/select_format.xml index e47d981c4d7..2bdbde83c2d 100644 --- a/tests/performance/select_format.xml +++ b/tests/performance/select_format.xml @@ -44,7 +44,7 @@ ODBCDriver2 MySQLWire Avro - MsgPack + From f704c465a7e2a3e069964df89a8fa1857d4e2569 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Apr 2020 03:42:23 +0300 Subject: [PATCH 267/743] Make aware of one special case of imcompletely written parts #9940 --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 60 +++++++++++-------- 1 file changed, 34 insertions(+), 26 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5d799d257bc..a5a7f1f4494 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -731,35 +731,43 @@ void IMergeTreeDataPart::remove() const return; } - try + if (checksums.empty()) { - /// Remove each expected file in directory, then remove directory itself. - -#if !__clang__ -# pragma GCC diagnostic push -# pragma GCC diagnostic ignored "-Wunused-variable" -#endif - for (const auto & [file, _] : checksums.files) - disk->remove(to + "/" + file); -#if !__clang__ -# pragma GCC diagnostic pop -#endif - - for (const auto & file : {"checksums.txt", "columns.txt"}) - disk->remove(to + "/" + file); - disk->removeIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_PATH); - - disk->remove(to); - } - catch (...) - { - /// Recursive directory removal does many excessive "stat" syscalls under the hood. - - LOG_ERROR(storage.log, "Cannot quickly remove directory " << fullPath(disk, to) << " by removing files; fallback to recursive removal. Reason: " - << getCurrentExceptionMessage(false)); - + /// If the part is not completely written, we cannot use fast path by listing files. disk->removeRecursive(to + "/"); } + else + { + try + { + /// Remove each expected file in directory, then remove directory itself. + + #if !__clang__ + # pragma GCC diagnostic push + # pragma GCC diagnostic ignored "-Wunused-variable" + #endif + for (const auto & [file, _] : checksums.files) + disk->remove(to + "/" + file); + #if !__clang__ + # pragma GCC diagnostic pop + #endif + + for (const auto & file : {"checksums.txt", "columns.txt"}) + disk->remove(to + "/" + file); + disk->removeIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_PATH); + + disk->remove(to); + } + catch (...) + { + /// Recursive directory removal does many excessive "stat" syscalls under the hood. + + LOG_ERROR(storage.log, "Cannot quickly remove directory " << fullPath(disk, to) << " by removing files; fallback to recursive removal. Reason: " + << getCurrentExceptionMessage(false)); + + disk->removeRecursive(to + "/"); + } + } } String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const From b2f2ceba1f52ceb6ba1b266f25659efbd9c708cb Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 03:44:46 +0300 Subject: [PATCH 268/743] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index a1494c23066..55718e71feb 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -76,6 +76,6 @@ toc_title: Adopters | [ЦВТ](https://htc-cs.ru/) | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | | [МКБ](https://mkb.ru/) | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | | [金数据](https://jinshuju.net) | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | -| [Instana](https://www.instana.com) | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | +| [Instana](https://www.instana.com) | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | [Wargaming](https://wargaming.com/en/) | | | — | — | [Interview](https://habr.com/en/post/496954/) | [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From 074fdb016b75b04b90ddf737aa1a2fad948fdaa8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 03:46:24 +0300 Subject: [PATCH 269/743] Update adopters.md --- docs/en/introduction/adopters.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 55718e71feb..be886576c1b 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -76,6 +76,7 @@ toc_title: Adopters | [ЦВТ](https://htc-cs.ru/) | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | | [МКБ](https://mkb.ru/) | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | | [金数据](https://jinshuju.net) | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | -| [Instana](https://www.instana.com) | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | [Wargaming](https://wargaming.com/en/) | | | — | — | [Interview](https://habr.com/en/post/496954/) | +| [Instana](https://www.instana.com) | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | +| [Wargaming](https://wargaming.com/en/) | | | — | — | [Interview](https://habr.com/en/post/496954/) | [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From 868d28be7b197904c1e86626d87b61892acf17e2 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 03:46:59 +0300 Subject: [PATCH 270/743] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index be886576c1b..4e3fd4d5f4b 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -77,6 +77,6 @@ toc_title: Adopters | [МКБ](https://mkb.ru/) | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | | [金数据](https://jinshuju.net) | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | | [Instana](https://www.instana.com) | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | -| [Wargaming](https://wargaming.com/en/) | | | — | — | [Interview](https://habr.com/en/post/496954/) | +| [Wargaming](https://wargaming.com/en/) | Online games | | — | — | [Interview](https://habr.com/en/post/496954/) | [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From 01d3c8319354bfc2abadd4befa9aba717634e776 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 03:48:07 +0300 Subject: [PATCH 271/743] Update adopters.md --- docs/en/introduction/adopters.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 4e3fd4d5f4b..b1463727b70 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -77,6 +77,8 @@ toc_title: Adopters | [МКБ](https://mkb.ru/) | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | | [金数据](https://jinshuju.net) | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | | [Instana](https://www.instana.com) | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | -| [Wargaming](https://wargaming.com/en/) | Online games | | — | — | [Interview](https://habr.com/en/post/496954/) | +| [Wargaming](https://wargaming.com/en/) | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | +| [Crazypanda](https://crazypanda.ru/en/) | Games | | — | — | Live session on ClickHouse meetup | + [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From 2a7396e41d46b1f9d9d708ce001ab9acfc5d4f14 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 03:49:47 +0300 Subject: [PATCH 272/743] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index b1463727b70..c6f2a63c1be 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -79,6 +79,7 @@ toc_title: Adopters | [Instana](https://www.instana.com) | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | | [Wargaming](https://wargaming.com/en/) | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | | [Crazypanda](https://crazypanda.ru/en/) | Games | | — | — | Live session on ClickHouse meetup | +| [FunCorp](https://fun.co/rp) | Games | | — | — | [Article](https://habr.com/en/company/funcorp/blog/433346/) | [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From 725969289ed6ae599c02f63176a2090a8e10d3b1 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 03:50:44 +0300 Subject: [PATCH 273/743] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index c6f2a63c1be..b75b1c942eb 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -79,7 +79,7 @@ toc_title: Adopters | [Instana](https://www.instana.com) | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | | [Wargaming](https://wargaming.com/en/) | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | | [Crazypanda](https://crazypanda.ru/en/) | Games | | — | — | Live session on ClickHouse meetup | -| [FunCorp](https://fun.co/rp) | Games | | — | — | [Article](https://habr.com/en/company/funcorp/blog/433346/) | +| [FunCorp](https://fun.co/rp) | Games | | — | — | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From cdeda4ab91542e9bc23c75e62aee0aa2a5aea573 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Apr 2020 23:28:05 +0300 Subject: [PATCH 274/743] Fix usage of max_parser_depth setting; remove harmful default function arguments --- programs/client/Client.cpp | 8 +- programs/compressor/Compressor.cpp | 3 +- programs/copier/ClusterCopier.cpp | 22 +- programs/copier/TaskTableAndShard.h | 10 +- programs/format/Format.cpp | 2 +- programs/local/LocalServer.cpp | 4 +- programs/odbc-bridge/ColumnInfoHandler.cpp | 4 +- src/Access/DiskAccessStorage.cpp | 3 +- src/Access/RowPolicyCache.cpp | 3 +- .../parseAggregateFunctionParameters.cpp | 3 +- .../tests/gtest_compressionCodec.cpp | 2 +- src/Core/Defines.h | 3 + src/Core/Settings.h | 2 +- src/DataStreams/tests/expression_stream.cpp | 2 +- src/DataStreams/tests/filter_stream.cpp | 2 +- src/DataTypes/DataTypeFactory.cpp | 4 +- src/Databases/DatabaseOrdinary.cpp | 3 +- .../tests/gtest_dictionary_configuration.cpp | 8 +- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 8 +- .../InterpreterKillQueryQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 23 +- .../JoinToSubqueryTransformVisitor.cpp | 3 +- src/Interpreters/SyntaxAnalyzer.cpp | 3 +- src/Interpreters/SystemLog.h | 3 +- src/Interpreters/loadMetadata.cpp | 2 +- src/Interpreters/tests/CMakeLists.txt | 3 - src/Interpreters/tests/create_query.cpp | 2 +- src/Interpreters/tests/expression.cpp | 2 +- .../tests/expression_analyzer.cpp | 2 +- .../tests/gtest_cycle_aliases.cpp | 6 +- .../tests/in_join_subqueries_preprocessor.cpp | 2 +- .../tests/logical_expressions_optimizer.cpp | 295 ------------------ src/Parsers/IParser.h | 2 - src/Parsers/parseQuery.cpp | 21 +- src/Parsers/parseQuery.h | 20 +- src/Parsers/tests/create_parser.cpp | 2 +- src/Parsers/tests/gtest_dictionary_parser.cpp | 18 +- src/Parsers/tests/select_parser.cpp | 2 +- src/Storages/ColumnsDescription.cpp | 4 +- src/Storages/ConstraintsDescription.cpp | 4 +- src/Storages/IndicesDescription.cpp | 5 +- src/Storages/MutationCommands.cpp | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 9 +- src/Storages/getStructureOfRemoteTable.cpp | 2 +- ..._transform_query_for_external_database.cpp | 2 +- .../parseColumnsListForTableFunction.cpp | 4 +- 48 files changed, 152 insertions(+), 394 deletions(-) delete mode 100644 src/Interpreters/tests/logical_expressions_optimizer.cpp diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e01eef98006..e4c046d18ba 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -684,7 +684,7 @@ private: if (ignore_error) { Tokens tokens(begin, end); - IParser::Pos token_iterator(tokens); + IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth); while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid()) ++token_iterator; begin = token_iterator->end; @@ -958,10 +958,12 @@ private: ParserQuery parser(end, true); ASTPtr res; + const auto & settings = context.getSettingsRef(); + if (is_interactive || ignore_error) { String message; - res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, 0); + res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, settings.max_query_size, settings.max_parser_depth); if (!res) { @@ -970,7 +972,7 @@ private: } } else - res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, 0); + res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, settings.max_query_size, settings.max_parser_depth); if (is_interactive) { diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index 98a3055da28..fecdad9bcea 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB @@ -123,7 +124,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) DB::ParserCodec codec_parser; std::string codecs_line = boost::algorithm::join(codecs, ","); - auto ast = DB::parseQuery(codec_parser, "(" + codecs_line + ")", 0); + auto ast = DB::parseQuery(codec_parser, "(" + codecs_line + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); codec = DB::CompressionCodecFactory::instance().get(ast, nullptr); } else diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index c4714ff201f..cd5b1e2a2cd 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1197,7 +1197,9 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( query += " LIMIT " + limit; ParserQuery p_query(query.data() + query.size()); - return parseQuery(p_query, query, 0); + + const auto & settings = context.getSettingsRef(); + return parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth); }; /// Load balancing @@ -1409,7 +1411,8 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES "; ParserQuery p_query(query.data() + query.size()); - query_insert_ast = parseQuery(p_query, query, 0); + const auto & settings = context.getSettingsRef(); + query_insert_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth); LOG_DEBUG(log, "Executing INSERT query: " << query); } @@ -1634,7 +1637,8 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time &task_cluster->settings_pull); ParserCreateQuery parser_create_query; - return parseQuery(parser_create_query, create_query_pull_str, 0); + const auto & settings = context.getSettingsRef(); + return parseQuery(parser_create_query, create_query_pull_str, settings.max_query_size, settings.max_parser_depth); } /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. @@ -1712,7 +1716,8 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti } ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); + const auto & settings = context.getSettingsRef(); + ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); @@ -1759,7 +1764,8 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, << partition_quoted_name << " existence, executing query: " << query); ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); +const auto & settings = context.getSettingsRef(); + ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); Context local_context = context; local_context.setSettings(task_cluster->settings_pull); @@ -1793,7 +1799,8 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi << "existence, executing query: " << query); ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); + const auto & settings = context.getSettingsRef(); + ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); Context local_context = context; local_context.setSettings(task_cluster->settings_pull); @@ -1826,7 +1833,8 @@ UInt64 ClusterCopier::executeQueryOnCluster( if (query_ast_ == nullptr) { ParserQuery p_query(query.data() + query.size()); - query_ast = parseQuery(p_query, query, 0); + const auto & settings = context.getSettingsRef(); + query_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth); } else query_ast = query_ast_; diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index 615ad297b79..32841e93a14 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -4,6 +4,9 @@ #include "Internals.h" #include "ClusterPartition.h" +#include + + namespace DB { namespace ErrorCodes @@ -260,9 +263,10 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf + "." + escapeForFileName(table_push.second); engine_push_str = config.getString(table_prefix + "engine"); + { ParserStorage parser_storage; - engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); + engine_push_ast = parseQuery(parser_storage, engine_push_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast); @@ -273,7 +277,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf auxiliary_engine_split_asts.reserve(number_of_splits); { ParserExpressionWithOptionalAlias parser_expression(false); - sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); + sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); @@ -291,7 +295,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf if (!where_condition_str.empty()) { ParserExpressionWithOptionalAlias parser_expression(false); - where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); + where_condition_ast = parseQuery(parser_expression, where_condition_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); // Will use canonical expression form where_condition_str = queryToString(where_condition_ast); diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index f826d6394bc..b5a4e2d1603 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -53,7 +53,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) const char * end = pos + query.size(); ParserQuery parser(end); - ASTPtr res = parseQuery(parser, pos, end, "query", 0); + ASTPtr res = parseQuery(parser, pos, end, "query", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); if (!quiet) { diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 26752da5d87..1ab07d79401 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -267,8 +267,10 @@ void LocalServer::processQueries() String initial_create_query = getInitialCreateTableQuery(); String queries_str = initial_create_query + config().getRawString("query"); + const auto & settings = context->getSettingsRef(); + std::vector queries; - auto parse_res = splitMultipartQuery(queries_str, 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); diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index b89d50569f6..e3c00f48fb5 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -120,12 +120,14 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques SCOPE_EXIT(SQLFreeStmt(hstmt, SQL_DROP)); + const auto & context_settings = context->getSettingsRef(); + /// TODO Why not do SQLColumns instead? std::string name = schema_name.empty() ? table_name : schema_name + "." + table_name; std::stringstream ss; std::string input = "SELECT * FROM " + name + " WHERE 1 = 0"; ParserQueryWithOutput parser; - ASTPtr select = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr select = parseQuery(parser, input.data(), input.data() + input.size(), "", context_settings.max_query_size, context_settings.max_parser_depth); IAST::FormatSettings settings(ss, true); settings.always_quote_identifiers = true; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 12c65e7df1e..d7ba8563f5a 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -93,7 +94,7 @@ namespace const char * end = begin + file_contents.size(); while (pos < end) { - queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0)); + queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH)); while (isWhitespaceASCII(*pos) || *pos == ';') ++pos; } diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index 44f2cd160d4..fc67b7e9b86 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -77,7 +78,7 @@ void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_) try { ParserExpression parser; - parsed_conditions[type] = parseQuery(parser, condition, 0); + parsed_conditions[type] = parseQuery(parser, condition, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); } catch (...) { diff --git a/src/AggregateFunctions/parseAggregateFunctionParameters.cpp b/src/AggregateFunctions/parseAggregateFunctionParameters.cpp index bcb73f1e9d9..2a6b9e3b499 100644 --- a/src/AggregateFunctions/parseAggregateFunctionParameters.cpp +++ b/src/AggregateFunctions/parseAggregateFunctionParameters.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -65,7 +66,7 @@ void getAggregateFunctionNameAndParametersArray( ParserExpressionList params_parser(false); ASTPtr args_ast = parseQuery(params_parser, parameters_str.data(), parameters_str.data() + parameters_str.size(), - "parameters of aggregate function in " + error_context, 0); + "parameters of aggregate function in " + error_context, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); if (args_ast->children.empty()) throw Exception("Incorrect list of parameters to aggregate function " diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index a6bfdaebb14..f3652366a24 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -462,7 +462,7 @@ CompressionCodecPtr makeCodec(const std::string & codec_string, const DataTypePt { const std::string codec_statement = "(" + codec_string + ")"; Tokens tokens(codec_statement.begin().base(), codec_statement.end().base()); - IParser::Pos token_iterator(tokens); + IParser::Pos token_iterator(tokens, 0); Expected expected; ASTPtr codec_ast; diff --git a/src/Core/Defines.h b/src/Core/Defines.h index c797f527be9..fe614cec6bd 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -95,3 +95,6 @@ /// Actually, there may be multiple acquisitions of different locks for a given table within one query. /// Check with IStorage class for the list of possible locks #define DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC 120 + +/// Default limit on recursion depth of recursive descend parser. +#define DBMS_DEFAULT_MAX_PARSER_DEPTH 1000 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 725171d4a1b..325abc16f3f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -404,7 +404,7 @@ struct Settings : public SettingsCollection M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ M(SettingUInt64, multiple_joins_rewriter_version, 1, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ M(SettingBool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ - M(SettingUInt64, max_parser_depth, 1000, "Maximum parser depth.", 0) \ + M(SettingUInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \ M(SettingBool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \ M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \ diff --git a/src/DataStreams/tests/expression_stream.cpp b/src/DataStreams/tests/expression_stream.cpp index bd4117f5aab..fbfde018ed6 100644 --- a/src/DataStreams/tests/expression_stream.cpp +++ b/src/DataStreams/tests/expression_stream.cpp @@ -33,7 +33,7 @@ try std::string input = "SELECT number, number / 3, number * number"; ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); Context context = Context::createGlobal(); context.makeGlobalContext(); diff --git a/src/DataStreams/tests/filter_stream.cpp b/src/DataStreams/tests/filter_stream.cpp index 5e324251440..8356b90957c 100644 --- a/src/DataStreams/tests/filter_stream.cpp +++ b/src/DataStreams/tests/filter_stream.cpp @@ -35,7 +35,7 @@ try std::string input = "SELECT number, number % 3 == 1"; ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); formatAST(*ast, std::cerr); std::cerr << std::endl; diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 20f7681ec1b..f81adfe347c 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -9,6 +9,8 @@ #include #include #include +#include + namespace DB { @@ -26,7 +28,7 @@ namespace ErrorCodes DataTypePtr DataTypeFactory::get(const String & full_name) const { ParserIdentifierWithOptionalParameters parser; - ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0); + ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); return get(ast); } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index a1f5ea1ae6f..11c4a4400cd 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -234,7 +234,8 @@ void DatabaseOrdinary::alterTable( } ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, statement.data(), statement.data() + statement.size(), "in file " + table_metadata_path, 0); + ASTPtr ast = parseQuery(parser, statement.data(), statement.data() + statement.size(), "in file " + table_metadata_path, + 0, context.getSettingsRef().max_parser_depth); auto & ast_create_query = ast->as(); diff --git a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp index 9c1ad9467b0..ae9c5385b8d 100644 --- a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp +++ b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp @@ -52,7 +52,7 @@ TEST(ConvertDictionaryAST, SimpleDictConfiguration) " RANGE(MIN second_column MAX third_column)"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create); @@ -120,7 +120,7 @@ TEST(ConvertDictionaryAST, TrickyAttributes) " SOURCE(CLICKHOUSE(HOST 'localhost'))"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create); @@ -165,7 +165,7 @@ TEST(ConvertDictionaryAST, ComplexKeyAndLayoutWithParams) " LIFETIME(MIN 1 MAX 10)"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create); @@ -216,7 +216,7 @@ TEST(ConvertDictionaryAST, ComplexSource) " RANGE(MIN second_column MAX third_column)"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create); /// source diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 4a39cc6b8a1..e786849d121 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -463,7 +463,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task) ParserQuery parser_query(end); String description; - task.query = parseQuery(parser_query, begin, end, description, 0); + task.query = parseQuery(parser_query, begin, end, description, 0, context.getSettingsRef().max_parser_depth); } // XXX: serious design flaw since `ASTQueryWithOnCluster` is not inherited from `IAST`! diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 076b654548b..37e2c8c5945 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -6,6 +6,8 @@ #include #include +#include + #include #include @@ -181,7 +183,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns) String type_name = column.type->getName(); auto pos = type_name.data(); const auto end = pos + type_name.size(); - column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0); + column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); columns_list->children.emplace_back(column_declaration); } @@ -207,7 +209,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) String type_name = column.type->getName(); auto type_name_pos = type_name.data(); const auto type_name_end = type_name_pos + type_name.size(); - column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0); + column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); if (column.default_desc.expression) { @@ -227,7 +229,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) auto codec_desc_pos = codec_desc.data(); const auto codec_desc_end = codec_desc_pos + codec_desc.size(); ParserIdentifierWithParameters codec_p; - column_declaration->codec = parseQuery(codec_p, codec_desc_pos, codec_desc_end, "column codec", 0); + column_declaration->codec = parseQuery(codec_p, codec_desc_pos, codec_desc_end, "column codec", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); } if (column.ttl) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index b23d88524e1..23f39ab3fc5 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -267,7 +267,7 @@ BlockIO InterpreterKillQueryQuery::execute() else { ParserAlterCommand parser; - auto command_ast = parseQuery(parser, command_col.getDataAt(i).toString(), 0); + auto command_ast = parseQuery(parser, command_col.getDataAt(i).toString(), 0, context.getSettingsRef().max_parser_depth); required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand(command_ast->as(), table_id.database_name, table_id.table_name); if (!access->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), required_access_rights)) { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 82fc4d0873e..691b3c1045b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -137,7 +137,7 @@ String InterpreterSelectQuery::generateFilterActions(ExpressionActionsPtr & acti for (const auto & column_str : prerequisite_columns) { ParserExpression expr_parser; - expr_list->children.push_back(parseQuery(expr_parser, column_str, 0)); + expr_list->children.push_back(parseQuery(expr_parser, column_str, 0, context->getSettingsRef().max_parser_depth)); } select_ast->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared()); diff --git a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index 4c2dcc19a88..0d3b88facce 100644 --- a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -37,7 +38,7 @@ namespace ASTPtr getCreateQueryImpl( const User & user, const AccessControlManager * manager /* not used if attach_mode == true */, - bool attach_mode = false) + bool attach_mode) { auto query = std::make_shared(); query->name = user.getName(); @@ -72,7 +73,7 @@ namespace } - ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager * manager, bool attach_mode = false) + ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager * manager, bool attach_mode) { auto query = std::make_shared(); query->name = role.getName(); @@ -90,7 +91,7 @@ namespace } - ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControlManager * manager, bool attach_mode = false) + ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControlManager * manager, bool attach_mode) { auto query = std::make_shared(); query->name = profile.getName(); @@ -121,7 +122,7 @@ namespace ASTPtr getCreateQueryImpl( const Quota & quota, const AccessControlManager * manager /* not used if attach_mode == true */, - bool attach_mode = false) + bool attach_mode) { auto query = std::make_shared(); query->name = quota.getName(); @@ -156,7 +157,7 @@ namespace ASTPtr getCreateQueryImpl( const RowPolicy & policy, const AccessControlManager * manager /* not used if attach_mode == true */, - bool attach_mode = false) + bool attach_mode) { auto query = std::make_shared(); query->name_parts = RowPolicy::FullNameParts{policy.getDatabase(), policy.getTableName(), policy.getName()}; @@ -171,7 +172,7 @@ namespace if (!condition.empty()) { ParserExpression parser; - ASTPtr expr = parseQuery(parser, condition, 0); + ASTPtr expr = parseQuery(parser, condition, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); query->conditions.push_back(std::pair{index, expr}); } } @@ -190,7 +191,7 @@ namespace ASTPtr getCreateQueryImpl( const IAccessEntity & entity, const AccessControlManager * manager /* not used if attach_mode == true */, - bool attach_mode = false) + bool attach_mode) { if (const User * user = typeid_cast(&entity)) return getCreateQueryImpl(*user, manager, attach_mode); @@ -264,24 +265,24 @@ ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreat if (show_query.current_user) { auto user = context.getUser(); - return getCreateQueryImpl(*user, &access_control); + return getCreateQueryImpl(*user, &access_control, false); } if (show_query.current_quota) { auto quota = access_control.read(context.getQuota()->getUsageInfo().quota_id); - return getCreateQueryImpl(*quota, &access_control); + return getCreateQueryImpl(*quota, &access_control, false); } auto type = getType(show_query.kind); if (show_query.kind == Kind::ROW_POLICY) { RowPolicyPtr policy = access_control.read(show_query.row_policy_name.getFullName(context)); - return getCreateQueryImpl(*policy, &access_control); + return getCreateQueryImpl(*policy, &access_control, false); } auto entity = access_control.read(access_control.getID(type, show_query.name)); - return getCreateQueryImpl(*entity, &access_control); + return getCreateQueryImpl(*entity, &access_control, false); } diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 331c364c5fa..6a08e11ad9e 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB @@ -35,7 +36,7 @@ namespace ASTPtr makeSubqueryTemplate() { ParserTablesInSelectQueryElement parser(true); - ASTPtr subquery_template = parseQuery(parser, "(select * from _t) as `--.s`", 0); + ASTPtr subquery_template = parseQuery(parser, "(select * from _t) as `--.s`", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); if (!subquery_template) throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR); return subquery_template; diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index 5c1b6c7e62b..e19961e7a7c 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -587,7 +588,7 @@ void replaceJoinedTable(const ASTSelectQuery & select_query) if (table_id.alias.empty() && table_id.isShort()) { ParserTableExpression parser; - table_expr = parseQuery(parser, expr, 0)->as(); + table_expr = parseQuery(parser, expr, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH)->as(); } } } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 7c8dc1606f7..9b0c273674c 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -460,7 +461,7 @@ ASTPtr SystemLog::getCreateTableQuery() ParserStorage storage_parser; ASTPtr storage_ast = parseQuery( storage_parser, storage_def.data(), storage_def.data() + storage_def.size(), - "Storage to create table for " + LogElement::name(), 0); + "Storage to create table for " + LogElement::name(), 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); create->set(create->storage, storage_ast); return create; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 53954faa2c0..42bef47a501 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -36,7 +36,7 @@ static void executeCreateQuery( bool has_force_restore_data_flag) { ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0); + ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0, context.getSettingsRef().max_parser_depth); auto & ast_create_query = ast->as(); ast_create_query.attach = true; diff --git a/src/Interpreters/tests/CMakeLists.txt b/src/Interpreters/tests/CMakeLists.txt index da45c1a5153..26ebf007e6c 100644 --- a/src/Interpreters/tests/CMakeLists.txt +++ b/src/Interpreters/tests/CMakeLists.txt @@ -47,9 +47,6 @@ add_executable (two_level_hash_map two_level_hash_map.cpp) target_include_directories (two_level_hash_map SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR}) target_link_libraries (two_level_hash_map PRIVATE dbms) -add_executable (logical_expressions_optimizer logical_expressions_optimizer.cpp) -target_link_libraries (logical_expressions_optimizer PRIVATE dbms clickhouse_parsers) - add_executable (in_join_subqueries_preprocessor in_join_subqueries_preprocessor.cpp) target_link_libraries (in_join_subqueries_preprocessor PRIVATE dbms clickhouse_parsers) add_check(in_join_subqueries_preprocessor) diff --git a/src/Interpreters/tests/create_query.cpp b/src/Interpreters/tests/create_query.cpp index 20a0bfcb062..ad006de3be6 100644 --- a/src/Interpreters/tests/create_query.cpp +++ b/src/Interpreters/tests/create_query.cpp @@ -76,7 +76,7 @@ try ") ENGINE = Log"; ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); Context context = Context::createGlobal(); context.makeGlobalContext(); diff --git a/src/Interpreters/tests/expression.cpp b/src/Interpreters/tests/expression.cpp index 8d64b4f64ce..5432e405d1c 100644 --- a/src/Interpreters/tests/expression.cpp +++ b/src/Interpreters/tests/expression.cpp @@ -41,7 +41,7 @@ int main(int argc, char ** argv) "s1 < s2 AND x % 3 < x % 5"; ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); formatAST(*ast, std::cerr); std::cerr << std::endl; diff --git a/src/Interpreters/tests/expression_analyzer.cpp b/src/Interpreters/tests/expression_analyzer.cpp index 2f8b6b2aef2..66b4900584f 100644 --- a/src/Interpreters/tests/expression_analyzer.cpp +++ b/src/Interpreters/tests/expression_analyzer.cpp @@ -72,7 +72,7 @@ private: ParserSelectQuery parser; std::string message; auto text = query.data(); - if (ASTPtr ast = tryParseQuery(parser, text, text + query.size(), message, false, "", false, 0)) + if (ASTPtr ast = tryParseQuery(parser, text, text + query.size(), message, false, "", false, 0, 0)) return ast; throw Exception(message, ErrorCodes::SYNTAX_ERROR); } diff --git a/src/Interpreters/tests/gtest_cycle_aliases.cpp b/src/Interpreters/tests/gtest_cycle_aliases.cpp index c8037b23d84..593db93de3e 100644 --- a/src/Interpreters/tests/gtest_cycle_aliases.cpp +++ b/src/Interpreters/tests/gtest_cycle_aliases.cpp @@ -12,11 +12,11 @@ TEST(QueryNormalizer, SimpleCycleAlias) { String query = "a as b, b as a"; ParserExpressionList parser(false); - ASTPtr ast = parseQuery(parser, query, 0); + ASTPtr ast = parseQuery(parser, query, 0, 0); Aliases aliases; - aliases["a"] = parseQuery(parser, "b as a", 0)->children[0]; - aliases["b"] = parseQuery(parser, "a as b", 0)->children[0]; + aliases["a"] = parseQuery(parser, "b as a", 0, 0)->children[0]; + aliases["b"] = parseQuery(parser, "a as b", 0, 0)->children[0]; Settings settings; QueryNormalizer::Data normalizer_data(aliases, settings); diff --git a/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index 9a17f03f32a..412557f13b8 100644 --- a/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -1224,7 +1224,7 @@ bool parse(DB::ASTPtr & ast, const std::string & query) std::string message; auto begin = query.data(); auto end = begin + query.size(); - ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0); + ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0, 0); return ast != nullptr; } diff --git a/src/Interpreters/tests/logical_expressions_optimizer.cpp b/src/Interpreters/tests/logical_expressions_optimizer.cpp deleted file mode 100644 index c21c4dda299..00000000000 --- a/src/Interpreters/tests/logical_expressions_optimizer.cpp +++ /dev/null @@ -1,295 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include - - -namespace -{ - -struct TestEntry -{ - std::string input; - std::string expected_output; - UInt64 limit; -}; - -using TestEntries = std::vector; -using TestResult = std::pair; - -void run(); -void performTests(const TestEntries & entries); -TestResult check(const TestEntry & entry); -bool parse(DB::ASTPtr & ast, const std::string & query); -bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs); -void reorder(DB::IAST * ast); - - -void run() -{ - /// NOTE: Queries are not always realistic, but we are only interested in the syntax. - TestEntries entries = - { - { - "SELECT 1", - "SELECT 1", - 3 - }, - - // WHERE - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - 4 - }, - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol')", - 3 - }, - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol')", - 2 - }, - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report WHERE (value = 1000) OR name IN ('Alice', 'Bob', 'Carol')", - 2 - }, - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol') OR (value = 2000)", - "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol') OR value IN (1000, 2000)", - 2 - }, - - { - "SELECT value FROM report WHERE ((value + 1) = 1000) OR ((2 * value) = 2000) OR ((2 * value) = 4000) OR ((value + 1) = 3000)", - "SELECT value FROM report WHERE ((value + 1) IN (1000, 3000)) OR ((2 * value) IN (2000, 4000))", - 2 - }, - - { - "SELECT name, value FROM report WHERE ((name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')) AND ((value = 1000) OR (value = 2000))", - "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol') AND ((value = 1000) OR (value = 2000))", - 3 - }, - - // PREWHERE - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - 4 - }, - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol')", - 3 - }, - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol')", - 2 - }, - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report PREWHERE (value = 1000) OR name IN ('Alice', 'Bob', 'Carol')", - 2 - }, - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol') OR (value = 2000)", - "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol') OR value IN (1000, 2000)", - 2 - }, - - { - "SELECT value FROM report PREWHERE ((value + 1) = 1000) OR ((2 * value) = 2000) OR ((2 * value) = 4000) OR ((value + 1) = 3000)", - "SELECT value FROM report PREWHERE (value + 1) IN (1000, 3000) OR (2 * value) IN (2000, 4000)", - 2 - }, - - // HAVING - - { - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number = 1", - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number = 1", - 2 - }, - - { - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)", - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number IN (1, 2)", - 2 - }, - - { - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)", - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)", - 3 - }, - - { - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING ((number + 1) = 1) OR ((number + 1) = 2) OR ((number + 3) = 7)", - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING ((number + 3) = 7) OR (number + 1) IN (1, 2)", - 2 - }, - - // PREWHERE + WHERE + HAVING - - { - "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE (U = 1) OR (U = 2) " - "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)", - "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE U IN (1, 2) " - "WHERE T IN (1, 2) GROUP BY number HAVING number IN (1, 2)", - 2 - }, - - { - "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE (U = 1) OR (U = 2) OR (U = 3) " - "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)", - "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE U IN (1, 2, 3) " - "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)", - 3 - }, - - { - "SELECT x = 1 OR x=2 OR (x = 3 AS x3) AS y, 4 AS x", - "SELECT x IN (1, 2, 3) AS y, 4 AS x", - 2 - } - }; - - performTests(entries); -} - -void performTests(const TestEntries & entries) -{ - unsigned int count = 0; - unsigned int i = 1; - - for (const auto & entry : entries) - { - auto res = check(entry); - if (res.first) - { - ++count; - std::cout << "Test " << i << " passed.\n"; - } - else - std::cout << "Test " << i << " failed. Expected: " << entry.expected_output << ". Received: " << res.second << "\n"; - - ++i; - } - std::cout << count << " out of " << entries.size() << " test(s) passed.\n"; -} - -TestResult check(const TestEntry & entry) -{ - try - { - /// Parse and optimize the incoming query. - DB::ASTPtr ast_input; - if (!parse(ast_input, entry.input)) - return TestResult(false, "parse error"); - - auto select_query = typeid_cast(&*ast_input); - - DB::LogicalExpressionsOptimizer optimizer(select_query, entry.limit); - optimizer.perform(); - - /// Parse the expected result. - DB::ASTPtr ast_expected; - if (!parse(ast_expected, entry.expected_output)) - return TestResult(false, "parse error"); - - /// Compare the optimized query and the expected result. - bool res = equals(ast_input, ast_expected); - std::string output = DB::queryToString(ast_input); - - return TestResult(res, output); - } - catch (DB::Exception & e) - { - return TestResult(false, e.displayText()); - } -} - -bool parse(DB::ASTPtr & ast, const std::string & query) -{ - DB::ParserSelectQuery parser; - std::string message; - auto begin = query.data(); - auto end = begin + query.size(); - ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0); - return ast != nullptr; -} - -bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs) -{ - DB::ASTPtr lhs_reordered = lhs->clone(); - reorder(&*lhs_reordered); - - DB::ASTPtr rhs_reordered = rhs->clone(); - reorder(&*rhs_reordered); - - return lhs_reordered->getTreeHash() == rhs_reordered->getTreeHash(); -} - -void reorderImpl(DB::IAST * ast) -{ - if (ast == nullptr) - return; - - auto & children = ast->children; - if (children.empty()) - return; - - for (auto & child : children) - reorderImpl(&*child); - - std::sort(children.begin(), children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs) - { - return lhs->getTreeHash() < rhs->getTreeHash(); - }); -} - -void reorder(DB::IAST * ast) -{ - if (ast == nullptr) - return; - - auto select_query = typeid_cast(ast); - if (select_query == nullptr) - return; - - reorderImpl(select_query->where().get()); - reorderImpl(select_query->prewhere().get()); - reorderImpl(select_query->having().get()); -} - -} - -int main() -{ - run(); - return 0; -} diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index 5bfbf1ed476..69c199c201e 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -56,8 +56,6 @@ public: /// Token iterator augmented with depth information. This allows to control recursion depth. struct Pos : TokenIterator { - using TokenIterator::TokenIterator; - uint32_t depth = 0; uint32_t max_depth = 0; diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index b8ab940f2dd..ff537e0f70e 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -328,19 +328,28 @@ ASTPtr parseQuery( IParser & parser, const std::string & query, const std::string & query_description, - size_t max_query_size) + size_t max_query_size, + size_t max_parser_depth) { - return parseQuery(parser, query.data(), query.data() + query.size(), query_description, max_query_size); + return parseQuery(parser, query.data(), query.data() + query.size(), query_description, max_query_size, max_parser_depth); } -ASTPtr parseQuery(IParser & parser, const std::string & query, size_t max_query_size) +ASTPtr parseQuery( + IParser & parser, + const std::string & query, + size_t max_query_size, + size_t max_parser_depth) { - return parseQuery(parser, query.data(), query.data() + query.size(), parser.getName(), max_query_size); + return parseQuery(parser, query.data(), query.data() + query.size(), parser.getName(), max_query_size, max_parser_depth); } -std::pair splitMultipartQuery(const std::string & queries, std::vector & queries_list) +std::pair splitMultipartQuery( + const std::string & queries, + std::vector & queries_list, + size_t max_query_size, + size_t max_parser_depth) { ASTPtr ast; @@ -356,7 +365,7 @@ std::pair splitMultipartQuery(const std::string & queries, s { begin = pos; - ast = parseQueryAndMovePosition(parser, pos, end, "", true, 0); + ast = parseQueryAndMovePosition(parser, pos, end, "", true, max_query_size, max_parser_depth); auto * insert = ast->as(); diff --git a/src/Parsers/parseQuery.h b/src/Parsers/parseQuery.h index feea204181e..14a9a85b22c 100644 --- a/src/Parsers/parseQuery.h +++ b/src/Parsers/parseQuery.h @@ -17,7 +17,7 @@ ASTPtr tryParseQuery( bool allow_multi_statements, /// If false, check for non-space characters after semicolon and set error message if any. size_t max_query_size, /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded". /// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query. - size_t max_parser_depth = 0); + size_t max_parser_depth); /// Parse query or throw an exception with error message. @@ -27,8 +27,8 @@ ASTPtr parseQueryAndMovePosition( const char * end, const std::string & description, bool allow_multi_statements, - size_t max_query_size = 0, - size_t max_parser_depth = 0); + size_t max_query_size, + size_t max_parser_depth); ASTPtr parseQuery( IParser & parser, @@ -36,24 +36,30 @@ ASTPtr parseQuery( const char * end, const std::string & description, size_t max_query_size, - size_t max_parser_depth = 0); + size_t max_parser_depth); ASTPtr parseQuery( IParser & parser, const std::string & query, const std::string & query_description, - size_t max_query_size); + size_t max_query_size, + size_t max_parser_depth); ASTPtr parseQuery( IParser & parser, const std::string & query, - size_t max_query_size); + size_t max_query_size, + size_t max_parser_depth); /** Split queries separated by ; on to list of single queries * Returns pointer to the end of last successfully parsed query (first), and true if all queries are successfully parsed (second) * NOTE: INSERT's data should be placed in single line. */ -std::pair splitMultipartQuery(const std::string & queries, std::vector & queries_list); +std::pair splitMultipartQuery( + const std::string & queries, + std::vector & queries_list, + size_t max_query_size, + size_t max_parser_depth); } diff --git a/src/Parsers/tests/create_parser.cpp b/src/Parsers/tests/create_parser.cpp index 6137d4d18da..fbdc967fa2a 100644 --- a/src/Parsers/tests/create_parser.cpp +++ b/src/Parsers/tests/create_parser.cpp @@ -12,7 +12,7 @@ int main(int, char **) std::string input = "CREATE TABLE hits (URL String, UserAgentMinor2 FixedString(2), EventTime DateTime) ENGINE = Log"; ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); formatAST(*ast, std::cerr); std::cerr << std::endl; diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index 25dcbb326ad..60eeab5f615 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -43,7 +43,7 @@ TEST(ParserDictionaryDDL, SimpleDictionary) " RANGE(MIN second_column MAX third_column)"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); EXPECT_EQ(create->table, "dict1"); EXPECT_EQ(create->database, "test"); @@ -139,7 +139,7 @@ TEST(ParserDictionaryDDL, AttributesWithMultipleProperties) " SOURCE(CLICKHOUSE(HOST 'localhost'))"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); EXPECT_EQ(create->table, "dict2"); EXPECT_EQ(create->database, ""); @@ -186,7 +186,7 @@ TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder) " LIFETIME(300)"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); /// test attributes @@ -241,7 +241,7 @@ TEST(ParserDictionaryDDL, NestedSource) " RANGE(MIN second_column MAX third_column)"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); EXPECT_EQ(create->table, "dict4"); EXPECT_EQ(create->database, ""); @@ -289,7 +289,7 @@ TEST(ParserDictionaryDDL, Formatting) " RANGE(MIN second_column MAX third_column)"; ParserCreateDictionaryQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); auto str = serializeAST(*create, true); EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)"); @@ -300,7 +300,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery) String input1 = "DROP DICTIONARY test.dict1"; ParserDropQuery parser; - ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0); + ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0, 0); ASTDropQuery * drop1 = ast1->as(); EXPECT_TRUE(drop1->is_dictionary); @@ -311,7 +311,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery) String input2 = "DROP DICTIONARY IF EXISTS dict2"; - ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0); + ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0, 0); ASTDropQuery * drop2 = ast2->as(); EXPECT_TRUE(drop2->is_dictionary); @@ -326,7 +326,7 @@ TEST(ParserDictionaryDDL, ParsePropertiesQueries) String input1 = "SHOW CREATE DICTIONARY test.dict1"; ParserTablePropertiesQuery parser; - ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0); + ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0, 0); ASTShowCreateDictionaryQuery * show1 = ast1->as(); EXPECT_EQ(show1->table, "dict1"); @@ -335,7 +335,7 @@ TEST(ParserDictionaryDDL, ParsePropertiesQueries) String input2 = "EXISTS DICTIONARY dict2"; - ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0); + ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0, 0); ASTExistsDictionaryQuery * show2 = ast2->as(); EXPECT_EQ(show2->table, "dict2"); diff --git a/src/Parsers/tests/select_parser.cpp b/src/Parsers/tests/select_parser.cpp index f5d94746aa1..7711f0d2527 100644 --- a/src/Parsers/tests/select_parser.cpp +++ b/src/Parsers/tests/select_parser.cpp @@ -22,7 +22,7 @@ try " FORMAT TabSeparated"; ParserQueryWithOutput parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); std::cout << "Success." << std::endl; formatAST(*ast, std::cerr); diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 76a55d059ee..c4f6d11605f 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -23,11 +23,13 @@ #include #include #include +#include #include #include #include #include + namespace DB { @@ -102,7 +104,7 @@ void ColumnDescription::readText(ReadBuffer & buf) ParserColumnDeclaration column_parser(/* require type */ true); String column_line; readEscapedStringUntilEOL(column_line, buf); - ASTPtr ast = parseQuery(column_parser, column_line, "column parser", 0); + ASTPtr ast = parseQuery(column_parser, column_line, "column parser", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); if (const auto * col_ast = ast->as()) { name = col_ast->name; diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index ad0cd76733a..8c38af0cd5e 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -5,6 +5,8 @@ #include #include +#include + namespace DB { @@ -28,7 +30,7 @@ ConstraintsDescription ConstraintsDescription::parse(const String & str) ConstraintsDescription res; ParserConstraintDeclarationList parser; - ASTPtr list = parseQuery(parser, str, 0); + ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); for (const auto & constraint : list->children) res.constraints.push_back(std::dynamic_pointer_cast(constraint)); diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index a5772a835bf..2363e7924ba 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -5,6 +5,9 @@ #include #include +#include + + namespace DB { @@ -42,7 +45,7 @@ IndicesDescription IndicesDescription::parse(const String & str) IndicesDescription res; ParserIndexDeclarationList parser; - ASTPtr list = parseQuery(parser, str, 0); + ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); for (const auto & index : list->children) res.indices.push_back(std::dynamic_pointer_cast(index)); diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index d48f8525162..f3569c344d9 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -137,7 +138,7 @@ void MutationCommands::readText(ReadBuffer & in) ParserAlterCommandList p_alter_commands; auto commands_ast = parseQuery( - p_alter_commands, commands_str.data(), commands_str.data() + commands_str.length(), "mutation commands list", 0); + p_alter_commands, commands_str.data(), commands_str.data() + commands_str.length(), "mutation commands list", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); for (ASTAlterCommand * command_ast : commands_ast->as().commands) { auto command = MutationCommand::parse(command_ast, true); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 556e7b41c52..a19a424c643 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1,4 +1,5 @@ -#include +#include + #include #include #include @@ -25,6 +26,8 @@ #include #include +#include + #include #include @@ -479,7 +482,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column if (metadata_diff.sorting_key_changed) { ParserNotEmptyExpressionList parser(false); - auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0); + auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); if (new_sorting_key_expr_list->children.size() == 1) metadata.order_by_ast = new_sorting_key_expr_list->children[0]; @@ -507,7 +510,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column if (metadata_diff.ttl_table_changed) { ParserTTLExpressionList parser; - metadata.ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0); + metadata.ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); } } diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index eb386c66d60..3b4ee489b46 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -137,7 +137,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( column.default_desc.kind = columnDefaultKindFromString(kind_name); String expr_str = (*default_expr)[i].get(); column.default_desc.expression = parseQuery( - expr_parser, expr_str.data(), expr_str.data() + expr_str.size(), "default expression", 0); + expr_parser, expr_str.data(), expr_str.data() + expr_str.size(), "default expression", 0, context.getSettingsRef().max_parser_depth); } res.add(column); diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 385e47201a5..22407ca1bd9 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -48,7 +48,7 @@ static State & state() static void check(const std::string & query, const std::string & expected, const Context & context, const NamesAndTypesList & columns) { ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, query, 1000); + ASTPtr ast = parseQuery(parser, query, 1000, 1000); SelectQueryInfo query_info; query_info.syntax_analyzer_result = SyntaxAnalyzer(context).analyzeSelect(ast, columns); query_info.query = ast; diff --git a/src/TableFunctions/parseColumnsListForTableFunction.cpp b/src/TableFunctions/parseColumnsListForTableFunction.cpp index 9b775b70492..c419164ac38 100644 --- a/src/TableFunctions/parseColumnsListForTableFunction.cpp +++ b/src/TableFunctions/parseColumnsListForTableFunction.cpp @@ -19,9 +19,7 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, con Expected expected; Tokens tokens(structure.c_str(), structure.c_str() + structure.size()); - IParser::Pos token_iterator(tokens); - const Settings & settings = context.getSettingsRef(); - token_iterator.max_depth = settings.max_parser_depth; + IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth); ParserColumnDeclarationList parser; ASTPtr columns_list_raw; From 8f6e90390f4a7a5e6dea6810fb9360ff004038b8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Apr 2020 23:32:14 +0300 Subject: [PATCH 275/743] Added a test --- .../queries/0_stateless/01196_max_parser_depth.reference | 3 +++ tests/queries/0_stateless/01196_max_parser_depth.sh | 8 ++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/01196_max_parser_depth.reference create mode 100755 tests/queries/0_stateless/01196_max_parser_depth.sh diff --git a/tests/queries/0_stateless/01196_max_parser_depth.reference b/tests/queries/0_stateless/01196_max_parser_depth.reference new file mode 100644 index 00000000000..a72c1b18aa2 --- /dev/null +++ b/tests/queries/0_stateless/01196_max_parser_depth.reference @@ -0,0 +1,3 @@ +Code: 306 +Code: 306 +Code: 306 diff --git a/tests/queries/0_stateless/01196_max_parser_depth.sh b/tests/queries/0_stateless/01196_max_parser_depth.sh new file mode 100755 index 00000000000..471c1c22ecb --- /dev/null +++ b/tests/queries/0_stateless/01196_max_parser_depth.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +{ printf "select "; for x in {1..1000}; do printf "coalesce(null, "; done; printf "1"; for x in {1..1000}; do printf ")"; done; } | $CLICKHOUSE_CLIENT 2>&1 | grep -o -F 'Code: 306' +{ printf "select "; for x in {1..1000}; do printf "coalesce(null, "; done; printf "1"; for x in {1..1000}; do printf ")"; done; } | $CLICKHOUSE_LOCAL 2>&1 | grep -o -F 'Code: 306' +{ printf "select "; for x in {1..1000}; do printf "coalesce(null, "; done; printf "1"; for x in {1..1000}; do printf ")"; done; } | $CLICKHOUSE_CURL --data-binary @- -vsS "$CLICKHOUSE_URL" 2>&1 | grep -o -F 'Code: 306' From 0221fba8734703ffe1a20683d47c305fabc47639 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Apr 2020 05:59:16 +0300 Subject: [PATCH 276/743] Fix tests --- programs/client/Client.cpp | 7 +++++-- tests/queries/0_stateless/00328_long_case_construction.sql | 1 - tests/queries/0_stateless/00990_hasToken.sh | 2 +- tests/queries/0_stateless/01068_parens.sql | 3 ++- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e4c046d18ba..b1dda4b4da9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -959,11 +959,14 @@ private: ASTPtr res; const auto & settings = context.getSettingsRef(); + size_t max_length = 0; + if (!allow_multi_statements) + max_length = settings.max_query_size; if (is_interactive || ignore_error) { String message; - res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, settings.max_query_size, settings.max_parser_depth); + res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth); if (!res) { @@ -972,7 +975,7 @@ private: } } else - res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, settings.max_query_size, settings.max_parser_depth); + res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth); if (is_interactive) { diff --git a/tests/queries/0_stateless/00328_long_case_construction.sql b/tests/queries/0_stateless/00328_long_case_construction.sql index d0fc90d0884..aa653d667f7 100644 --- a/tests/queries/0_stateless/00328_long_case_construction.sql +++ b/tests/queries/0_stateless/00328_long_case_construction.sql @@ -1,4 +1,3 @@ - /* Trivial case */ SELECT CASE WHEN 1 THEN 2 WHEN 3 THEN 4 ELSE 5 END; diff --git a/tests/queries/0_stateless/00990_hasToken.sh b/tests/queries/0_stateless/00990_hasToken.sh index 4ccb77b8ecc..4ef62bc69c0 100755 --- a/tests/queries/0_stateless/00990_hasToken.sh +++ b/tests/queries/0_stateless/00990_hasToken.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python $CURDIR/00990_hasToken.python | ${CLICKHOUSE_CLIENT} -nm +python $CURDIR/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -nm diff --git a/tests/queries/0_stateless/01068_parens.sql b/tests/queries/0_stateless/01068_parens.sql index 7cb4f097b15..42948760594 100644 --- a/tests/queries/0_stateless/01068_parens.sql +++ b/tests/queries/0_stateless/01068_parens.sql @@ -1 +1,2 @@ -((((((((((((((SELECT((((((((((((((((((((((((((((((((1)))))))))))))))))))))))))))))))))))))))))))))) +SET max_parser_depth = 10000; +((((((((((((((SELECT((((((((((((((((((((((((((((((((1)))))))))))))))))))))))))))))))))))))))))))))); From db2a307a8e9ee8191c8c2e4195112463055b7dd7 Mon Sep 17 00:00:00 2001 From: Ramazan Polat Date: Thu, 16 Apr 2020 09:51:01 +0300 Subject: [PATCH 277/743] First Turkish translation contribution of ClickHouse! (#10297) * Manual translation to TR First Turkish translation contribution of ClickHouse! * Update index.md Co-authored-by: Ivan Blinkov --- docs/tr/index.md | 52 +++++++++++++++++++++++------------------------- 1 file changed, 25 insertions(+), 27 deletions(-) diff --git a/docs/tr/index.md b/docs/tr/index.md index 0ef0f264b87..fb65930b39c 100644 --- a/docs/tr/index.md +++ b/docs/tr/index.md @@ -1,17 +1,15 @@ --- -machine_translated: true -machine_translated_rev: e8cd92bba3269f47787db090899f7c242adf7818 toc_priority: 0 toc_title: "Genel bak\u0131\u015F" --- # ClickHouse nedir? {#what-is-clickhouse} -ClickHouse, sorguların çevrimiçi analitik işlenmesi (OLAP) için sütun odaklı bir veritabanı yönetim sistemidir (DBMS). +ClickHouse, sorguların çevrimiçi analitik işlenmesi (*Online Analytical Processing* - OLAP) için sütun odaklı bir Veritabanı Yönetim Sistemidir (*DataBase Management System* - DBMS). -İn a “normal” satır yönelimli DBMS, veri bu sırayla saklanır: +“Normal” bir satır odaklı DBMS içinde veriler şu şekilde saklanır: -| Satır | Watchıd | JavaEnable | Başlık | GoodEvent | EventTime | +| Satır | WatchId | JavaEnable | Başlık | İyiOlay | OlayZamanı | |-------|-------------|------------|----------------------|-----------|---------------------| | \#0 | 89354350662 | 1 | Yatırımcı İlişkileri | 1 | 2016-05-18 05:19:20 | | \#1 | 90329509958 | 0 | Bize ulaşın | 1 | 2016-05-18 08:10:20 | @@ -20,47 +18,47 @@ ClickHouse, sorguların çevrimiçi analitik işlenmesi (OLAP) için sütun odak Başka bir deyişle, bir satırla ilgili tüm değerler fiziksel olarak yan yana depolanır. -Satır yönelimli DBMS örnekleri MySQL, Postgres ve MS SQL Server'dır. +MySQL, Postgres ve MS SQL Server gibi veritabanları satır odaklı DBMS örnekleridir. -Sütun yönelimli bir DBMS'DE, veriler şu şekilde saklanır: +Sütun odaklı bir DBMS'de ise veriler şu şekilde saklanır: | Satır: | \#0 | \#1 | \#2 | \#N | |-------------|----------------------|---------------------|---------------------|-----| -| Watchıd: | 89354350662 | 90329509958 | 89953706054 | … | +| WatchId: | 89354350662 | 90329509958 | 89953706054 | … | | JavaEnable: | 1 | 0 | 1 | … | | Başlık: | Yatırımcı İlişkileri | Bize ulaşın | Görev | … | -| GoodEvent: | 1 | 1 | 1 | … | -| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | … | +| İyiOlay: | 1 | 1 | 1 | … | +| OlayZamanı: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | … | Bu örnekler yalnızca verilerin düzenlendiği sırayı gösterir. Farklı sütunlardaki değerler ayrı olarak depolanır ve aynı sütundaki veriler birlikte depolanır. -Bir sütun odaklı DBMS örnekleri: Vertica, Paraccel (Actian Matrix ve Amazon Redshift), Sybase IQ, Exasol, Infobright, InfiniDB, MonetDB (VectorWise ve Actian vektör), LucidDB, SAP HANA, Google Dremel, Google PowerDrill, Druid ve kdb+. +Sütun odaklı DBMS örnekleri: Vertica, Paraccel (Actian Matrix ve Amazon Redshift), Sybase IQ, Exasol, Infobright, InfiniDB, MonetDB (VectorWise ve Actian vektör), LucidDB, SAP HANA, Google Dremel, Google PowerDrill, Druid ve kdb+. -Different orders for storing data are better suited to different scenarios. The data access scenario refers to what queries are made, how often, and in what proportion; how much data is read for each type of query – rows, columns, and bytes; the relationship between reading and updating data; the working size of the data and how locally it is used; whether transactions are used, and how isolated they are; requirements for data replication and logical integrity; requirements for latency and throughput for each type of query, and so on. +Verinin farklı bir şekilde sıralanarak depolanması, bazı veri erişim senaryoları için daha uygundur. Veri erişim senaryosu, hangi sorguların ne kadar sıklıkla yapıldığını, ne kadar verinin okunduğu, bunların hangi tiplerde hangi kolonlardan, satırlardan ve hangi miktarda(bayt olarak) okunacağını; verinin okunması ile güncellenmesi arasındaki ilişkiyi; verinin işlenen boyutu ve ne kadar yerel olduğunu; veri değiş-tokuşunun(transaction) olup olmayacağını, olacaksa diğer işlemlerden ne kadat yalıtılacağını; verilerin kopyalanması ve mantıksal bütünlük intiyaçlarını; her sorgu türünün gecikme ve iletim debisi ihtiyaçlarını gösterir. -Sistem üzerindeki yük ne kadar yüksek olursa, kullanım senaryosunun gereksinimlerine uyacak şekilde ayarlanmış sistemi özelleştirmek o kadar önemlidir ve bu özelleştirme o kadar ince taneli olur. Önemli ölçüde farklı senaryolara eşit derecede uygun bir sistem yoktur. Bir sistem geniş bir senaryo kümesine uyarlanabilirse, yüksek bir yük altında, sistem tüm senaryoları eşit derecede zayıf bir şekilde ele alır veya olası senaryolardan yalnızca biri veya birkaçı için iyi çalışır. +Sistem üzerindeki yük ne kadar fazlaysa, sistem ayarlarının kullanım senaryolarına uyarlanması ve bu ayarların ne kadar hassas olduğu da o kadar önemli hale gelir. Birbirinden büyük ölçüde farklı olan veri erişim senaryolarına tam uyum sağlayan, yani her işe ve yüke gelen bir sistem yoktur. Eğer bir sistem yük altında her türlü veri erişim senaryosuna adapte olabiliyorsa, o halde böyle bir sistem ya tüm senaryolara ya da senaryoların bir veya birkaçına karşı zayıp bir performans gösterir. ## OLAP senaryosunun temel özellikleri {#key-properties-of-olap-scenario} -- İsteklerin büyük çoğunluğu okuma erişimi içindir. +- İsteklerin büyük çoğunluğu, okuma erişimi içindir. - Veriler, tek satırlarla değil, oldukça büyük gruplar halinde (\> 1000 satır) güncellenir; veya hiç güncellenmez. -- Veri DB eklenir, ancak değiştirilmez. -- Okumalar için, dB'den oldukça fazla sayıda satır çıkarılır,ancak yalnızca küçük bir sütun alt kümesi. -- Tablolar şunlardır “wide,” çok sayıda sütun içerdikleri anlamına gelir. -- Sorgular nispeten nadirdir (genellikle sunucu başına yüzlerce sorgu veya saniyede daha az). +- Veri, veritabanına eklenir, ancak değiştirilmez. +- Bazı sorgular için veritabanından den oldukça fazla sayıda satır çekilir, ancak sonuç sadece birkaç satır ve sütunludur. +- Tablolar "geniştir", yani bir tabloda çok sayıda kolon vardır(onlarca). +- Sorgular sıkılığı diğer senaryolara göre daha azdır (genellikle sunucu başına saniyede 100 veya daha az sorgu gelir). - Basit sorgular için, 50 ms civarında gecikmelere izin verilir. -- Sütun değerleri oldukça küçüktür: sayılar ve kısa dizeler (örneğin, URL başına 60 bayt). -- Tek bir sorguyu işlerken yüksek verim gerektirir (sunucu başına saniyede milyarlarca satıra kadar). -- İşlemler gerekli değildir. -- Veri tutarlılığı için düşük gereksinimler. -- Sorgu başına bir büyük tablo var. Biri hariç tüm tablolar küçüktür. -- Bir sorgu sonucu, kaynak veriden önemli ölçüde daha küçüktür. Başka bir deyişle, veriler filtrelenir veya toplanır, böylece sonuç tek bir sunucunun RAM'İNE sığar. +- Saklanan veriler oldukça küçüktür: genelde sadece sayılar ve kısa metinler içerir(örneğin, URL başına 60 bayt). +- Tek bir sorguyu işlemek yüksek miktarda veri okunmasını gerektirir(sunucu başına saniyede milyarlarca satıra kadar). +- Veri değiş-tokuşu(transaction) gerekli değildir. +- Veri tutarlılığı o kadar da önemli değildir. +- Genelde bir tane çok büyük tablo vardır, gerisi küçük tablolardan oluşur +- Bir sorgu sonucu elde edilen veri, okuanan veri miktarından oldukça küçüktür. Başka bir deyişle, milyarlarca satır içinden veriler süzgeçlenerek veya birleştirilerek elde edilen verilerin tek bir sunucunun RAM'ine sığar. -OLAP senaryosunun diğer popüler senaryolardan (OLTP veya anahtar değeri erişimi gibi) çok farklı olduğunu görmek kolaydır. Bu nedenle, iyi bir performans elde etmek istiyorsanız, analitik sorguları işlemek için OLTP veya anahtar değeri DB'Yİ kullanmayı denemek mantıklı değildir. Örneğin, analitik için MongoDB veya Redis kullanmaya çalışırsanız, OLAP veritabanlarına kıyasla çok düşük performans elde edersiniz. +OLAP senaryosunun diğer popüler senaryolardan (*Online Transactional Processing* - OLTP veya *Key-Value* veritabanı) çok farklı olduğu açıkça görülebilir. Bu nedenle, iyi bir performans elde etmek istiyorsanız, analitik sorguları işlemek için OLTP veya *Key-Value* veritabanlarını kullanmak pek mantıklı olmaz. Örneğin, analitik için MongoDB veya Redis kullanmaya çalışırsanız, OLAP veritabanlarına kıyasla çok düşük performans elde edersiniz. ## Sütun yönelimli veritabanları OLAP senaryosunda neden daha iyi çalışır {#why-column-oriented-databases-work-better-in-the-olap-scenario} -Sütun yönelimli veritabanları OLAP senaryolarına daha uygundur: çoğu sorgunun işlenmesinde en az 100 kat daha hızlıdır. Nedenleri aşağıda ayrıntılı olarak açıklanmıştır, ancak gerçek görsel olarak göstermek daha kolaydır: +Sütun yönelimli veritabanları OLAP senaryolarına daha uygundur: hatta o kadar ki, çoğu sorgunun işlenmesi en az 100 kat daha hızlıdır. Her ne kadar OLAP veritabanlarının neden bu kadar hızlı olduğuna dair nedenler aşağıda ayrıntılı verilmiş olsa da görseller üzerinden anlatmak daha kolay olacakttır: **Satır yönelimli DBMS** @@ -70,7 +68,7 @@ Sütun yönelimli veritabanları OLAP senaryolarına daha uygundur: çoğu sorgu ![Column-oriented](images/column_oriented.gif#) -Farkı görüyor musun? +Farkı görüyor musunuz? ### Giriş/çıkış {#inputoutput} From 9aabff4c40ae868535b7fcb5a3232fcd174dc458 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 16 Apr 2020 09:51:39 +0300 Subject: [PATCH 278/743] Update README.md --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 3db5e08d2a9..8da2c5771a9 100644 --- a/README.md +++ b/README.md @@ -15,6 +15,5 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse Monitoring Round Table (online in English)](https://www.eventbrite.com/e/clickhouse-april-virtual-meetup-tickets-102272923066) on April 15, 2020. * [ClickHouse Workshop in Novosibirsk](https://2020.codefest.ru/lecture/1628) on TBD date. * [Yandex C++ Open-Source Sprints in Moscow](https://events.yandex.ru/events/otkrytyj-kod-v-yandek-28-03-2020) on TBD date. From f1f862a6a858f07611c285e2fda0a9b95fd11ff2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 16 Apr 2020 11:35:32 +0300 Subject: [PATCH 279/743] Update docker for stateless_with_coverage. --- docker/test/stateless_with_coverage/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 0a914030c4a..6eec6f278dc 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -50,6 +50,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/; \ + ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/; \ From ff0d04de05c7b76f8185ed39883807662c413368 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 16 Apr 2020 11:47:12 +0300 Subject: [PATCH 280/743] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8da2c5771a9..955f9d1a5d1 100644 --- a/README.md +++ b/README.md @@ -11,7 +11,7 @@ ClickHouse is an open-source column-oriented database management system that all * [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-d2zxkf9e-XyxDa_ucfPxzuH4SJIm~Ng) and [Telegram](https://telegram.me/clickhouse_en) allow to chat with ClickHouse users in real-time. * [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events. * [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any. -* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. +* You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person. ## Upcoming Events From 9b85ca0f3efd8220aaa8c9989a91198e372dcfe0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 16 Apr 2020 12:09:04 +0300 Subject: [PATCH 281/743] Update build.py --- docs/tools/build.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index 30a8d6f1eee..4f3a5378b34 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -384,7 +384,7 @@ if __name__ == '__main__': arg_parser.add_argument('--output-dir', default='build') arg_parser.add_argument('--enable-stable-releases', action='store_true') arg_parser.add_argument('--stable-releases-limit', type=int, default='4') - arg_parser.add_argument('--lts-releases-limit', type=int, default='1') + arg_parser.add_argument('--lts-releases-limit', type=int, default='2') arg_parser.add_argument('--version-prefix', type=str, default='') arg_parser.add_argument('--is-stable-release', action='store_true') arg_parser.add_argument('--skip-single-page', action='store_true') From b9c8d919aed4295bd9c5f6e33865c74f2618a418 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 16 Apr 2020 12:14:30 +0300 Subject: [PATCH 282/743] Update install.md --- docs/en/getting_started/install.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting_started/install.md b/docs/en/getting_started/install.md index 316cc5a47ef..3a9cac802fc 100644 --- a/docs/en/getting_started/install.md +++ b/docs/en/getting_started/install.md @@ -38,7 +38,7 @@ sudo apt-get update sudo apt-get install clickhouse-client clickhouse-server ``` -You can also download and install packages manually from here: https://repo.yandex.ru/clickhouse/deb/stable/main/. +You can also download and install packages manually from [here](https://repo.yandex.ru/clickhouse/deb/stable/main/). #### Packages {#packages} @@ -67,7 +67,7 @@ Then run these commands to install packages: sudo yum install clickhouse-server clickhouse-client ``` -You can also download and install packages manually from here: https://repo.clickhouse.tech/rpm/stable/x86\_64. +You can also download and install packages manually from [here](https://repo.clickhouse.tech/rpm/stable/x86_64). ### From Tgz Archives {#from-tgz-archives} From 0296931784bc7373c408328ea30b9786bf7331f7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Apr 2020 12:55:04 +0300 Subject: [PATCH 283/743] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a865f351af6..0c5076021ce 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,7 +11,6 @@ * Fix error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fixed bug where ClickHouse would throw "Unknown function lambda." error message when user tries to run ALTER UPDATE/DELETE on tables with ENGINE = Replicated*. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). -* Fix `'Cannot add column'` error while creating `range_hashed` dictionary using DDL query. Fixes [#10093](https://github.com/ClickHouse/ClickHouse/issues/10093). [#10235](https://github.com/ClickHouse/ClickHouse/pull/10235) ([alesapin](https://github.com/alesapin)). * Fixed "generateRandom" function for Date type. This fixes [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). Fix an edge case when dates with year 2106 are inserted to MergeTree tables with old-style partitioning but partitions are named with year 1970. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Convert types if the table definition of a View does not correspond to the SELECT query. This fixes [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022). [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). From 398ed4d9c27b2b521e024c670219f9d616b69ba2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Apr 2020 14:23:37 +0300 Subject: [PATCH 284/743] Fix images.json list --- docker/images.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/images.json b/docker/images.json index c7dfc82d906..434a3c7af10 100644 --- a/docker/images.json +++ b/docker/images.json @@ -1,10 +1,10 @@ { "docker/packager/deb": "yandex/clickhouse-deb-builder", "docker/packager/binary": "yandex/clickhouse-binary-builder", + "docker/test/coverage": "yandex/clickhouse-coverage", "docker/test/compatibility/centos": "yandex/clickhouse-test-old-centos", "docker/test/compatibility/ubuntu": "yandex/clickhouse-test-old-ubuntu", "docker/test/integration": "yandex/clickhouse-integration-test", - "docker/test/performance": "yandex/clickhouse-performance-test", "docker/test/performance-comparison": "yandex/clickhouse-performance-comparison", "docker/test/pvs": "yandex/clickhouse-pvs-test", "docker/test/stateful": "yandex/clickhouse-stateful-test", @@ -14,5 +14,6 @@ "docker/test/unit": "yandex/clickhouse-unit-test", "docker/test/stress": "yandex/clickhouse-stress-test", "docker/test/split_build_smoke_test": "yandex/clickhouse-split-build-smoke-test", + "docker/test/codebrowser": "yandex/clickhouse-codebrowser", "tests/integration/image": "yandex/clickhouse-integration-tests-runner" } From 8d9240d3e9f34648c0cdbfe06a86323f378c1e63 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Apr 2020 15:26:57 +0300 Subject: [PATCH 285/743] Second kill in statlesess image --- docker/test/stateless_with_coverage/run.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 6eec6f278dc..238c7567694 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -14,6 +14,11 @@ kill_clickhouse () { sleep 10 fi done + + echo "Will try to send second kill signal for sure" + kill `pgrep -u clickhouse` 2>/dev/null + sleep 5 + echo "clickhouse pids" `ps aux | grep clickhouse` | ts '%Y-%m-%d %H:%M:%S' } start_clickhouse () { From e230632645a29d917a934a1b46d2b8a81a5bb9d1 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 16 Apr 2020 15:31:57 +0300 Subject: [PATCH 286/743] Changes required for auto-sync with Arcadia --- base/common/types.h | 4 + base/common/ya.make | 35 ++ base/daemon/BaseDaemon.cpp | 16 +- base/daemon/ya.make | 14 + base/loggers/ya.make | 15 + base/pcg-random/ya.make | 5 + base/widechar_width/ya.make | 9 + base/ya.make | 4 + programs/server/HTTPHandler.cpp | 6 +- programs/server/MySQLHandler.cpp | 16 +- programs/server/MySQLHandler.h | 8 +- programs/server/MySQLHandlerFactory.h | 8 +- programs/server/Server.cpp | 21 +- programs/server/TCPHandler.cpp | 5 +- programs/server/ya.make | 30 ++ programs/ya.make | 3 + src/Access/AccessFlags.h | 8 +- src/Access/ContextAccess.cpp | 6 +- src/Access/DiskAccessStorage.cpp | 4 +- src/Access/ExtendedRoleSet.cpp | 18 +- src/Access/QuotaCache.cpp | 18 +- src/Access/RoleCache.cpp | 17 +- src/Access/RowPolicyCache.cpp | 17 +- src/Access/SettingsProfilesCache.cpp | 19 +- src/Access/ya.make | 40 ++ .../registerAggregateFunctions.cpp | 2 + src/AggregateFunctions/ya.make | 54 +++ src/Client/Connection.cpp | 9 +- src/Client/ya.make | 15 + src/Columns/Collator.cpp | 20 +- src/Columns/ya.make | 34 ++ src/Common/ClickHouseRevision.cpp | 5 +- src/Common/Exception.cpp | 5 +- src/Common/OpenSSLHelpers.h | 9 +- src/Common/OptimizedRegularExpression.h | 10 +- src/Common/QueryProfiler.cpp | 1 - src/Common/QueryProfiler.h | 7 +- src/Common/StackTrace.cpp | 7 +- src/Common/XDBCBridgeHelper.h | 5 +- src/Common/getNumberOfPhysicalCPUCores.cpp | 11 +- src/Common/new_delete.cpp | 33 +- src/Common/ya.make | 109 +++++ src/Compression/ya.make | 33 ++ src/Core/Field.h | 2 + src/Core/MySQLProtocol.h | 10 +- src/Core/NamesAndTypes.cpp | 8 +- src/Core/ya.make | 24 + src/DataStreams/ya.make | 71 +++ src/DataTypes/Native.h | 30 +- src/DataTypes/ya.make | 41 ++ src/Databases/DatabaseFactory.cpp | 11 +- src/Databases/DatabaseMySQL.cpp | 47 +- src/Databases/ya.make | 19 + src/Dictionaries/CMakeLists.txt | 7 - .../CacheDictionary_generate1.cpp | 32 ++ .../CacheDictionary_generate1.cpp.in | 22 - .../CacheDictionary_generate2.cpp | 34 ++ .../CacheDictionary_generate2.cpp.in | 23 - .../CacheDictionary_generate3.cpp | 31 ++ .../CacheDictionary_generate3.cpp.in | 20 - .../ComplexKeyCacheDictionary_generate1.cpp | 32 ++ ...ComplexKeyCacheDictionary_generate1.cpp.in | 22 - .../ComplexKeyCacheDictionary_generate2.cpp | 35 ++ ...ComplexKeyCacheDictionary_generate2.cpp.in | 25 -- .../ComplexKeyCacheDictionary_generate3.cpp | 35 ++ ...ComplexKeyCacheDictionary_generate3.cpp.in | 25 -- src/Dictionaries/HashedDictionary.h | 6 + src/Dictionaries/MongoDBBlockInputStream.cpp | 40 +- src/Dictionaries/MongoDBDictionarySource.h | 8 +- src/Dictionaries/MySQLDictionarySource.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.h | 6 +- src/Dictionaries/RedisBlockInputStream.h | 5 +- src/Dictionaries/RedisDictionarySource.h | 7 +- src/Dictionaries/XDBCDictionarySource.cpp | 5 +- src/Dictionaries/registerDictionaries.cpp | 2 + src/Dictionaries/ya.make | 63 +++ src/Disks/DiskS3.h | 4 +- src/Disks/registerDisks.cpp | 4 +- src/Disks/ya.make | 17 + src/Formats/FormatFactory.cpp | 10 +- src/Formats/MySQLBlockInputStream.cpp | 24 +- src/Formats/ProtobufColumnMatcher.h | 21 +- src/Formats/ProtobufReader.cpp | 140 +++--- src/Formats/ProtobufReader.h | 14 +- src/Formats/ProtobufSchemas.cpp | 14 +- src/Formats/ProtobufWriter.cpp | 84 ++-- src/Formats/ProtobufWriter.h | 18 +- src/Formats/ya.make | 24 + src/Functions/DivisionUtils.h | 4 +- src/Functions/FunctionBase64Conversion.h | 21 +- src/Functions/FunctionBinaryArithmetic.h | 13 +- src/Functions/FunctionIfBase.h | 4 +- src/Functions/FunctionMathBinaryFloat64.h | 5 +- src/Functions/FunctionMathUnary.h | 7 +- src/Functions/FunctionUnaryArithmetic.h | 13 +- src/Functions/FunctionsEmbeddedDictionaries.h | 5 +- src/Functions/FunctionsExternalDictionaries.h | 18 +- src/Functions/FunctionsHashing.cpp | 2 + src/Functions/FunctionsHashing.h | 45 +- src/Functions/FunctionsJSON.h | 5 +- src/Functions/FunctionsStringRegex.cpp | 15 +- src/Functions/IFunction.cpp | 13 +- src/Functions/IFunction.h | 5 +- src/Functions/RapidJSONParser.h | 14 +- src/Functions/Regexps.h | 10 +- src/Functions/SimdJSONParser.h | 15 +- src/Functions/base64Encode.cpp | 7 +- src/Functions/convertCharset.cpp | 32 +- src/Functions/intDiv.cpp | 1 - src/Functions/registerFunctions.cpp | 2 + .../registerFunctionsConsistentHashing.cpp | 4 + src/Functions/registerFunctionsGeo.cpp | 4 +- .../registerFunctionsMiscellaneous.cpp | 4 +- src/Functions/registerFunctionsString.cpp | 4 +- src/Functions/version.cpp | 4 +- src/Functions/ya.make | 418 ++++++++++++++++++ src/IO/BrotliReadBuffer.cpp | 10 +- src/IO/BrotliWriteBuffer.cpp | 10 +- src/IO/CompressionMethod.cpp | 4 +- src/IO/HTTPCommon.cpp | 19 +- src/IO/ReadWriteBufferFromHTTP.h | 6 +- src/IO/UseSSL.cpp | 6 +- src/IO/WriteBufferFromHTTPServerResponse.cpp | 5 +- src/IO/WriteBufferFromHTTPServerResponse.h | 5 +- src/IO/ya.make | 59 +++ src/Interpreters/Aggregator.cpp | 5 +- src/Interpreters/AsynchronousMetrics.cpp | 20 +- src/Interpreters/ClientInfo.cpp | 5 +- src/Interpreters/Context.h | 5 +- src/Interpreters/ExpressionActions.cpp | 5 +- src/Interpreters/ExpressionActions.h | 5 +- src/Interpreters/ExpressionJIT.h | 16 +- .../ExternalDictionariesLoader.cpp | 11 +- src/Interpreters/InterpreterSetRoleQuery.cpp | 4 +- src/Interpreters/InterpreterSystemQuery.cpp | 5 +- src/Interpreters/ya.make | 138 ++++++ src/Parsers/ASTSystemQuery.h | 5 +- src/Parsers/ya.make | 113 +++++ .../Formats/Impl/ProtobufRowInputFormat.h | 12 +- .../Formats/Impl/ProtobufRowOutputFormat.h | 16 +- src/Processors/ya.make | 100 +++++ src/Storages/IStorage.cpp | 9 +- .../MergeTree/MergeTreeIndexFullText.cpp | 3 +- src/Storages/StorageMySQL.h | 13 +- .../System/StorageSystemBuildOptions.cpp | 4 + src/Storages/registerStorages.cpp | 3 - src/Storages/registerStorages.h | 7 +- src/Storages/ya.make | 167 +++++++ src/TableFunctions/ITableFunctionXDBC.h | 5 +- src/TableFunctions/TableFunctionMySQL.cpp | 42 +- src/TableFunctions/registerTableFunctions.h | 7 +- src/TableFunctions/ya.make | 26 ++ src/ya.make | 26 ++ ya.make | 2 + 154 files changed, 2719 insertions(+), 717 deletions(-) create mode 100644 base/daemon/ya.make create mode 100644 base/loggers/ya.make create mode 100644 base/pcg-random/ya.make create mode 100644 base/widechar_width/ya.make create mode 100644 programs/server/ya.make create mode 100644 programs/ya.make create mode 100644 src/Access/ya.make create mode 100644 src/AggregateFunctions/ya.make create mode 100644 src/Client/ya.make create mode 100644 src/Columns/ya.make create mode 100644 src/Common/ya.make create mode 100644 src/Compression/ya.make create mode 100644 src/Core/ya.make create mode 100644 src/DataStreams/ya.make create mode 100644 src/DataTypes/ya.make create mode 100644 src/Databases/ya.make create mode 100644 src/Dictionaries/CacheDictionary_generate1.cpp delete mode 100644 src/Dictionaries/CacheDictionary_generate1.cpp.in create mode 100644 src/Dictionaries/CacheDictionary_generate2.cpp delete mode 100644 src/Dictionaries/CacheDictionary_generate2.cpp.in create mode 100644 src/Dictionaries/CacheDictionary_generate3.cpp delete mode 100644 src/Dictionaries/CacheDictionary_generate3.cpp.in create mode 100644 src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp delete mode 100644 src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in create mode 100644 src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp delete mode 100644 src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in create mode 100644 src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp delete mode 100644 src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in create mode 100644 src/Dictionaries/ya.make create mode 100644 src/Disks/ya.make create mode 100644 src/Formats/ya.make create mode 100644 src/Functions/ya.make create mode 100644 src/IO/ya.make create mode 100644 src/Interpreters/ya.make create mode 100644 src/Parsers/ya.make create mode 100644 src/Processors/ya.make create mode 100644 src/Storages/ya.make create mode 100644 src/TableFunctions/ya.make create mode 100644 src/ya.make diff --git a/base/common/types.h b/base/common/types.h index 8f125146add..238e4e3956b 100644 --- a/base/common/types.h +++ b/base/common/types.h @@ -11,6 +11,10 @@ using Int16 = int16_t; using Int32 = int32_t; using Int64 = int64_t; +#if __cplusplus <= 201703L +using char8_t = unsigned char; +#endif + using UInt8 = char8_t; using UInt16 = uint16_t; using UInt32 = uint32_t; diff --git a/base/common/ya.make b/base/common/ya.make index a41c8f4c583..31a66893330 100644 --- a/base/common/ya.make +++ b/base/common/ya.make @@ -1,12 +1,47 @@ LIBRARY() +ADDINCL( + GLOBAL clickhouse/base + contrib/libs/cctz/include +) + +CFLAGS (GLOBAL -DARCADIA_BUILD) + +IF (OS_DARWIN) + CFLAGS (GLOBAL -DOS_DARWIN) +ELSEIF (OS_FREEBSD) + CFLAGS (GLOBAL -DOS_FREEBSD) +ELSEIF (OS_LINUX) + CFLAGS (GLOBAL -DOS_LINUX) +ENDIF () + PEERDIR( + contrib/libs/cctz/src + contrib/libs/cxxsupp/libcxx-filesystem + contrib/libs/poco/Net contrib/libs/poco/Util + contrib/restricted/boost + contrib/restricted/cityhash-1.0.2 ) SRCS( argsToConfig.cpp coverage.cpp + DateLUT.cpp + DateLUTImpl.cpp + demangle.cpp + getFQDNOrHostName.cpp + getMemoryAmount.cpp + getThreadId.cpp + JSON.cpp + LineReader.cpp + mremap.cpp + phdr_cache.cpp + preciseExp10.c + setTerminalEcho.cpp + shift10.cpp + sleep.cpp + terminalColors.cpp ) END() diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index c150dc03014..74df1a41a88 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -50,11 +50,13 @@ #include #include #include -#include -#ifdef __APPLE__ -// ucontext is not available without _XOPEN_SOURCE -#define _XOPEN_SOURCE 700 +#if !defined(ARCADIA_BUILD) +# include +#endif + +#if defined(OS_DARWIN) +# define _XOPEN_SOURCE 700 // ucontext is not available without _XOPEN_SOURCE #endif #include @@ -410,7 +412,7 @@ std::string BaseDaemon::getDefaultCorePath() const void BaseDaemon::closeFDs() { -#if defined(__FreeBSD__) || (defined(__APPLE__) && defined(__MACH__)) +#if defined(OS_FREEBSD) || defined(OS_DARWIN) Poco::File proc_path{"/dev/fd"}; #else Poco::File proc_path{"/proc/self/fd"}; @@ -430,7 +432,7 @@ void BaseDaemon::closeFDs() else { int max_fd = -1; -#ifdef _SC_OPEN_MAX +#if defined(_SC_OPEN_MAX) max_fd = sysconf(_SC_OPEN_MAX); if (max_fd == -1) #endif @@ -448,7 +450,7 @@ namespace /// the maximum is 1000, and chromium uses 300 for its tab processes. Ignore /// whatever errors that occur, because it's just a debugging aid and we don't /// care if it breaks. -#if defined(__linux__) && !defined(NDEBUG) +#if defined(OS_LINUX) && !defined(NDEBUG) void debugIncreaseOOMScore() { const std::string new_score = "555"; diff --git a/base/daemon/ya.make b/base/daemon/ya.make new file mode 100644 index 00000000000..1c72af3ed53 --- /dev/null +++ b/base/daemon/ya.make @@ -0,0 +1,14 @@ +LIBRARY() + +NO_COMPILER_WARNINGS() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + BaseDaemon.cpp + GraphiteWriter.cpp +) + +END() diff --git a/base/loggers/ya.make b/base/loggers/ya.make new file mode 100644 index 00000000000..b1c84042eee --- /dev/null +++ b/base/loggers/ya.make @@ -0,0 +1,15 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + ExtendedLogChannel.cpp + Loggers.cpp + OwnFormattingChannel.cpp + OwnPatternFormatter.cpp + OwnSplitChannel.cpp +) + +END() diff --git a/base/pcg-random/ya.make b/base/pcg-random/ya.make new file mode 100644 index 00000000000..c6a50887178 --- /dev/null +++ b/base/pcg-random/ya.make @@ -0,0 +1,5 @@ +LIBRARY() + +ADDINCL (GLOBAL clickhouse/base/pcg-random) + +END() diff --git a/base/widechar_width/ya.make b/base/widechar_width/ya.make new file mode 100644 index 00000000000..fa0b4f705db --- /dev/null +++ b/base/widechar_width/ya.make @@ -0,0 +1,9 @@ +LIBRARY() + +ADDINCL(GLOBAL clickhouse/base/widechar_width) + +SRCS( + widechar_width.cpp +) + +END() diff --git a/base/ya.make b/base/ya.make index 25ab5886b2c..004da9af2ae 100644 --- a/base/ya.make +++ b/base/ya.make @@ -1,3 +1,7 @@ RECURSE( common + daemon + loggers + pcg-random + widechar_width ) diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index ec890c0a96d..77236210741 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -36,6 +35,11 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + + namespace DB { diff --git a/programs/server/MySQLHandler.cpp b/programs/server/MySQLHandler.cpp index bfab19061ce..97d9f1919ac 100644 --- a/programs/server/MySQLHandler.cpp +++ b/programs/server/MySQLHandler.cpp @@ -1,10 +1,8 @@ -#include - #include "MySQLHandler.h" + #include #include #include -#include #include #include #include @@ -18,11 +16,15 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_NETSSL -#include -#include -#include -#include +# include +# include +# include +# include #endif namespace DB diff --git a/programs/server/MySQLHandler.h b/programs/server/MySQLHandler.h index ca5d045beb0..01634dd68dc 100644 --- a/programs/server/MySQLHandler.h +++ b/programs/server/MySQLHandler.h @@ -1,13 +1,17 @@ #pragma once -#include + #include #include #include #include #include "IServer.h" +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_NETSSL -#include +# include #endif namespace CurrentMetrics diff --git a/programs/server/MySQLHandlerFactory.h b/programs/server/MySQLHandlerFactory.h index b7df9fa60e0..74f0bb35a40 100644 --- a/programs/server/MySQLHandlerFactory.h +++ b/programs/server/MySQLHandlerFactory.h @@ -1,11 +1,15 @@ #pragma once -#include #include #include #include "IServer.h" + +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_SSL -#include +# include #endif namespace DB diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5322514e7c2..3490ff6a445 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -26,7 +25,6 @@ #include #include #include -#include "config_core.h" #include #include #include @@ -59,19 +57,24 @@ #include "MetricsTransmitter.h" #include #include "TCPHandlerFactory.h" -#include "Common/config_version.h" #include #include #include "MySQLHandlerFactory.h" +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +# include "Common/config_version.h" +#endif + #if defined(OS_LINUX) -#include -#include +# include +# include #endif #if USE_POCO_NETSSL -#include -#include +# include +# include #endif namespace CurrentMetrics @@ -248,7 +251,7 @@ int Server::main(const std::vector & /*args*/) const auto memory_amount = getMemoryAmount(); -#if defined(__linux__) +#if defined(OS_LINUX) std::string executable_path = getExecutablePath(); if (executable_path.empty()) executable_path = "/usr/bin/clickhouse"; /// It is used for information messages. @@ -631,7 +634,7 @@ int Server::main(const std::vector & /*args*/) dns_cache_updater = std::make_unique(*global_context, config().getInt("dns_cache_update_period", 15)); } -#if defined(__linux__) +#if defined(OS_LINUX) if (!TaskStatsInfoGetter::checkPermissions()) { LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, 'taskstats' performance statistics will be disabled." diff --git a/programs/server/TCPHandler.cpp b/programs/server/TCPHandler.cpp index d82c6e31528..70825cb4f7e 100644 --- a/programs/server/TCPHandler.cpp +++ b/programs/server/TCPHandler.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -33,6 +32,10 @@ #include "TCPHandler.h" +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/programs/server/ya.make b/programs/server/ya.make new file mode 100644 index 00000000000..7f0a3969fd2 --- /dev/null +++ b/programs/server/ya.make @@ -0,0 +1,30 @@ +PROGRAM(clickhouse-server) + +PEERDIR( + clickhouse/base/common + clickhouse/base/daemon + clickhouse/base/loggers + clickhouse/src + contrib/libs/poco/NetSSL_OpenSSL +) + +SRCS( + clickhouse-server.cpp + + HTTPHandler.cpp + HTTPHandlerFactory.cpp + InterserverIOHTTPHandler.cpp + MetricsTransmitter.cpp + MySQLHandler.cpp + MySQLHandlerFactory.cpp + NotFoundHandler.cpp + PingRequestHandler.cpp + PrometheusMetricsWriter.cpp + PrometheusRequestHandler.cpp + ReplicasStatusHandler.cpp + RootRequestHandler.cpp + Server.cpp + TCPHandler.cpp +) + +END() diff --git a/programs/ya.make b/programs/ya.make new file mode 100644 index 00000000000..6c773c312b8 --- /dev/null +++ b/programs/ya.make @@ -0,0 +1,3 @@ +RECURSE( + server +) diff --git a/src/Access/AccessFlags.h b/src/Access/AccessFlags.h index c8f57fcd419..cbba295be1a 100644 --- a/src/Access/AccessFlags.h +++ b/src/Access/AccessFlags.h @@ -253,7 +253,7 @@ private: } else { - if (nodes.contains(keyword)) + if (nodes.count(keyword)) throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR); node = std::make_unique(keyword, node_type); nodes[node->keyword] = node.get(); @@ -279,7 +279,7 @@ private: { auto parent_node = std::make_unique(parent_keyword); it_parent = nodes.emplace(parent_node->keyword, parent_node.get()).first; - assert(!owned_nodes.contains(parent_node->keyword)); + assert(!owned_nodes.count(parent_node->keyword)); std::string_view parent_keyword_as_string_view = parent_node->keyword; owned_nodes[parent_keyword_as_string_view] = std::move(parent_node); } @@ -299,9 +299,9 @@ private: #undef MAKE_ACCESS_FLAGS_TO_KEYWORD_TREE_NODE - if (!owned_nodes.contains("NONE")) + if (!owned_nodes.count("NONE")) throw Exception("'NONE' not declared", ErrorCodes::LOGICAL_ERROR); - if (!owned_nodes.contains("ALL")) + if (!owned_nodes.count("ALL")) throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR); flags_to_keyword_tree = std::move(owned_nodes["ALL"]); diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 915593f58f0..ab504e32579 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -147,9 +147,9 @@ void ContextAccess::setUser(const UserPtr & user_) const current_roles.reserve(params.current_roles.size()); for (const auto & id : params.current_roles) { - if (user->granted_roles.contains(id)) + if (user->granted_roles.count(id)) current_roles.push_back(id); - if (user->granted_roles_with_admin_option.contains(id)) + if (user->granted_roles_with_admin_option.count(id)) current_roles_with_admin_option.push_back(id); } } @@ -358,7 +358,7 @@ void ContextAccess::checkAdminOption(const UUID & role_id) const return; auto roles_with_admin_option_loaded = roles_with_admin_option.load(); - if (roles_with_admin_option_loaded && roles_with_admin_option_loaded->contains(role_id)) + if (roles_with_admin_option_loaded && roles_with_admin_option_loaded->count(role_id)) return; std::optional role_name = manager->readName(role_id); diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 12c65e7df1e..263f4e82d95 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -560,7 +560,7 @@ std::vector DiskAccessStorage::findAllImpl(std::type_index type) const bool DiskAccessStorage::existsImpl(const UUID & id) const { std::lock_guard lock{mutex}; - return id_to_entry_map.contains(id); + return id_to_entry_map.count(id); } @@ -709,7 +709,7 @@ void DiskAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & update_ if (name_changed) { const auto & name_to_id_map = name_to_id_maps.at(type); - if (name_to_id_map.contains(new_name)) + if (name_to_id_map.count(new_name)) throwNameCollisionCannotRename(type, String{old_name}, new_name); scheduleWriteLists(type); } diff --git a/src/Access/ExtendedRoleSet.cpp b/src/Access/ExtendedRoleSet.cpp index eed475bc3cc..145bd0fe7e5 100644 --- a/src/Access/ExtendedRoleSet.cpp +++ b/src/Access/ExtendedRoleSet.cpp @@ -253,44 +253,44 @@ void ExtendedRoleSet::add(const boost::container::flat_set & ids_) bool ExtendedRoleSet::match(const UUID & id) const { - return (all || ids.contains(id)) && !except_ids.contains(id); + return (all || ids.count(id)) && !except_ids.count(id); } bool ExtendedRoleSet::match(const UUID & user_id, const std::vector & enabled_roles) const { - if (!all && !ids.contains(user_id)) + if (!all && !ids.count(user_id)) { bool found_enabled_role = std::any_of( - enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return ids.contains(enabled_role); }); + enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return ids.count(enabled_role); }); if (!found_enabled_role) return false; } - if (except_ids.contains(user_id)) + if (except_ids.count(user_id)) return false; bool in_except_list = std::any_of( - enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return except_ids.contains(enabled_role); }); + enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return except_ids.count(enabled_role); }); return !in_except_list; } bool ExtendedRoleSet::match(const UUID & user_id, const boost::container::flat_set & enabled_roles) const { - if (!all && !ids.contains(user_id)) + if (!all && !ids.count(user_id)) { bool found_enabled_role = std::any_of( - enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return ids.contains(enabled_role); }); + enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return ids.count(enabled_role); }); if (!found_enabled_role) return false; } - if (except_ids.contains(user_id)) + if (except_ids.count(user_id)) return false; bool in_except_list = std::any_of( - enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return except_ids.contains(enabled_role); }); + enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return except_ids.count(enabled_role); }); return !in_except_list; } diff --git a/src/Access/QuotaCache.cpp b/src/Access/QuotaCache.cpp index 6db3eb66c5d..96cd6b3e8bb 100644 --- a/src/Access/QuotaCache.cpp +++ b/src/Access/QuotaCache.cpp @@ -250,16 +250,18 @@ void QuotaCache::quotaRemoved(const UUID & quota_id) void QuotaCache::chooseQuotaToConsume() { /// `mutex` is already locked. - std::erase_if( - enabled_quotas, - [&](const std::pair> & pr) + + for (auto i = enabled_quotas.begin(), e = enabled_quotas.end(); i != e;) + { + auto elem = i->second.lock(); + if (!elem) + i = enabled_quotas.erase(i); + else { - auto elem = pr.second.lock(); - if (!elem) - return true; // remove from the `enabled_quotas` list. chooseQuotaToConsumeFor(*elem); - return false; // keep in the `enabled_quotas` list. - }); + ++i; + } + } } void QuotaCache::chooseQuotaToConsumeFor(EnabledQuota & enabled) diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index 63e19a3cb40..0263b793017 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -103,16 +103,17 @@ void RoleCache::collectRolesInfo() { /// `mutex` is already locked. - std::erase_if( - enabled_roles, - [&](const std::pair> & pr) + for (auto i = enabled_roles.begin(), e = enabled_roles.end(); i != e;) + { + auto elem = i->second.lock(); + if (!elem) + i = enabled_roles.erase(i); + else { - auto elem = pr.second.lock(); - if (!elem) - return true; // remove from the `enabled_roles` map. collectRolesInfoFor(*elem); - return false; // keep in the `enabled_roles` map. - }); + ++i; + } + } } diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index 44f2cd160d4..04b4202c211 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -178,16 +178,17 @@ void RowPolicyCache::rowPolicyRemoved(const UUID & policy_id) void RowPolicyCache::mixConditions() { /// `mutex` is already locked. - std::erase_if( - enabled_row_policies, - [&](const std::pair> & pr) + for (auto i = enabled_row_policies.begin(), e = enabled_row_policies.end(); i != e;) + { + auto elem = i->second.lock(); + if (!elem) + i = enabled_row_policies.erase(i); + else { - auto elem = pr.second.lock(); - if (!elem) - return true; // remove from the `enabled_row_policies` map. mixConditionsFor(*elem); - return false; // keep in the `enabled_row_policies` map. - }); + ++i; + } + } } diff --git a/src/Access/SettingsProfilesCache.cpp b/src/Access/SettingsProfilesCache.cpp index 552ed324635..f283715e129 100644 --- a/src/Access/SettingsProfilesCache.cpp +++ b/src/Access/SettingsProfilesCache.cpp @@ -104,16 +104,17 @@ void SettingsProfilesCache::setDefaultProfileName(const String & default_profile void SettingsProfilesCache::mergeSettingsAndConstraints() { /// `mutex` is already locked. - std::erase_if( - enabled_settings, - [&](const std::pair> & pr) + for (auto i = enabled_settings.begin(), e = enabled_settings.end(); i != e;) + { + auto enabled = i->second.lock(); + if (!enabled) + i = enabled_settings.erase(i); + else { - auto enabled = pr.second.lock(); - if (!enabled) - return true; // remove from the `enabled_settings` list. mergeSettingsAndConstraintsFor(*enabled); - return false; // keep in the `enabled_settings` list. - }); + ++i; + } + } } @@ -161,7 +162,7 @@ void SettingsProfilesCache::substituteProfiles(SettingsProfileElements & element auto parent_profile_id = *element.parent_profile; element.parent_profile.reset(); - if (already_substituted.contains(parent_profile_id)) + if (already_substituted.count(parent_profile_id)) continue; already_substituted.insert(parent_profile_id); diff --git a/src/Access/ya.make b/src/Access/ya.make new file mode 100644 index 00000000000..fb2e23e0684 --- /dev/null +++ b/src/Access/ya.make @@ -0,0 +1,40 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + AccessControlManager.cpp + AccessRights.cpp + AccessRightsElement.cpp + AllowedClientHosts.cpp + Authentication.cpp + ContextAccess.cpp + DiskAccessStorage.cpp + EnabledQuota.cpp + EnabledRoles.cpp + EnabledRolesInfo.cpp + EnabledRowPolicies.cpp + EnabledSettings.cpp + ExtendedRoleSet.cpp + IAccessEntity.cpp + IAccessStorage.cpp + MemoryAccessStorage.cpp + MultipleAccessStorage.cpp + Quota.cpp + QuotaCache.cpp + QuotaUsageInfo.cpp + Role.cpp + RoleCache.cpp + RowPolicy.cpp + RowPolicyCache.cpp + SettingsConstraints.cpp + SettingsProfile.cpp + SettingsProfileElement.cpp + SettingsProfilesCache.cpp + User.cpp + UsersConfigAccessStorage.cpp +) + +END() diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index a4fc41e9c06..a9ab1d4f8ea 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -32,7 +32,9 @@ void registerAggregateFunctions() registerAggregateFunctionUniqUpTo(factory); registerAggregateFunctionTopK(factory); registerAggregateFunctionsBitwise(factory); +#if !defined(ARCADIA_BUILD) registerAggregateFunctionsBitmap(factory); +#endif registerAggregateFunctionsMaxIntersections(factory); registerAggregateFunctionHistogram(factory); registerAggregateFunctionRetention(factory); diff --git a/src/AggregateFunctions/ya.make b/src/AggregateFunctions/ya.make new file mode 100644 index 00000000000..bfa32b6dd78 --- /dev/null +++ b/src/AggregateFunctions/ya.make @@ -0,0 +1,54 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + AggregateFunctionAggThrow.cpp + AggregateFunctionArray.cpp + AggregateFunctionAvg.cpp + AggregateFunctionAvgWeighted.cpp + AggregateFunctionBitwise.cpp + AggregateFunctionBoundingRatio.cpp + AggregateFunctionCategoricalInformationValue.cpp + AggregateFunctionCombinatorFactory.cpp + AggregateFunctionCount.cpp + AggregateFunctionEntropy.cpp + AggregateFunctionFactory.cpp + AggregateFunctionForEach.cpp + AggregateFunctionGroupArray.cpp + AggregateFunctionGroupArrayInsertAt.cpp + AggregateFunctionGroupArrayMoving.cpp + AggregateFunctionGroupUniqArray.cpp + AggregateFunctionHistogram.cpp + AggregateFunctionIf.cpp + AggregateFunctionMaxIntersections.cpp + AggregateFunctionMerge.cpp + AggregateFunctionMinMaxAny.cpp + AggregateFunctionMLMethod.cpp + AggregateFunctionNull.cpp + AggregateFunctionOrFill.cpp + AggregateFunctionQuantile.cpp + AggregateFunctionResample.cpp + AggregateFunctionRetention.cpp + AggregateFunctionSequenceMatch.cpp + AggregateFunctionSimpleLinearRegression.cpp + AggregateFunctionState.cpp + AggregateFunctionStatistics.cpp + AggregateFunctionStatisticsSimple.cpp + AggregateFunctionSum.cpp + AggregateFunctionSumMap.cpp + AggregateFunctionTimeSeriesGroupSum.cpp + AggregateFunctionTopK.cpp + AggregateFunctionUniq.cpp + AggregateFunctionUniqCombined.cpp + AggregateFunctionUniqUpTo.cpp + AggregateFunctionWindowFunnel.cpp + parseAggregateFunctionParameters.cpp + registerAggregateFunctions.cpp + UniqCombinedBiasData.cpp + UniqVariadicHash.cpp +) + +END() diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index f530652caae..cc79e3ec2c8 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -19,16 +19,19 @@ #include #include #include -#include #include #include #include #include #include -#include +#if !defined(ARCADIA_BUILD) +# include +# include +#endif + #if USE_POCO_NETSSL -#include +# include #endif namespace CurrentMetrics diff --git a/src/Client/ya.make b/src/Client/ya.make new file mode 100644 index 00000000000..685e0540ad7 --- /dev/null +++ b/src/Client/ya.make @@ -0,0 +1,15 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/poco/NetSSL_OpenSSL +) + +SRCS( + Connection.cpp + ConnectionPoolWithFailover.cpp + MultiplexedConnections.cpp + TimeoutSetter.cpp +) + +END() diff --git a/src/Columns/Collator.cpp b/src/Columns/Collator.cpp index c3915125fa1..77735564077 100644 --- a/src/Columns/Collator.cpp +++ b/src/Columns/Collator.cpp @@ -1,17 +1,19 @@ #include -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif #if USE_ICU - #include - #include - #include - #include +# include +# include +# include +# include #else - #ifdef __clang__ - #pragma clang diagnostic ignored "-Wunused-private-field" - #pragma clang diagnostic ignored "-Wmissing-noreturn" - #endif +# if defined(__clang__) +# pragma clang diagnostic ignored "-Wunused-private-field" +# pragma clang diagnostic ignored "-Wmissing-noreturn" +# endif #endif #include diff --git a/src/Columns/ya.make b/src/Columns/ya.make new file mode 100644 index 00000000000..b36a524e775 --- /dev/null +++ b/src/Columns/ya.make @@ -0,0 +1,34 @@ +LIBRARY() + +ADDINCL( + contrib/libs/icu/common + contrib/libs/icu/i18n + contrib/libs/pdqsort +) + +PEERDIR( + clickhouse/src/Common + contrib/libs/icu + contrib/libs/pdqsort +) + +SRCS( + Collator.cpp + ColumnAggregateFunction.cpp + ColumnArray.cpp + ColumnConst.cpp + ColumnDecimal.cpp + ColumnFixedString.cpp + ColumnFunction.cpp + ColumnLowCardinality.cpp + ColumnNullable.cpp + ColumnsCommon.cpp + ColumnString.cpp + ColumnTuple.cpp + ColumnVector.cpp + FilterDescription.cpp + getLeastSuperColumn.cpp + IColumn.cpp +) + +END() diff --git a/src/Common/ClickHouseRevision.cpp b/src/Common/ClickHouseRevision.cpp index 5513922a655..0b81026adca 100644 --- a/src/Common/ClickHouseRevision.cpp +++ b/src/Common/ClickHouseRevision.cpp @@ -1,5 +1,8 @@ #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif namespace ClickHouseRevision { diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index bc3d4a78969..edeb3be069a 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -8,11 +8,14 @@ #include #include #include -#include #include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/src/Common/OpenSSLHelpers.h b/src/Common/OpenSSLHelpers.h index 8155ffd30ef..e77fc3037c1 100644 --- a/src/Common/OpenSSLHelpers.h +++ b/src/Common/OpenSSLHelpers.h @@ -1,8 +1,11 @@ #pragma once -#include -#if USE_SSL -#include +#if !defined(ARCADIA_BUILD) +# include +#endif + +#if USE_SSL +# include namespace DB diff --git a/src/Common/OptimizedRegularExpression.h b/src/Common/OptimizedRegularExpression.h index 0c31558cecb..fddefe596c4 100644 --- a/src/Common/OptimizedRegularExpression.h +++ b/src/Common/OptimizedRegularExpression.h @@ -5,12 +5,16 @@ #include #include #include -#include #include + +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_RE2_ST - #include +# include #else - #define re2_st re2 +# define re2_st re2 #endif diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index dd9f36fb3ae..a78c2b2ea6c 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include diff --git a/src/Common/QueryProfiler.h b/src/Common/QueryProfiler.h index 17280372237..f00a081972e 100644 --- a/src/Common/QueryProfiler.h +++ b/src/Common/QueryProfiler.h @@ -1,11 +1,14 @@ #pragma once #include -#include -#include #include #include +#if !defined(ARCADIA_BUILD) +# include +# include +#endif + namespace Poco { diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 785a97e4d74..48633ef01c1 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -14,8 +13,12 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_UNWIND -# include +# include #endif std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context) diff --git a/src/Common/XDBCBridgeHelper.h b/src/Common/XDBCBridgeHelper.h index b9d1f2cdcdf..41aeb421394 100644 --- a/src/Common/XDBCBridgeHelper.h +++ b/src/Common/XDBCBridgeHelper.h @@ -13,10 +13,13 @@ #include #include #include -#include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { namespace ErrorCodes diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index 32b70b76fbd..1eaa64ae2a4 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -1,11 +1,16 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) +# include +#else +# include +#endif + #if USE_CPUID -# include +# include #elif USE_CPUINFO -# include +# include #endif diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index 0aa5f8aacce..ed0d99679e4 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -14,14 +13,13 @@ /// Replace default new/delete with memory tracking versions. /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new /// https://en.cppreference.com/w/cpp/memory/new/operator_delete -#if !UNBUNDLED namespace Memory { inline ALWAYS_INLINE void trackMemory(std::size_t size) { -#if USE_JEMALLOC +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 /// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function /// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic. if (likely(size != 0)) @@ -49,18 +47,18 @@ inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t { try { -#if USE_JEMALLOC +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 /// @note It's also possible to use je_malloc_usable_size() here. if (likely(ptr != nullptr)) CurrentMemoryTracker::free(sallocx(ptr, 0)); #else if (size) CurrentMemoryTracker::free(size); -# ifdef _GNU_SOURCE +# if defined(_GNU_SOURCE) /// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size. else CurrentMemoryTracker::free(malloc_usable_size(ptr)); -# endif +# endif #endif } catch (...) @@ -130,26 +128,3 @@ void operator delete[](void * ptr, std::size_t size) noexcept Memory::untrackMemory(ptr, size); Memory::deleteSized(ptr, size); } - -#else - -/// new - -void * operator new(std::size_t size) { return Memory::newImpl(size); } -void * operator new[](std::size_t size) { return Memory::newImpl(size); } - -void * operator new(std::size_t size, const std::nothrow_t &) noexcept { return Memory::newNoExept(size); } -void * operator new[](std::size_t size, const std::nothrow_t &) noexcept { return Memory::newNoExept(size); } - -/// delete - -void operator delete(void * ptr) noexcept { Memory::deleteImpl(ptr); } -void operator delete[](void * ptr) noexcept { Memory::deleteImpl(ptr); } - -void operator delete(void * ptr, const std::nothrow_t &) noexcept { Memory::deleteImpl(ptr); } -void operator delete[](void * ptr, const std::nothrow_t &) noexcept { Memory::deleteImpl(ptr); } - -void operator delete(void * ptr, std::size_t size) noexcept { Memory::deleteSized(ptr, size); } -void operator delete[](void * ptr, std::size_t size) noexcept { Memory::deleteSized(ptr, size); } - -#endif diff --git a/src/Common/ya.make b/src/Common/ya.make new file mode 100644 index 00000000000..53155ac471f --- /dev/null +++ b/src/Common/ya.make @@ -0,0 +1,109 @@ +LIBRARY() + +ADDINCL ( + GLOBAL clickhouse/src + contrib/libs/libcpuid + contrib/libs/libunwind/include + GLOBAL contrib/restricted/ryu +) + +PEERDIR( + clickhouse/base/common + clickhouse/base/pcg-random + clickhouse/base/widechar_width + contrib/libs/libcpuid/libcpuid + contrib/libs/openssl + contrib/libs/re2 + contrib/restricted/ryu +) + +# TODO: stub for config_version.h +CFLAGS (GLOBAL -DDBMS_NAME=\"ClickHouse\") +CFLAGS (GLOBAL -DDBMS_VERSION_MAJOR=0) +CFLAGS (GLOBAL -DDBMS_VERSION_MINOR=0) +CFLAGS (GLOBAL -DDBMS_VERSION_PATCH=0) +CFLAGS (GLOBAL -DVERSION_FULL=\"Clickhouse\") +CFLAGS (GLOBAL -DVERSION_INTEGER=0) +CFLAGS (GLOBAL -DVERSION_NAME=\"Clickhouse\") +CFLAGS (GLOBAL -DVERSION_OFFICIAL=\"\\\(arcadia\\\)\") +CFLAGS (GLOBAL -DVERSION_REVISION=0) +CFLAGS (GLOBAL -DVERSION_STRING=\"Unknown\") + +SRCS( + ActionLock.cpp + AlignedBuffer.cpp + checkStackSize.cpp + ClickHouseRevision.cpp + Config/AbstractConfigurationComparison.cpp + Config/ConfigProcessor.cpp + Config/configReadClient.cpp + Config/ConfigReloader.cpp + createHardLink.cpp + CurrentMetrics.cpp + CurrentThread.cpp + DNSResolver.cpp + Dwarf.cpp + Elf.cpp + ErrorCodes.cpp + escapeForFileName.cpp + Exception.cpp + ExternalLoaderStatus.cpp + FieldVisitors.cpp + FileChecker.cpp + filesystemHelpers.cpp + formatIPv6.cpp + formatReadable.cpp + getExecutablePath.cpp + getMultipleKeysFromConfig.cpp + getNumberOfPhysicalCPUCores.cpp + hasLinuxCapability.cpp + hex.cpp + IntervalKind.cpp + IPv6ToBinary.cpp + isLocalAddress.cpp + Macros.cpp + malloc.cpp + MemoryTracker.cpp + new_delete.cpp + OptimizedRegularExpression.cpp + parseAddress.cpp + parseGlobs.cpp + parseRemoteDescription.cpp + PipeFDs.cpp + PODArray.cpp + ProfileEvents.cpp + QueryProfiler.cpp + quoteString.cpp + randomSeed.cpp + RemoteHostFilter.cpp + RWLock.cpp + SensitiveDataMasker.cpp + setThreadName.cpp + SharedLibrary.cpp + ShellCommand.cpp + StackTrace.cpp + StatusFile.cpp + StatusInfo.cpp + Stopwatch.cpp + StringUtils/StringUtils.cpp + StudentTTest.cpp + SymbolIndex.cpp + TaskStatsInfoGetter.cpp + TerminalSize.cpp + thread_local_rng.cpp + ThreadFuzzer.cpp + ThreadPool.cpp + ThreadStatus.cpp + TraceCollector.cpp + UTF8Helpers.cpp + WeakHash.cpp + ZooKeeper/IKeeper.cpp + ZooKeeper/Lock.cpp + ZooKeeper/TestKeeper.cpp + ZooKeeper/ZooKeeper.cpp + ZooKeeper/ZooKeeperHolder.cpp + ZooKeeper/ZooKeeperImpl.cpp + ZooKeeper/ZooKeeperNodeCache.cpp +) + +END() diff --git a/src/Compression/ya.make b/src/Compression/ya.make new file mode 100644 index 00000000000..55b5287e3f3 --- /dev/null +++ b/src/Compression/ya.make @@ -0,0 +1,33 @@ +LIBRARY() + +ADDINCL( + contrib/libs/lz4 + contrib/libs/zstd +) + +PEERDIR( + clickhouse/src/Common + contrib/libs/lz4 + contrib/libs/zstd +) + +SRCS( + CachedCompressedReadBuffer.cpp + CompressedReadBuffer.cpp + CompressedReadBufferBase.cpp + CompressedReadBufferFromFile.cpp + CompressedWriteBuffer.cpp + CompressionCodecDelta.cpp + CompressionCodecDoubleDelta.cpp + CompressionCodecGorilla.cpp + CompressionCodecLZ4.cpp + CompressionCodecMultiple.cpp + CompressionCodecNone.cpp + CompressionCodecT64.cpp + CompressionCodecZSTD.cpp + CompressionFactory.cpp + ICompressionCodec.cpp + LZ4_decompress_faster.cpp +) + +END() diff --git a/src/Core/Field.h b/src/Core/Field.h index 5e62e4ef3e4..152ae29bd1e 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -157,7 +157,9 @@ private: template <> struct NearestFieldTypeImpl { using Type = std::conditional_t, Int64, UInt64>; }; template <> struct NearestFieldTypeImpl { using Type = Int64; }; template <> struct NearestFieldTypeImpl { using Type = UInt64; }; +#if __cplusplus > 201703L template <> struct NearestFieldTypeImpl { using Type = UInt64; }; +#endif template <> struct NearestFieldTypeImpl { using Type = UInt64; }; template <> struct NearestFieldTypeImpl { using Type = UInt64; }; diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 5255c6f263e..20dd25b6b2d 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -22,10 +22,14 @@ #include #include #include -#include "config_core.h" + +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_SSL -#include -#include +# include +# include #endif /// Implementation of MySQL wire protocol. diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index b45e7d771a9..3a55a4328a7 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -137,8 +137,12 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const { - /// NOTE It's better to make a map in `IStorage` than to create it here every time again. - ::google::dense_hash_map types; + /// NOTE: It's better to make a map in `IStorage` than to create it here every time again. +#if !defined(ARCADIA_BUILD) + google::dense_hash_map types; +#else + google::sparsehash::dense_hash_map types; +#endif types.set_empty_key(StringRef()); for (const NameAndTypePair & column : *this) diff --git a/src/Core/ya.make b/src/Core/ya.make new file mode 100644 index 00000000000..4999fe334bc --- /dev/null +++ b/src/Core/ya.make @@ -0,0 +1,24 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/sparsehash + contrib/restricted/boost/libs +) + +SRCS( + BackgroundSchedulePool.cpp + Block.cpp + BlockInfo.cpp + ColumnWithTypeAndName.cpp + ExternalResultDescription.cpp + ExternalTable.cpp + Field.cpp + iostream_debug_helpers.cpp + MySQLProtocol.cpp + NamesAndTypes.cpp + Settings.cpp + SettingsCollection.cpp +) + +END() diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make new file mode 100644 index 00000000000..e6f8d8a94d7 --- /dev/null +++ b/src/DataStreams/ya.make @@ -0,0 +1,71 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +NO_COMPILER_WARNINGS() + +SRCS( + AddingDefaultBlockOutputStream.cpp + AddingDefaultsBlockInputStream.cpp + AggregatingBlockInputStream.cpp + AggregatingSortedBlockInputStream.cpp + AsynchronousBlockInputStream.cpp + BlockIO.cpp + BlockStreamProfileInfo.cpp + CheckConstraintsBlockOutputStream.cpp + CheckSortedBlockInputStream.cpp + CollapsingFinalBlockInputStream.cpp + CollapsingSortedBlockInputStream.cpp + ColumnGathererStream.cpp + ConvertingBlockInputStream.cpp + copyData.cpp + CountingBlockOutputStream.cpp + CreatingSetsBlockInputStream.cpp + CubeBlockInputStream.cpp + DistinctBlockInputStream.cpp + DistinctSortedBlockInputStream.cpp + ExecutionSpeedLimits.cpp + ExpressionBlockInputStream.cpp + FillingBlockInputStream.cpp + FilterBlockInputStream.cpp + FilterColumnsBlockInputStream.cpp + finalizeBlock.cpp + FinishSortingBlockInputStream.cpp + GraphiteRollupSortedBlockInputStream.cpp + IBlockInputStream.cpp + InputStreamFromASTInsertQuery.cpp + InternalTextLogsRowOutputStream.cpp + LimitBlockInputStream.cpp + LimitByBlockInputStream.cpp + materializeBlock.cpp + MaterializingBlockInputStream.cpp + MergeSortingBlockInputStream.cpp + MergingAggregatedBlockInputStream.cpp + MergingAggregatedMemoryEfficientBlockInputStream.cpp + MergingSortedBlockInputStream.cpp + narrowBlockInputStreams.cpp + NativeBlockInputStream.cpp + NativeBlockOutputStream.cpp + ParallelAggregatingBlockInputStream.cpp + ParallelParsingBlockInputStream.cpp + PartialSortingBlockInputStream.cpp + processConstants.cpp + PushingToViewsBlockOutputStream.cpp + RemoteBlockInputStream.cpp + RemoteBlockOutputStream.cpp + ReplacingSortedBlockInputStream.cpp + ReverseBlockInputStream.cpp + RollupBlockInputStream.cpp + SizeLimits.cpp + SquashingBlockInputStream.cpp + SquashingBlockOutputStream.cpp + SquashingTransform.cpp + SummingSortedBlockInputStream.cpp + TotalsHavingBlockInputStream.cpp + TTLBlockInputStream.cpp + VersionedCollapsingSortedBlockInputStream.cpp +) + +END() diff --git a/src/DataTypes/Native.h b/src/DataTypes/Native.h index 3652e85e2d3..754365dfc87 100644 --- a/src/DataTypes/Native.h +++ b/src/DataTypes/Native.h @@ -1,23 +1,25 @@ #pragma once -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_EMBEDDED_COMPILER +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" +# include -#include - -#pragma GCC diagnostic pop +# pragma GCC diagnostic pop namespace DB diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make new file mode 100644 index 00000000000..c3a1e452d0d --- /dev/null +++ b/src/DataTypes/ya.make @@ -0,0 +1,41 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + clickhouse/src/Formats +) + +SRCS( + convertMySQLDataType.cpp + DataTypeAggregateFunction.cpp + DataTypeArray.cpp + DataTypeCustomIPv4AndIPv6.cpp + DataTypeCustomSimpleAggregateFunction.cpp + DataTypeCustomSimpleTextSerialization.cpp + DataTypeDate.cpp + DataTypeDateTime.cpp + DataTypeDateTime64.cpp + DataTypeDecimalBase.cpp + DataTypeEnum.cpp + DataTypeFactory.cpp + DataTypeFixedString.cpp + DataTypeFunction.cpp + DataTypeInterval.cpp + DataTypeLowCardinality.cpp + DataTypeLowCardinalityHelpers.cpp + DataTypeNothing.cpp + DataTypeNullable.cpp + DataTypeNumberBase.cpp + DataTypesDecimal.cpp + DataTypesNumber.cpp + DataTypeString.cpp + DataTypeTuple.cpp + DataTypeUUID.cpp + FieldToDataType.cpp + getLeastSupertype.cpp + getMostSubtype.cpp + IDataType.cpp + NestedUtils.cpp +) + +END() diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index f1cea04dc29..b4a5537a192 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -9,15 +9,16 @@ #include #include #include -#include "config_core.h" #include "DatabaseFactory.h" #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_MYSQL - -#include -#include - +# include +# include #endif diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/DatabaseMySQL.cpp index 1cbbd4b06d9..7528e3fa719 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/DatabaseMySQL.cpp @@ -1,29 +1,30 @@ -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif #if USE_MYSQL +# 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 -#include -#include -#include - -#include -#include +# include +# include namespace DB diff --git a/src/Databases/ya.make b/src/Databases/ya.make new file mode 100644 index 00000000000..4aff1054023 --- /dev/null +++ b/src/Databases/ya.make @@ -0,0 +1,19 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + DatabaseDictionary.cpp + DatabaseFactory.cpp + DatabaseLazy.cpp + DatabaseMemory.cpp + DatabaseMySQL.cpp + DatabaseOnDisk.cpp + DatabaseOrdinary.cpp + DatabasesCommon.cpp + DatabaseWithDictionaries.cpp +) + +END() diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index c7766fbc628..0a7693edc60 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -1,14 +1,7 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) -include(${ClickHouse_SOURCE_DIR}/cmake/generate_code.cmake) add_headers_and_sources(clickhouse_dictionaries .) -generate_code(ComplexKeyCacheDictionary_generate1 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) -generate_code(ComplexKeyCacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) -generate_code(ComplexKeyCacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) -generate_code(CacheDictionary_generate1 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) -generate_code(CacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) -generate_code(CacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) add_headers_and_sources(clickhouse_dictionaries ${CMAKE_CURRENT_BINARY_DIR}/generated/) list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp getDictionaryConfigurationFromAST.cpp) diff --git a/src/Dictionaries/CacheDictionary_generate1.cpp b/src/Dictionaries/CacheDictionary_generate1.cpp new file mode 100644 index 00000000000..edb4b89d550 --- /dev/null +++ b/src/Dictionaries/CacheDictionary_generate1.cpp @@ -0,0 +1,32 @@ +#include +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void CacheDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) \ + const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + const auto null_value = std::get(attribute.null_values); \ + getItemsNumberImpl(attribute, ids, out, [&](const size_t) { return null_value; }); \ + } + +DEFINE(UInt8) +DEFINE(UInt16) +DEFINE(UInt32) +DEFINE(UInt64) +DEFINE(UInt128) +DEFINE(Int8) +DEFINE(Int16) +DEFINE(Int32) +DEFINE(Int64) +DEFINE(Float32) +DEFINE(Float64) +DEFINE(Decimal32) +DEFINE(Decimal64) +DEFINE(Decimal128) + +#undef DEFINE +} diff --git a/src/Dictionaries/CacheDictionary_generate1.cpp.in b/src/Dictionaries/CacheDictionary_generate1.cpp.in deleted file mode 100644 index b870a0ed69b..00000000000 --- a/src/Dictionaries/CacheDictionary_generate1.cpp.in +++ /dev/null @@ -1,22 +0,0 @@ -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; -void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const -{ - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - const auto null_value = std::get(attribute.null_values); - - getItemsNumberImpl(attribute, ids, out, [&](const size_t) { return null_value; }); -} - -} diff --git a/src/Dictionaries/CacheDictionary_generate2.cpp b/src/Dictionaries/CacheDictionary_generate2.cpp new file mode 100644 index 00000000000..97fa9e1a365 --- /dev/null +++ b/src/Dictionaries/CacheDictionary_generate2.cpp @@ -0,0 +1,34 @@ +#include +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void CacheDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const PaddedPODArray & ids, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + getItemsNumberImpl(attribute, ids, out, [&](const size_t row) { return def[row]; }); \ + } + +DEFINE(UInt8) +DEFINE(UInt16) +DEFINE(UInt32) +DEFINE(UInt64) +DEFINE(UInt128) +DEFINE(Int8) +DEFINE(Int16) +DEFINE(Int32) +DEFINE(Int64) +DEFINE(Float32) +DEFINE(Float64) +DEFINE(Decimal32) +DEFINE(Decimal64) +DEFINE(Decimal128) + +#undef DEFINE +} diff --git a/src/Dictionaries/CacheDictionary_generate2.cpp.in b/src/Dictionaries/CacheDictionary_generate2.cpp.in deleted file mode 100644 index 367e150c2cb..00000000000 --- a/src/Dictionaries/CacheDictionary_generate2.cpp.in +++ /dev/null @@ -1,23 +0,0 @@ -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; -void CacheDictionary::get@NAME@(const std::string & attribute_name, - const PaddedPODArray & ids, - const PaddedPODArray & def, - ResultArrayType & out) const -{ - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - getItemsNumberImpl(attribute, ids, out, [&](const size_t row) { return def[row]; }); -} - -} diff --git a/src/Dictionaries/CacheDictionary_generate3.cpp b/src/Dictionaries/CacheDictionary_generate3.cpp new file mode 100644 index 00000000000..8a94ca6bc20 --- /dev/null +++ b/src/Dictionaries/CacheDictionary_generate3.cpp @@ -0,0 +1,31 @@ +#include +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void CacheDictionary::get##TYPE( \ + const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + getItemsNumberImpl(attribute, ids, out, [&](const size_t) { return def; }); \ + } + +DEFINE(UInt8) +DEFINE(UInt16) +DEFINE(UInt32) +DEFINE(UInt64) +DEFINE(UInt128) +DEFINE(Int8) +DEFINE(Int16) +DEFINE(Int32) +DEFINE(Int64) +DEFINE(Float32) +DEFINE(Float64) +DEFINE(Decimal32) +DEFINE(Decimal64) +DEFINE(Decimal128) + +#undef DEFINE +} diff --git a/src/Dictionaries/CacheDictionary_generate3.cpp.in b/src/Dictionaries/CacheDictionary_generate3.cpp.in deleted file mode 100644 index 8e2c26302e8..00000000000 --- a/src/Dictionaries/CacheDictionary_generate3.cpp.in +++ /dev/null @@ -1,20 +0,0 @@ -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; -void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const -{ - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - getItemsNumberImpl(attribute, ids, out, [&](const size_t) { return def; }); -} - -} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp new file mode 100644 index 00000000000..6f9761cd064 --- /dev/null +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp @@ -0,0 +1,32 @@ +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void ComplexKeyCacheDictionary::get##TYPE( \ + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + const auto null_value = std::get(attribute.null_values); \ + getItemsNumberImpl(attribute, key_columns, out, [&](const size_t) { return null_value; }); \ + } + +DEFINE(UInt8) +DEFINE(UInt16) +DEFINE(UInt32) +DEFINE(UInt64) +DEFINE(UInt128) +DEFINE(Int8) +DEFINE(Int16) +DEFINE(Int32) +DEFINE(Int64) +DEFINE(Float32) +DEFINE(Float64) +DEFINE(Decimal32) +DEFINE(Decimal64) +DEFINE(Decimal128) + +#undef DEFINE +} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in b/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in deleted file mode 100644 index 5c0ed408a55..00000000000 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in +++ /dev/null @@ -1,22 +0,0 @@ -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; -void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const -{ - dict_struct.validateKeyTypes(key_types); - - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - const auto null_value = std::get(attribute.null_values); - - getItemsNumberImpl(attribute, key_columns, out, [&](const size_t) { return null_value; }); -} -} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp new file mode 100644 index 00000000000..297da2c91a0 --- /dev/null +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp @@ -0,0 +1,35 @@ +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void ComplexKeyCacheDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + getItemsNumberImpl(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); \ + } + +DEFINE(UInt8) +DEFINE(UInt16) +DEFINE(UInt32) +DEFINE(UInt64) +DEFINE(UInt128) +DEFINE(Int8) +DEFINE(Int16) +DEFINE(Int32) +DEFINE(Int64) +DEFINE(Float32) +DEFINE(Float64) +DEFINE(Decimal32) +DEFINE(Decimal64) +DEFINE(Decimal128) + +#undef DEFINE +} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in b/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in deleted file mode 100644 index b3233cd05e1..00000000000 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in +++ /dev/null @@ -1,25 +0,0 @@ -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; - -void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name, - const Columns & key_columns, - const DataTypes & key_types, - const PaddedPODArray & def, - ResultArrayType & out) const -{ - dict_struct.validateKeyTypes(key_types); - - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - getItemsNumberImpl(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); -} -} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp new file mode 100644 index 00000000000..222e9c8ee77 --- /dev/null +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp @@ -0,0 +1,35 @@ +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void ComplexKeyCacheDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const TYPE def, \ + ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + getItemsNumberImpl(attribute, key_columns, out, [&](const size_t) { return def; }); \ + } + +DEFINE(UInt8) +DEFINE(UInt16) +DEFINE(UInt32) +DEFINE(UInt64) +DEFINE(UInt128) +DEFINE(Int8) +DEFINE(Int16) +DEFINE(Int32) +DEFINE(Int64) +DEFINE(Float32) +DEFINE(Float64) +DEFINE(Decimal32) +DEFINE(Decimal64) +DEFINE(Decimal128) + +#undef DEFINE +} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in b/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in deleted file mode 100644 index 02e77c01a4a..00000000000 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in +++ /dev/null @@ -1,25 +0,0 @@ -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; - -void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name, - const Columns & key_columns, - const DataTypes & key_types, - const TYPE def, - ResultArrayType & out) const -{ - dict_struct.validateKeyTypes(key_types); - - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - getItemsNumberImpl(attribute, key_columns, out, [&](const size_t) { return def; }); -} -} diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 3f8eec979bb..a361352a8bd 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -156,8 +156,14 @@ private: template using CollectionPtrType = std::unique_ptr>; +#if !defined(ARCADIA_BUILD) template using SparseCollectionType = google::sparse_hash_map>; +#else + template + using SparseCollectionType = google::sparsehash::sparse_hash_map>; +#endif + template using SparseCollectionPtrType = std::unique_ptr>; diff --git a/src/Dictionaries/MongoDBBlockInputStream.cpp b/src/Dictionaries/MongoDBBlockInputStream.cpp index b1c2f67cc86..e0ff7cb6529 100644 --- a/src/Dictionaries/MongoDBBlockInputStream.cpp +++ b/src/Dictionaries/MongoDBBlockInputStream.cpp @@ -1,25 +1,27 @@ -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_POCO_MONGODB +# include +# include +# include -#include -#include -#include +# include +# include +# include +# include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include "DictionaryStructure.h" -#include "MongoDBBlockInputStream.h" +# include +# include +# include +# include +# include +# include +# include +# include +# include "DictionaryStructure.h" +# include "MongoDBBlockInputStream.h" namespace DB diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index bf4669248dc..622d61c5b09 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -1,9 +1,12 @@ #pragma once #include -#include "config_core.h" -#if USE_POCO_MONGODB +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_POCO_MONGODB # include "DictionaryStructure.h" # include "IDictionarySource.h" @@ -91,4 +94,5 @@ private: }; } + #endif diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 00527064d79..4ba16ca19de 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -1,8 +1,8 @@ #include "MySQLDictionarySource.h" + #include #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" -#include "config_core.h" #include "registerDictionaries.h" namespace DB diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 95e660d220f..34f784cdfeb 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -2,9 +2,11 @@ #include -#include "config_core.h" -#if USE_MYSQL +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif +#if USE_MYSQL # include # include # include "DictionaryStructure.h" diff --git a/src/Dictionaries/RedisBlockInputStream.h b/src/Dictionaries/RedisBlockInputStream.h index 86448095787..6e350e7c4f1 100644 --- a/src/Dictionaries/RedisBlockInputStream.h +++ b/src/Dictionaries/RedisBlockInputStream.h @@ -1,8 +1,11 @@ #pragma once -#include "config_core.h" #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_POCO_REDIS # include # include diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index dc62d7dc625..e42bd2678cb 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -1,10 +1,12 @@ #pragma once -#include "config_core.h" #include -#if USE_POCO_REDIS +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif +#if USE_POCO_REDIS # include "DictionaryStructure.h" # include "IDictionarySource.h" @@ -101,4 +103,5 @@ namespace ErrorCodes }; } + #endif diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 8647c8ccac5..3fea0a024ef 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -16,9 +16,12 @@ #include "DictionaryStructure.h" #include "readInvalidateQuery.h" -#include #include "registerDictionaries.h" +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_SQLODBC || USE_POCO_DATAODBC # include #endif diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 60218d457b4..cbd2d6010cb 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -25,7 +25,9 @@ void registerDictionaries() registerDictionaryRangeHashed(factory); registerDictionaryComplexKeyHashed(factory); registerDictionaryComplexKeyCache(factory); +#if !defined(ARCADIA_BUILD) registerDictionaryTrie(factory); +#endif registerDictionaryFlat(factory); registerDictionaryHashed(factory); registerDictionaryCache(factory); diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make new file mode 100644 index 00000000000..3f831c3c9fe --- /dev/null +++ b/src/Dictionaries/ya.make @@ -0,0 +1,63 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/poco/Data + contrib/libs/poco/Data/ODBC + contrib/libs/poco/MongoDB + contrib/libs/poco/Redis + contrib/libs/sparsehash +) + +NO_COMPILER_WARNINGS() + +SRCS( + CacheDictionary_generate1.cpp + CacheDictionary_generate2.cpp + CacheDictionary_generate3.cpp + CacheDictionary.cpp + ClickHouseDictionarySource.cpp + ComplexKeyCacheDictionary_createAttributeWithType.cpp + ComplexKeyCacheDictionary_generate1.cpp + ComplexKeyCacheDictionary_generate2.cpp + ComplexKeyCacheDictionary_generate3.cpp + ComplexKeyCacheDictionary_setAttributeValue.cpp + ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp + ComplexKeyCacheDictionary.cpp + ComplexKeyHashedDictionary.cpp + DictionaryBlockInputStreamBase.cpp + DictionaryFactory.cpp + DictionarySourceFactory.cpp + DictionarySourceHelpers.cpp + DictionaryStructure.cpp + Embedded/GeodataProviders/HierarchiesProvider.cpp + Embedded/GeodataProviders/HierarchyFormatReader.cpp + Embedded/GeodataProviders/NamesFormatReader.cpp + Embedded/GeodataProviders/NamesProvider.cpp + Embedded/GeoDictionariesLoader.cpp + Embedded/RegionsHierarchies.cpp + Embedded/RegionsHierarchy.cpp + Embedded/RegionsNames.cpp + ExecutableDictionarySource.cpp + ExternalQueryBuilder.cpp + FileDictionarySource.cpp + FlatDictionary.cpp + getDictionaryConfigurationFromAST.cpp + HashedDictionary.cpp + HTTPDictionarySource.cpp + LibraryDictionarySource.cpp + LibraryDictionarySourceExternal.cpp + MongoDBBlockInputStream.cpp + MongoDBDictionarySource.cpp + MySQLDictionarySource.cpp + PolygonDictionary.cpp + RangeHashedDictionary.cpp + readInvalidateQuery.cpp + RedisBlockInputStream.cpp + RedisDictionarySource.cpp + registerDictionaries.cpp + writeParenthesisedString.cpp + XDBCDictionarySource.cpp +) + +END() diff --git a/src/Disks/DiskS3.h b/src/Disks/DiskS3.h index 1b61ed1cde3..c89fde4b103 100644 --- a/src/Disks/DiskS3.h +++ b/src/Disks/DiskS3.h @@ -1,6 +1,8 @@ #pragma once -#include +#if !defined(ARCADIA_BUILD) +# include +#endif #if USE_AWS_S3 # include "DiskFactory.h" diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index f28918cdbd0..2da39e62b19 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -2,7 +2,9 @@ #include "DiskFactory.h" -#include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Disks/ya.make b/src/Disks/ya.make new file mode 100644 index 00000000000..71eb47c489c --- /dev/null +++ b/src/Disks/ya.make @@ -0,0 +1,17 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + DiskFactory.cpp + DiskLocal.cpp + DiskMemory.cpp + DiskS3.cpp + DiskSpaceMonitor.cpp + IDisk.cpp + registerDisks.cpp +) + +END() diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 7d741004766..d48b2dde206 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -1,12 +1,12 @@ +#include + #include -#include #include #include #include #include #include #include -#include #include #include #include @@ -16,6 +16,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { @@ -347,12 +351,14 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONCompactEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); +#if !defined(ARCADIA_BUILD) registerInputFormatProcessorCapnProto(*this); registerInputFormatProcessorORC(*this); registerInputFormatProcessorParquet(*this); registerOutputFormatProcessorParquet(*this); registerInputFormatProcessorAvro(*this); registerOutputFormatProcessorAvro(*this); +#endif registerInputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this); registerInputFormatProcessorRegexp(*this); diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 4f8291ffebe..17c09cdc14d 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -1,15 +1,17 @@ -#include "config_core.h" -#if USE_MYSQL +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif -#include -#include -#include -#include -#include -#include -#include -#include -#include "MySQLBlockInputStream.h" +#if USE_MYSQL +# include +# include +# include +# include +# include +# include +# include +# include +# include "MySQLBlockInputStream.h" namespace DB diff --git a/src/Formats/ProtobufColumnMatcher.h b/src/Formats/ProtobufColumnMatcher.h index ed6c632f8e8..03c5ec40fc6 100644 --- a/src/Formats/ProtobufColumnMatcher.h +++ b/src/Formats/ProtobufColumnMatcher.h @@ -1,15 +1,17 @@ #pragma once -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include -#include -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include +# include +# include +# include namespace google { @@ -190,4 +192,5 @@ namespace ProtobufColumnMatcher } } + #endif diff --git a/src/Formats/ProtobufReader.cpp b/src/Formats/ProtobufReader.cpp index 5426e8fac62..249bcf12711 100644 --- a/src/Formats/ProtobufReader.cpp +++ b/src/Formats/ProtobufReader.cpp @@ -1,16 +1,14 @@ -#include "config_formats.h" -#if USE_PROTOBUF - #include "ProtobufReader.h" -#include -#include -#include -#include -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include +# include +# include +# include +# include namespace DB @@ -272,25 +270,25 @@ UInt64 ProtobufReader::SimpleReader::continueReadingVarint(UInt64 first_byte) UInt64 result = (first_byte & ~static_cast(0x80)); char c; -#define PROTOBUF_READER_READ_VARINT_BYTE(byteNo) \ - do \ - { \ - in.readStrict(c); \ - ++cursor; \ - if constexpr ((byteNo) < 10) \ +# define PROTOBUF_READER_READ_VARINT_BYTE(byteNo) \ + do \ { \ - result |= static_cast(static_cast(c)) << (7 * ((byteNo) - 1)); \ - if (likely(!(c & 0x80))) \ - return result; \ - } \ - else \ - { \ - if (likely(c == 1)) \ - return result; \ - } \ - if constexpr ((byteNo) < 9) \ - result &= ~(static_cast(0x80) << (7 * ((byteNo) - 1))); \ - } while (false) + in.readStrict(c); \ + ++cursor; \ + if constexpr ((byteNo) < 10) \ + { \ + result |= static_cast(static_cast(c)) << (7 * ((byteNo)-1)); \ + if (likely(!(c & 0x80))) \ + return result; \ + } \ + else \ + { \ + if (likely(c == 1)) \ + return result; \ + } \ + if constexpr ((byteNo) < 9) \ + result &= ~(static_cast(0x80) << (7 * ((byteNo)-1))); \ + } while (false) PROTOBUF_READER_READ_VARINT_BYTE(2); PROTOBUF_READER_READ_VARINT_BYTE(3); @@ -302,7 +300,7 @@ UInt64 ProtobufReader::SimpleReader::continueReadingVarint(UInt64 first_byte) PROTOBUF_READER_READ_VARINT_BYTE(9); PROTOBUF_READER_READ_VARINT_BYTE(10); -#undef PROTOBUF_READER_READ_VARINT_BYTE +# undef PROTOBUF_READER_READ_VARINT_BYTE throwUnknownFormat(); } @@ -311,22 +309,22 @@ void ProtobufReader::SimpleReader::ignoreVarint() { char c; -#define PROTOBUF_READER_IGNORE_VARINT_BYTE(byteNo) \ - do \ - { \ - in.readStrict(c); \ - ++cursor; \ - if constexpr ((byteNo) < 10) \ +# define PROTOBUF_READER_IGNORE_VARINT_BYTE(byteNo) \ + do \ { \ - if (likely(!(c & 0x80))) \ - return; \ - } \ - else \ - { \ - if (likely(c == 1)) \ - return; \ - } \ - } while (false) + in.readStrict(c); \ + ++cursor; \ + if constexpr ((byteNo) < 10) \ + { \ + if (likely(!(c & 0x80))) \ + return; \ + } \ + else \ + { \ + if (likely(c == 1)) \ + return; \ + } \ + } while (false) PROTOBUF_READER_IGNORE_VARINT_BYTE(1); PROTOBUF_READER_IGNORE_VARINT_BYTE(2); @@ -338,7 +336,8 @@ void ProtobufReader::SimpleReader::ignoreVarint() PROTOBUF_READER_IGNORE_VARINT_BYTE(8); PROTOBUF_READER_IGNORE_VARINT_BYTE(9); PROTOBUF_READER_IGNORE_VARINT_BYTE(10); -#undef PROTOBUF_READER_IGNORE_VARINT_BYTE + +# undef PROTOBUF_READER_IGNORE_VARINT_BYTE throwUnknownFormat(); } @@ -694,16 +693,17 @@ private: std::optional> enum_name_to_value_map; }; -#define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \ - template <> \ - std::unique_ptr ProtobufReader::createConverter( \ - const google::protobuf::FieldDescriptor * field) \ - { \ - return std::make_unique(simple_reader, field); \ - } +# define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \ + template <> \ + std::unique_ptr ProtobufReader::createConverter( \ + const google::protobuf::FieldDescriptor * field) \ + { \ + return std::make_unique(simple_reader, field); \ + } PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_STRING) PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_BYTES) -#undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS + +# undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS template @@ -850,13 +850,14 @@ private: std::optional> set_of_enum_values; }; -#define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \ - template <> \ - std::unique_ptr ProtobufReader::createConverter( \ - const google::protobuf::FieldDescriptor * field) \ - { \ - return std::make_unique>(simple_reader, field); /* NOLINT */ \ - } +# define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \ + template <> \ + std::unique_ptr ProtobufReader::createConverter( \ + const google::protobuf::FieldDescriptor * field) \ + { \ + return std::make_unique>(simple_reader, field); /* NOLINT */ \ + } + PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT32, Int64); PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT32, Int64); PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT32, UInt64); @@ -869,7 +870,8 @@ PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::Fi PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED64, Int64); PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FLOAT, float); PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_DOUBLE, double); -#undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS + +# undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS class ProtobufReader::ConverterFromBool : public ConverterBaseImpl @@ -1073,10 +1075,10 @@ void ProtobufReader::setTraitsDataAfterMatchingColumns(Message * message) } switch (field.field_descriptor->type()) { -#define PROTOBUF_READER_CONVERTER_CREATING_CASE(field_type_id) \ - case field_type_id: \ - field.data.converter = createConverter(field.field_descriptor); \ - break +# define PROTOBUF_READER_CONVERTER_CREATING_CASE(field_type_id) \ + case field_type_id: \ + field.data.converter = createConverter(field.field_descriptor); \ + break PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_STRING); PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BYTES); PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_INT32); @@ -1093,8 +1095,9 @@ void ProtobufReader::setTraitsDataAfterMatchingColumns(Message * message) PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_DOUBLE); PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BOOL); PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_ENUM); -#undef PROTOBUF_READER_CONVERTER_CREATING_CASE - default: __builtin_unreachable(); +# undef PROTOBUF_READER_CONVERTER_CREATING_CASE + default: + __builtin_unreachable(); } message->data.field_number_to_field_map.emplace(field.field_number, &field); } @@ -1171,4 +1174,5 @@ bool ProtobufReader::readColumnIndex(size_t & column_index) } } + #endif diff --git a/src/Formats/ProtobufReader.h b/src/Formats/ProtobufReader.h index 393b15eb343..a50c2f202f0 100644 --- a/src/Formats/ProtobufReader.h +++ b/src/Formats/ProtobufReader.h @@ -5,13 +5,15 @@ #include #include -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include "ProtobufColumnMatcher.h" -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include "ProtobufColumnMatcher.h" namespace google { diff --git a/src/Formats/ProtobufSchemas.cpp b/src/Formats/ProtobufSchemas.cpp index f4973263bc8..9c6ed76ef27 100644 --- a/src/Formats/ProtobufSchemas.cpp +++ b/src/Formats/ProtobufSchemas.cpp @@ -1,10 +1,12 @@ -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include namespace DB diff --git a/src/Formats/ProtobufWriter.cpp b/src/Formats/ProtobufWriter.cpp index 8ee309d41c9..d68999c7ce4 100644 --- a/src/Formats/ProtobufWriter.cpp +++ b/src/Formats/ProtobufWriter.cpp @@ -1,18 +1,16 @@ -#include "config_formats.h" -#if USE_PROTOBUF - #include "ProtobufWriter.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB @@ -496,19 +494,19 @@ private: std::optional> enum_value_to_name_map; }; -#define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \ - template <> \ - std::unique_ptr ProtobufWriter::createConverter( \ - const google::protobuf::FieldDescriptor * field) \ - { \ - if (shouldSkipNullValue(field)) \ - return std::make_unique>(simple_writer, field); \ - else \ - return std::make_unique>(simple_writer, field); \ - } +# define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \ + template <> \ + std::unique_ptr ProtobufWriter::createConverter( \ + const google::protobuf::FieldDescriptor * field) \ + { \ + if (shouldSkipNullValue(field)) \ + return std::make_unique>(simple_writer, field); \ + else \ + return std::make_unique>(simple_writer, field); \ + } PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_STRING) PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_BYTES) -#undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS +# undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS template @@ -606,18 +604,19 @@ private: } }; -#define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \ - template <> \ - std::unique_ptr ProtobufWriter::createConverter( \ - const google::protobuf::FieldDescriptor * field) \ - { \ - if (shouldSkipNullValue(field)) \ - return std::make_unique>(simple_writer, field); \ - else if (shouldPackRepeated(field)) \ - return std::make_unique>(simple_writer, field); \ - else \ - return std::make_unique>(simple_writer, field); \ - } +# define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \ + template <> \ + std::unique_ptr ProtobufWriter::createConverter( \ + const google::protobuf::FieldDescriptor * field) \ + { \ + if (shouldSkipNullValue(field)) \ + return std::make_unique>(simple_writer, field); \ + else if (shouldPackRepeated(field)) \ + return std::make_unique>(simple_writer, field); \ + else \ + return std::make_unique>(simple_writer, field); \ + } + PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT32, Int32); PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT32, Int32); PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT32, UInt32); @@ -630,7 +629,7 @@ PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::Fi PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED64, Int64); PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FLOAT, float); PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_DOUBLE, double); -#undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS +# undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS template @@ -871,10 +870,10 @@ void ProtobufWriter::setTraitsDataAfterMatchingColumns(Message * message) } switch (field.field_descriptor->type()) { -#define PROTOBUF_WRITER_CONVERTER_CREATING_CASE(field_type_id) \ - case field_type_id: \ - field.data.converter = createConverter(field.field_descriptor); \ - break +# define PROTOBUF_WRITER_CONVERTER_CREATING_CASE(field_type_id) \ + case field_type_id: \ + field.data.converter = createConverter(field.field_descriptor); \ + break PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_STRING); PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BYTES); PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_INT32); @@ -891,7 +890,7 @@ void ProtobufWriter::setTraitsDataAfterMatchingColumns(Message * message) PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_DOUBLE); PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BOOL); PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_ENUM); -#undef PROTOBUF_WRITER_CONVERTER_CREATING_CASE +# undef PROTOBUF_WRITER_CONVERTER_CREATING_CASE default: throw Exception( String("Protobuf type '") + field.field_descriptor->type_name() + "' isn't supported", ErrorCodes::NOT_IMPLEMENTED); @@ -991,4 +990,5 @@ void ProtobufWriter::setNestedMessageNeedsRepeat() } } + #endif diff --git a/src/Formats/ProtobufWriter.h b/src/Formats/ProtobufWriter.h index 3862ed88cdb..1778f2adb21 100644 --- a/src/Formats/ProtobufWriter.h +++ b/src/Formats/ProtobufWriter.h @@ -3,17 +3,17 @@ #include #include #include - -#include "config_formats.h" - #include -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include "ProtobufColumnMatcher.h" -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include "ProtobufColumnMatcher.h" namespace google @@ -233,6 +233,8 @@ private: } #else +# include + namespace DB { diff --git a/src/Formats/ya.make b/src/Formats/ya.make new file mode 100644 index 00000000000..de61820e58d --- /dev/null +++ b/src/Formats/ya.make @@ -0,0 +1,24 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/protobuf_std +) + +SRCS( + FormatFactory.cpp + FormatSchemaInfo.cpp + IRowInputStream.cpp + IRowOutputStream.cpp + MySQLBlockInputStream.cpp + NativeFormat.cpp + NullFormat.cpp + ParsedTemplateFormatString.cpp + ProtobufColumnMatcher.cpp + ProtobufReader.cpp + ProtobufSchemas.cpp + ProtobufWriter.cpp + verbosePrintString.cpp +) + +END() diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index df3b86f721d..5a0c1b8232a 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -2,9 +2,11 @@ #include #include -#include #include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 27a873f69fa..5b100bf68ce 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -1,13 +1,16 @@ -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + #if USE_BASE64 -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 2ad52d69f09..30b6da8b696 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -27,13 +27,16 @@ #include "FunctionFactory.h" #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif #if USE_EMBEDDED_COMPILER -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" +# include +# pragma GCC diagnostic pop #endif diff --git a/src/Functions/FunctionIfBase.h b/src/Functions/FunctionIfBase.h index 48aa1c0b6b2..bdb883ee342 100644 --- a/src/Functions/FunctionIfBase.h +++ b/src/Functions/FunctionIfBase.h @@ -1,9 +1,11 @@ #pragma once -#include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Functions/FunctionMathBinaryFloat64.h b/src/Functions/FunctionMathBinaryFloat64.h index 2927d4ef228..f696e4728fb 100644 --- a/src/Functions/FunctionMathBinaryFloat64.h +++ b/src/Functions/FunctionMathBinaryFloat64.h @@ -8,7 +8,10 @@ #include #include #include -#include "config_functions.h" + +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif /** More efficient implementations of mathematical functions are possible when using a separate library. * Disabled due to license compatibility limitations. diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index 716b52b40c5..b048b994951 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -7,7 +7,10 @@ #include #include #include -#include "config_functions.h" + +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif /** More efficient implementations of mathematical functions are possible when using a separate library. * Disabled due to license compatibility limitations. @@ -20,7 +23,7 @@ * Enabled by default. */ #if USE_FASTOPS -#include +# include #endif diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 89687b5b23e..4e3ccdc8952 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -10,13 +10,16 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif #if USE_EMBEDDED_COMPILER -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" +# include +# pragma GCC diagnostic pop #endif diff --git a/src/Functions/FunctionsEmbeddedDictionaries.h b/src/Functions/FunctionsEmbeddedDictionaries.h index 12b478a26b6..44fdca9736d 100644 --- a/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/src/Functions/FunctionsEmbeddedDictionaries.h @@ -15,9 +15,12 @@ #include #include #include -#include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index fc3c2c583a9..61ba555be92 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -135,7 +135,9 @@ private: !executeDispatchSimple(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && +#if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif !executeDispatchComplex(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; } @@ -306,7 +308,9 @@ private: !executeDispatch(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && +#if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchRange(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; @@ -488,8 +492,10 @@ private: !executeDispatch(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && - !executeDispatchComplex(block, arguments, result, dict_ptr) && - !executeDispatchComplex(block, arguments, result, dict_ptr)) +#if !defined(ARCADIA_BUILD) + !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif + !executeDispatchComplex(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; } @@ -826,7 +832,9 @@ private: !executeDispatch(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && +#if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchRange(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; @@ -1086,8 +1094,10 @@ private: !executeDispatch(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && - !executeDispatchComplex(block, arguments, result, dict_ptr) && - !executeDispatchComplex(block, arguments, result, dict_ptr)) +#if !defined(ARCADIA_BUILD) + !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif + !executeDispatchComplex(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; } diff --git a/src/Functions/FunctionsHashing.cpp b/src/Functions/FunctionsHashing.cpp index 8705e6bfaa3..900455a1015 100644 --- a/src/Functions/FunctionsHashing.cpp +++ b/src/Functions/FunctionsHashing.cpp @@ -26,12 +26,14 @@ void registerFunctionsHashing(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); +#if !defined(ARCADIA_BUILD) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); +#endif #if USE_XXHASH factory.registerFunction(); diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index a589b220351..f647390e1c8 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -3,22 +3,24 @@ #include #include #include -#include -#include +#if !defined(ARCADIA_BUILD) +# include +# include +# include "config_functions.h" +# include "config_core.h" +#endif #include #include #include -#include "config_functions.h" #if USE_XXHASH -# include +# include #endif -#include "config_core.h" #if USE_SSL -# include -# include +# include +# include #endif #include @@ -219,7 +221,7 @@ struct SipHash128Impl } }; - +#if !defined(ARCADIA_BUILD) /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. * Usually there is no reason to use MurmurHash. @@ -331,6 +333,18 @@ struct MurmurHash3Impl64 static constexpr bool use_int_hash_for_pods = false; }; +struct MurmurHash3Impl128 +{ + static constexpr auto name = "murmurHash3_128"; + enum { length = 16 }; + + static void apply(const char * begin, const size_t size, unsigned char * out_char_data) + { + MurmurHash3_x64_128(begin, size, 0, out_char_data); + } +}; +#endif + /// http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452 /// Care should be taken to do all calculation in unsigned integers (to avoid undefined behaviour on overflow) /// but obtain the same result as it is done in singed integers with two's complement arithmetic. @@ -411,17 +425,6 @@ struct HiveHashImpl static constexpr bool use_int_hash_for_pods = false; }; -struct MurmurHash3Impl128 -{ - static constexpr auto name = "murmurHash3_128"; - enum { length = 16 }; - - static void apply(const char * begin, const size_t size, unsigned char * out_char_data) - { - MurmurHash3_x64_128(begin, size, 0, out_char_data); - } -}; - struct ImplCityHash64 { static constexpr auto name = "cityHash64"; @@ -1145,12 +1148,16 @@ using FunctionSipHash128 = FunctionStringHashFixedString; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; using FunctionMetroHash64 = FunctionAnyHash; + +#if !defined(ARCADIA_BUILD) using FunctionMurmurHash2_32 = FunctionAnyHash; using FunctionMurmurHash2_64 = FunctionAnyHash; using FunctionGccMurmurHash = FunctionAnyHash; using FunctionMurmurHash3_32 = FunctionAnyHash; using FunctionMurmurHash3_64 = FunctionAnyHash; using FunctionMurmurHash3_128 = FunctionStringHashFixedString; +#endif + using FunctionJavaHash = FunctionAnyHash; using FunctionJavaHashUTF16LE = FunctionAnyHash; using FunctionHiveHash = FunctionAnyHash; diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 468b499b78c..acd7c42732c 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -5,7 +5,6 @@ #include #include #include -#include "config_functions.h" #include #include #include @@ -27,6 +26,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + namespace DB { diff --git a/src/Functions/FunctionsStringRegex.cpp b/src/Functions/FunctionsStringRegex.cpp index 856852ae197..2f4c2432d54 100644 --- a/src/Functions/FunctionsStringRegex.cpp +++ b/src/Functions/FunctionsStringRegex.cpp @@ -17,16 +17,15 @@ #include #include -#include "config_functions.h" -#if USE_HYPERSCAN -# if __has_include() -# include -# else -# include -# endif +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +# include +#endif + +#if USE_HYPERSCAN +# include #endif -#include #if USE_RE2_ST # include #else diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 7355b5df9ad..911810a272c 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -24,11 +23,15 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_EMBEDDED_COMPILER -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" +# include +# pragma GCC diagnostic pop #endif diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 3887f3f1669..5ef6e5aaf22 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -2,12 +2,15 @@ #include -#include "config_core.h" #include #include #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + /// This file contains user interface for functions. /// For developer interface (in case you need to implement a new function) see IFunctionImpl.h diff --git a/src/Functions/RapidJSONParser.h b/src/Functions/RapidJSONParser.h index ff4ecd506fd..07b0aea3543 100644 --- a/src/Functions/RapidJSONParser.h +++ b/src/Functions/RapidJSONParser.h @@ -1,13 +1,15 @@ #pragma once -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + #if USE_RAPIDJSON +# include +# include +# include -#include -#include -#include - -#include +# include namespace DB diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index 5d93e823419..433e416a18f 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -14,14 +14,12 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif -#include "config_functions.h" #if USE_HYPERSCAN -# if __has_include() -# include -# else -# include -# endif +# include #endif namespace ProfileEvents diff --git a/src/Functions/SimdJSONParser.h b/src/Functions/SimdJSONParser.h index a12119c5c6d..d5630cf1cf1 100644 --- a/src/Functions/SimdJSONParser.h +++ b/src/Functions/SimdJSONParser.h @@ -1,13 +1,15 @@ #pragma once -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + #if USE_SIMDJSON +# include +# include +# include -#include -#include -#include - -#include +# include namespace DB @@ -137,4 +139,5 @@ private: }; } + #endif diff --git a/src/Functions/base64Encode.cpp b/src/Functions/base64Encode.cpp index f76fedda71e..8cf54eca768 100644 --- a/src/Functions/base64Encode.cpp +++ b/src/Functions/base64Encode.cpp @@ -1,9 +1,12 @@ #include #include -#include "config_functions.h" + +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif #if USE_BASE64 -#include +# include namespace DB { diff --git a/src/Functions/convertCharset.cpp b/src/Functions/convertCharset.cpp index 372bf08a0d2..9bd1f1b1cf6 100644 --- a/src/Functions/convertCharset.cpp +++ b/src/Functions/convertCharset.cpp @@ -1,20 +1,22 @@ -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_ICU +# 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 diff --git a/src/Functions/intDiv.cpp b/src/Functions/intDiv.cpp index 062a374c00f..d21e779045f 100644 --- a/src/Functions/intDiv.cpp +++ b/src/Functions/intDiv.cpp @@ -3,7 +3,6 @@ #if defined(__SSE2__) # define LIBDIVIDE_SSE2 1 -# define LIBDIVIDE_VECTOR_TYPE #endif #include diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index 233018c7f16..02013e33d16 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -46,7 +46,9 @@ void registerFunctions() registerFunctionsArithmetic(factory); registerFunctionsArray(factory); registerFunctionsTuple(factory); +#if !defined(ARCADIA_BUILD) registerFunctionsBitmap(factory); +#endif registerFunctionsCoding(factory); registerFunctionsComparison(factory); registerFunctionsConditional(factory); diff --git a/src/Functions/registerFunctionsConsistentHashing.cpp b/src/Functions/registerFunctionsConsistentHashing.cpp index 95a856b6d3c..ceec6dca5e6 100644 --- a/src/Functions/registerFunctionsConsistentHashing.cpp +++ b/src/Functions/registerFunctionsConsistentHashing.cpp @@ -5,14 +5,18 @@ class FunctionFactory; void registerFunctionYandexConsistentHash(FunctionFactory & factory); void registerFunctionJumpConsistentHash(FunctionFactory & factory); +#if !defined(ARCADIA_BUILD) void registerFunctionSumburConsistentHash(FunctionFactory & factory); +#endif void registerFunctionsConsistentHashing(FunctionFactory & factory) { registerFunctionYandexConsistentHash(factory); registerFunctionJumpConsistentHash(factory); +#if !defined(ARCADIA_BUILD) registerFunctionSumburConsistentHash(factory); +#endif } } diff --git a/src/Functions/registerFunctionsGeo.cpp b/src/Functions/registerFunctionsGeo.cpp index cb3c268a19a..2ed02bbd73b 100644 --- a/src/Functions/registerFunctionsGeo.cpp +++ b/src/Functions/registerFunctionsGeo.cpp @@ -1,4 +1,6 @@ -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif namespace DB { diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 30cab4cc53a..221e14fcce1 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -1,4 +1,6 @@ -#include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index cc94e877bbf..5493b98d3fc 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -1,4 +1,6 @@ -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif namespace DB { diff --git a/src/Functions/version.cpp b/src/Functions/version.cpp index 34eabd30336..e11f45f160a 100644 --- a/src/Functions/version.cpp +++ b/src/Functions/version.cpp @@ -1,9 +1,11 @@ #include #include #include -#include #include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Functions/ya.make b/src/Functions/ya.make new file mode 100644 index 00000000000..efcae6bf73b --- /dev/null +++ b/src/Functions/ya.make @@ -0,0 +1,418 @@ +LIBRARY() + +ADDINCL( + library/consistent_hashing + contrib/libs/farmhash + contrib/libs/hyperscan/src + contrib/libs/icu/common + contrib/libs/libdivide + contrib/libs/rapidjson/include + contrib/libs/xxhash +) + +PEERDIR( + clickhouse/src/Common + clickhouse/src/Dictionaries + contrib/libs/farmhash + contrib/libs/fastops/fastops + contrib/libs/hyperscan + contrib/libs/icu + contrib/libs/libdivide + contrib/libs/metrohash + contrib/libs/rapidjson + contrib/libs/xxhash + library/consistent_hashing +) + +SRCS( + array/array.cpp + array/arrayAll.cpp + array/arrayAUC.cpp + array/arrayCompact.cpp + array/arrayConcat.cpp + array/arrayCount.cpp + array/arrayCumSum.cpp + array/arrayCumSumNonNegative.cpp + array/arrayDifference.cpp + array/arrayDistinct.cpp + array/arrayElement.cpp + array/arrayEnumerate.cpp + array/arrayEnumerateDense.cpp + array/arrayEnumerateDenseRanked.cpp + array/arrayEnumerateRanked.cpp + array/arrayEnumerateUniq.cpp + array/arrayEnumerateUniqRanked.cpp + array/arrayExists.cpp + array/arrayFill.cpp + array/arrayFilter.cpp + array/arrayFirst.cpp + array/arrayFirstIndex.cpp + array/arrayFlatten.cpp + array/arrayIntersect.cpp + array/arrayJoin.cpp + array/arrayMap.cpp + array/arrayPopBack.cpp + array/arrayPopFront.cpp + array/arrayPushBack.cpp + array/arrayPushFront.cpp + array/arrayReduce.cpp + array/arrayReduceInRanges.cpp + array/arrayResize.cpp + array/arrayReverse.cpp + array/arraySlice.cpp + array/arraySort.cpp + array/arraySplit.cpp + array/arraySum.cpp + array/arrayUniq.cpp + array/arrayWithConstant.cpp + array/arrayZip.cpp + array/countEqual.cpp + array/emptyArray.cpp + array/emptyArrayToSingle.cpp + array/has.cpp + array/hasAll.cpp + array/hasAny.cpp + array/indexOf.cpp + array/length.cpp + array/range.cpp + array/registerFunctionsArray.cpp + GatherUtils/concat.cpp + GatherUtils/createArraySink.cpp + GatherUtils/createArraySource.cpp + GatherUtils/createValueSource.cpp + GatherUtils/has.cpp + GatherUtils/push.cpp + GatherUtils/resizeConstantSize.cpp + GatherUtils/resizeDynamicSize.cpp + GatherUtils/sliceDynamicOffsetBounded.cpp + GatherUtils/sliceDynamicOffsetUnbounded.cpp + GatherUtils/sliceFromLeftConstantOffsetBounded.cpp + GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp + GatherUtils/sliceFromRightConstantOffsetBounded.cpp + GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp + URL/basename.cpp + URL/cutFragment.cpp + URL/cutQueryString.cpp + URL/cutQueryStringAndFragment.cpp + URL/cutToFirstSignificantSubdomain.cpp + URL/cutURLParameter.cpp + URL/cutWWW.cpp + URL/decodeURLComponent.cpp + URL/domain.cpp + URL/domainWithoutWWW.cpp + URL/extractURLParameter.cpp + URL/extractURLParameterNames.cpp + URL/extractURLParameters.cpp + URL/firstSignificantSubdomain.cpp + URL/fragment.cpp + URL/path.cpp + URL/pathFull.cpp + URL/protocol.cpp + URL/queryString.cpp + URL/queryStringAndFragment.cpp + URL/registerFunctionsURL.cpp + URL/tldLookup.generated.cpp + URL/topLevelDomain.cpp + URL/URLHierarchy.cpp + URL/URLPathHierarchy.cpp + abs.cpp + acos.cpp + addDays.cpp + addHours.cpp + addMinutes.cpp + addMonths.cpp + addQuarters.cpp + addressToLine.cpp + addressToSymbol.cpp + addSeconds.cpp + addWeeks.cpp + addYears.cpp + appendTrailingCharIfAbsent.cpp + asin.cpp + assumeNotNull.cpp + atan.cpp + bar.cpp + base64Decode.cpp + base64Encode.cpp + bitAnd.cpp + bitBoolMaskAnd.cpp + bitBoolMaskOr.cpp + bitCount.cpp + bitNot.cpp + bitOr.cpp + bitRotateLeft.cpp + bitRotateRight.cpp + bitShiftLeft.cpp + bitShiftRight.cpp + bitSwapLastTwo.cpp + bitTest.cpp + bitTestAll.cpp + bitTestAny.cpp + bitWrapperFunc.cpp + bitXor.cpp + blockNumber.cpp + blockSerializedSize.cpp + blockSize.cpp + caseWithExpression.cpp + cbrt.cpp + coalesce.cpp + concat.cpp + convertCharset.cpp + cos.cpp + CRC.cpp + currentDatabase.cpp + currentQuota.cpp + currentRowPolicies.cpp + currentUser.cpp + dateDiff.cpp + defaultValueOfArgumentType.cpp + demange.cpp + divide.cpp + dumpColumnStructure.cpp + e.cpp + empty.cpp + endsWith.cpp + equals.cpp + erf.cpp + erfc.cpp + evalMLMethod.cpp + exp.cpp + exp2.cpp + exp10.cpp + extractTimeZoneFromFunctionArguments.cpp + filesystem.cpp + finalizeAggregation.cpp + formatDateTime.cpp + formatString.cpp + FunctionFactory.cpp + FunctionFQDN.cpp + FunctionHelpers.cpp + FunctionJoinGet.cpp + FunctionsCoding.cpp + FunctionsConversion.cpp + FunctionsEmbeddedDictionaries.cpp + FunctionsExternalDictionaries.cpp + FunctionsExternalModels.cpp + FunctionsFormatting.cpp + FunctionsHashing.cpp + FunctionsJSON.cpp + FunctionsLogical.cpp + FunctionsRandom.cpp + FunctionsRound.cpp + FunctionsStringArray.cpp + FunctionsStringRegex.cpp + FunctionsStringSimilarity.cpp + gcd.cpp + generateUUIDv4.cpp + GeoHash.cpp + geohashDecode.cpp + geohashEncode.cpp + geohashesInBox.cpp + getMacro.cpp + getScalar.cpp + getSizeOfEnumType.cpp + greatCircleDistance.cpp + greater.cpp + greaterOrEquals.cpp + greatest.cpp + hasColumnInTable.cpp + hasToken.cpp + hasTokenCaseInsensitive.cpp + hostName.cpp + identity.cpp + if.cpp + ifNotFinite.cpp + ifNull.cpp + IFunction.cpp + ignore.cpp + ignoreExceptNull.cpp + in.cpp + intDiv.cpp + intDivOrZero.cpp + intExp2.cpp + intExp10.cpp + isConstant.cpp + isFinite.cpp + isInfinite.cpp + isNaN.cpp + isNotNull.cpp + isNull.cpp + isValidUTF8.cpp + jumpConsistentHash.cpp + lcm.cpp + least.cpp + lengthUTF8.cpp + less.cpp + lessOrEquals.cpp + lgamma.cpp + log.cpp + log2.cpp + log10.cpp + lowCardinalityIndices.cpp + lowCardinalityKeys.cpp + lower.cpp + lowerUTF8.cpp + materialize.cpp + minus.cpp + modulo.cpp + moduloOrZero.cpp + multiIf.cpp + multiply.cpp + multiSearchAllPositions.cpp + multiSearchAllPositionsCaseInsensitive.cpp + multiSearchAllPositionsCaseInsensitiveUTF8.cpp + multiSearchAllPositionsUTF8.cpp + multiSearchAny.cpp + multiSearchAnyCaseInsensitive.cpp + multiSearchAnyCaseInsensitiveUTF8.cpp + multiSearchAnyUTF8.cpp + multiSearchFirstIndex.cpp + multiSearchFirstIndexCaseInsensitive.cpp + multiSearchFirstIndexCaseInsensitiveUTF8.cpp + multiSearchFirstIndexUTF8.cpp + multiSearchFirstPosition.cpp + multiSearchFirstPositionCaseInsensitive.cpp + multiSearchFirstPositionCaseInsensitiveUTF8.cpp + multiSearchFirstPositionUTF8.cpp + negate.cpp + neighbor.cpp + notEmpty.cpp + notEquals.cpp + now.cpp + now64.cpp + nullIf.cpp + pi.cpp + plus.cpp + pointInEllipses.cpp + pointInPolygon.cpp + position.cpp + positionCaseInsensitive.cpp + positionCaseInsensitiveUTF8.cpp + positionUTF8.cpp + pow.cpp + rand.cpp + rand64.cpp + randConstant.cpp + randomPrintableASCII.cpp + regexpQuoteMeta.cpp + registerFunctions.cpp + registerFunctionsArithmetic.cpp + registerFunctionsComparison.cpp + registerFunctionsConditional.cpp + registerFunctionsConsistentHashing.cpp + registerFunctionsDateTime.cpp + registerFunctionsGeo.cpp + registerFunctionsHigherOrder.cpp + registerFunctionsIntrospection.cpp + registerFunctionsMath.cpp + registerFunctionsMiscellaneous.cpp + registerFunctionsNull.cpp + registerFunctionsRandom.cpp + registerFunctionsReinterpret.cpp + registerFunctionsString.cpp + registerFunctionsStringSearch.cpp + registerFunctionsTuple.cpp + registerFunctionsVisitParam.cpp + reinterpretAsFixedString.cpp + reinterpretAsString.cpp + reinterpretStringAs.cpp + repeat.cpp + replicate.cpp + reverse.cpp + reverseUTF8.cpp + roundAge.cpp + roundDuration.cpp + roundToExp2.cpp + rowNumberInAllBlocks.cpp + rowNumberInBlock.cpp + runningAccumulate.cpp + runningDifference.cpp + runningDifferenceStartingWithFirstValue.cpp + sigmoid.cpp + sin.cpp + sleep.cpp + sleepEachRow.cpp + sqrt.cpp + startsWith.cpp + substring.cpp + subtractDays.cpp + subtractHours.cpp + subtractMinutes.cpp + subtractMonths.cpp + subtractQuarters.cpp + subtractSeconds.cpp + subtractWeeks.cpp + subtractYears.cpp + tan.cpp + tanh.cpp + tgamma.cpp + throwIf.cpp + timeSlot.cpp + timeSlots.cpp + timezone.cpp + toColumnTypeName.cpp + toCustomWeek.cpp + today.cpp + toDayOfMonth.cpp + toDayOfWeek.cpp + toDayOfYear.cpp + toHour.cpp + toISOWeek.cpp + toISOYear.cpp + toLowCardinality.cpp + toMinute.cpp + toMonday.cpp + toMonth.cpp + toNullable.cpp + toQuarter.cpp + toRelativeDayNum.cpp + toRelativeHourNum.cpp + toRelativeMinuteNum.cpp + toRelativeMonthNum.cpp + toRelativeQuarterNum.cpp + toRelativeSecondNum.cpp + toRelativeWeekNum.cpp + toRelativeYearNum.cpp + toSecond.cpp + toStartOfDay.cpp + toStartOfFifteenMinutes.cpp + toStartOfFiveMinute.cpp + toStartOfHour.cpp + toStartOfInterval.cpp + toStartOfISOYear.cpp + toStartOfMinute.cpp + toStartOfMonth.cpp + toStartOfQuarter.cpp + toStartOfTenMinutes.cpp + toStartOfYear.cpp + toTime.cpp + toTimeZone.cpp + toTypeName.cpp + toValidUTF8.cpp + toYear.cpp + toYYYYMM.cpp + toYYYYMMDD.cpp + toYYYYMMDDhhmmss.cpp + transform.cpp + trap.cpp + trim.cpp + tryBase64Decode.cpp + tuple.cpp + tupleElement.cpp + upper.cpp + upperUTF8.cpp + uptime.cpp + version.cpp + visibleWidth.cpp + visitParamExtractBool.cpp + visitParamExtractFloat.cpp + visitParamExtractInt.cpp + visitParamExtractRaw.cpp + visitParamExtractString.cpp + visitParamExtractUInt.cpp + visitParamHas.cpp + yandexConsistentHash.cpp + yesterday.cpp +) + +END() diff --git a/src/IO/BrotliReadBuffer.cpp b/src/IO/BrotliReadBuffer.cpp index f68424cd54e..ee888b8af22 100644 --- a/src/IO/BrotliReadBuffer.cpp +++ b/src/IO/BrotliReadBuffer.cpp @@ -1,8 +1,10 @@ -#include -#if USE_BROTLI +#if !defined(ARCADIA_BUILD) +# include +#endif -#include "BrotliReadBuffer.h" -#include +#if USE_BROTLI +# include +# include "BrotliReadBuffer.h" namespace DB { diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index ac1e2b3c188..e3001409e86 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -1,8 +1,10 @@ -#include -#if USE_BROTLI +#if !defined(ARCADIA_BUILD) +# include +#endif -#include -#include +#if USE_BROTLI +# include +# include namespace DB { diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index 20f1ea44301..a0a5e19f4fa 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -7,7 +7,9 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index d8ce3427c91..97a0525a9b8 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -9,14 +8,18 @@ #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_NETSSL -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include #endif #include diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index f5e6f91537d..2d4e3f0472e 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -14,10 +14,13 @@ #include #include #include -#include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800 #define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1 @@ -302,4 +305,3 @@ public: }; } - diff --git a/src/IO/UseSSL.cpp b/src/IO/UseSSL.cpp index 2aad0ef49e2..6770de2cbad 100644 --- a/src/IO/UseSSL.cpp +++ b/src/IO/UseSSL.cpp @@ -1,9 +1,11 @@ #include "UseSSL.h" -#include +#if !defined(ARCADIA_BUILD) +# include +#endif #if USE_POCO_NETSSL -#include +# include #endif namespace DB diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index 77198ca93ea..0f30f1352e3 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -7,7 +7,10 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index 1307c007656..ffa36c11c5b 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -13,7 +13,10 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif namespace Poco diff --git a/src/IO/ya.make b/src/IO/ya.make new file mode 100644 index 00000000000..81bf6060a94 --- /dev/null +++ b/src/IO/ya.make @@ -0,0 +1,59 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/brotli/dec + contrib/libs/brotli/enc + contrib/libs/poco/NetSSL_OpenSSL +) + +SRCS( + AIO.cpp + AIOContextPool.cpp + BrotliReadBuffer.cpp + BrotliWriteBuffer.cpp + CascadeWriteBuffer.cpp + CompressionMethod.cpp + copyData.cpp + createReadBufferFromFileBase.cpp + createWriteBufferFromFileBase.cpp + DoubleConverter.cpp + HashingWriteBuffer.cpp + HexWriteBuffer.cpp + HTTPCommon.cpp + LimitReadBuffer.cpp + MemoryReadWriteBuffer.cpp + MMapReadBufferFromFile.cpp + MMapReadBufferFromFileDescriptor.cpp + NullWriteBuffer.cpp + parseDateTimeBestEffort.cpp + PeekableReadBuffer.cpp + Progress.cpp + ReadBufferAIO.cpp + ReadBufferFromFile.cpp + ReadBufferFromFileBase.cpp + ReadBufferFromFileDescriptor.cpp + ReadBufferFromIStream.cpp + ReadBufferFromMemory.cpp + ReadBufferFromPocoSocket.cpp + readFloatText.cpp + ReadHelpers.cpp + ReadWriteBufferFromHTTP.cpp + UseSSL.cpp + WriteBufferAIO.cpp + WriteBufferFromFile.cpp + WriteBufferFromFileBase.cpp + WriteBufferFromFileDescriptor.cpp + WriteBufferFromFileDescriptorDiscardOnFailure.cpp + WriteBufferFromHTTP.cpp + WriteBufferFromHTTPServerResponse.cpp + WriteBufferFromOStream.cpp + WriteBufferFromPocoSocket.cpp + WriteBufferFromTemporaryFile.cpp + WriteBufferValidUTF8.cpp + WriteHelpers.cpp + ZlibDeflatingWriteBuffer.cpp + ZlibInflatingReadBuffer.cpp +) + +END() diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 07c1d7476ad..03fc04d2152 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -25,11 +25,14 @@ #include #include #include -#include #include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace ProfileEvents { diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 35a352b67b6..304db048f4f 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -5,7 +5,6 @@ #include #include #include -#include "config_core.h" #include #include #include @@ -13,12 +12,12 @@ #include #include -#if __has_include() -#include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" #endif #if USE_JEMALLOC - #include +# include #endif @@ -216,9 +215,9 @@ void AsynchronousMetrics::update() set("NumberOfTables", total_number_of_tables); } -#if USE_JEMALLOC +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 4 { - #define FOR_EACH_METRIC(M) \ +# define FOR_EACH_METRIC(M) \ M("allocated", size_t) \ M("active", size_t) \ M("metadata", size_t) \ @@ -228,9 +227,9 @@ void AsynchronousMetrics::update() M("retained", size_t) \ M("background_thread.num_threads", size_t) \ M("background_thread.num_runs", uint64_t) \ - M("background_thread.run_interval", uint64_t) \ + M("background_thread.run_interval", uint64_t) - #define GET_METRIC(NAME, TYPE) \ +# define GET_METRIC(NAME, TYPE) \ do \ { \ TYPE value{}; \ @@ -241,13 +240,12 @@ void AsynchronousMetrics::update() FOR_EACH_METRIC(GET_METRIC) - #undef GET_METRIC - #undef FOR_EACH_METRIC +# undef GET_METRIC +# undef FOR_EACH_METRIC } #endif /// Add more metrics as you wish. } - } diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index ed806e5ad57..909582aa308 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -6,9 +6,12 @@ #include #include #include -#include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f68b08bf9f0..6cf1a066b18 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -13,7 +13,6 @@ #include #include #include -#include "config_core.h" #include #include #include @@ -25,6 +24,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace Poco { diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 435e493ffa9..447c54c769a 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1,4 +1,3 @@ -#include "config_core.h" #include #include #include @@ -15,6 +14,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace ProfileEvents { diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 0c3027dfbab..5a29eaaab9e 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -7,12 +7,15 @@ #include #include #include -#include "config_core.h" #include #include #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace DB { diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index 995fb35e52c..0a103aab378 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -1,13 +1,15 @@ #pragma once -#include "config_core.h" -#if USE_EMBEDDED_COMPILER +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif -#include -#include -#include -#include -#include +#if USE_EMBEDDED_COMPILER +# include +# include +# include +# include +# include namespace DB diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 02388028c73..889b1c58b55 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -1,6 +1,9 @@ #include #include -#include "config_core.h" + +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif #if USE_MYSQL # include @@ -32,9 +35,9 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( void ExternalDictionariesLoader::resetAll() { - #if USE_MYSQL - mysqlxx::PoolFactory::instance().reset(); - #endif +#if USE_MYSQL + mysqlxx::PoolFactory::instance().reset(); +#endif } } diff --git a/src/Interpreters/InterpreterSetRoleQuery.cpp b/src/Interpreters/InterpreterSetRoleQuery.cpp index 2a6f2317a9c..8f085d66c4c 100644 --- a/src/Interpreters/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/InterpreterSetRoleQuery.cpp @@ -50,7 +50,7 @@ void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) { for (const auto & id : roles_from_query.getMatchingIDs()) { - if (!user->granted_roles.contains(id)) + if (!user->granted_roles.count(id)) throw Exception("Role should be granted to set current", ErrorCodes::SET_NON_GRANTED_ROLE); new_current_roles.push_back(id); } @@ -85,7 +85,7 @@ void InterpreterSetRoleQuery::updateUserSetDefaultRoles(User & user, const Exten { for (const auto & id : roles_from_query.getMatchingIDs()) { - if (!user.granted_roles.contains(id)) + if (!user.granted_roles.count(id)) throw Exception("Role should be granted to set default", ErrorCodes::SET_NON_GRANTED_ROLE); } } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 36713be1ff7..01a84559e92 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "config_core.h" #include #include #include @@ -32,6 +31,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace DB { diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make new file mode 100644 index 00000000000..7eb8a899e9c --- /dev/null +++ b/src/Interpreters/ya.make @@ -0,0 +1,138 @@ +LIBRARY() + +ADDINCL( + contrib/libs/libdivide + contrib/libs/pdqsort +) + +PEERDIR( + clickhouse/src/Core + contrib/libs/libdivide + contrib/libs/pdqsort +) + +NO_COMPILER_WARNINGS() + +SRCS( + ActionLocksManager.cpp + ActionsVisitor.cpp + addMissingDefaults.cpp + addTypeConversionToAST.cpp + Aggregator.cpp + ArrayJoinAction.cpp + AsynchronousMetrics.cpp + BloomFilter.cpp + castColumn.cpp + CatBoostModel.cpp + ClientInfo.cpp + Cluster.cpp + ClusterProxy/executeQuery.cpp + ClusterProxy/SelectStreamFactory.cpp + CollectJoinOnKeysVisitor.cpp + ColumnNamesContext.cpp + Context.cpp + convertFieldToType.cpp + createBlockSelector.cpp + CrossToInnerJoinVisitor.cpp + DatabaseAndTableWithAlias.cpp + DatabaseCatalog.cpp + DDLWorker.cpp + DNSCacheUpdater.cpp + EmbeddedDictionaries.cpp + evaluateConstantExpression.cpp + executeQuery.cpp + ExecuteScalarSubqueriesVisitor.cpp + ExpressionActions.cpp + ExpressionAnalyzer.cpp + ExternalDictionariesLoader.cpp + ExternalLoader.cpp + ExternalLoaderDatabaseConfigRepository.cpp + ExternalLoaderTempConfigRepository.cpp + ExternalLoaderXMLConfigRepository.cpp + ExternalModelsLoader.cpp + ExtractExpressionInfoVisitor.cpp + FillingRow.cpp + getClusterName.cpp + getTableExpressions.cpp + HashJoin.cpp + IdentifierSemantic.cpp + IExternalLoadable.cpp + InJoinSubqueriesPreprocessor.cpp + inplaceBlockConversions.cpp + InternalTextLogsQueue.cpp + InterpreterAlterQuery.cpp + InterpreterCheckQuery.cpp + InterpreterCreateQuery.cpp + InterpreterCreateQuotaQuery.cpp + InterpreterCreateRoleQuery.cpp + InterpreterCreateRowPolicyQuery.cpp + InterpreterCreateSettingsProfileQuery.cpp + InterpreterCreateUserQuery.cpp + InterpreterDescribeQuery.cpp + InterpreterDropAccessEntityQuery.cpp + InterpreterDropQuery.cpp + InterpreterExistsQuery.cpp + InterpreterExplainQuery.cpp + InterpreterFactory.cpp + InterpreterGrantQuery.cpp + InterpreterInsertQuery.cpp + InterpreterKillQueryQuery.cpp + InterpreterOptimizeQuery.cpp + InterpreterRenameQuery.cpp + InterpreterSelectQuery.cpp + InterpreterSelectWithUnionQuery.cpp + InterpreterSetQuery.cpp + InterpreterSetRoleQuery.cpp + InterpreterShowCreateAccessEntityQuery.cpp + InterpreterShowCreateQuery.cpp + InterpreterShowGrantsQuery.cpp + InterpreterShowProcesslistQuery.cpp + InterpreterShowQuotasQuery.cpp + InterpreterShowRowPoliciesQuery.cpp + InterpreterShowTablesQuery.cpp + InterpreterSystemQuery.cpp + InterpreterUseQuery.cpp + InterpreterWatchQuery.cpp + interpretSubquery.cpp + join_common.cpp + JoinedTables.cpp + JoinSwitcher.cpp + JoinToSubqueryTransformVisitor.cpp + loadMetadata.cpp + LogicalExpressionsOptimizer.cpp + MarkTableIdentifiersVisitor.cpp + MergeJoin.cpp + MetricLog.cpp + MutationsInterpreter.cpp + NullableUtils.cpp + OptimizeIfChains.cpp + OptimizeIfWithConstantConditionVisitor.cpp + PartLog.cpp + PredicateExpressionsOptimizer.cpp + PredicateRewriteVisitor.cpp + ProcessList.cpp + ProfileEventsExt.cpp + QueryAliasesVisitor.cpp + QueryLog.cpp + QueryNormalizer.cpp + QueryThreadLog.cpp + RenameColumnVisitor.cpp + ReplaceQueryParameterVisitor.cpp + RequiredSourceColumnsVisitor.cpp + RowRefs.cpp + Set.cpp + SetVariants.cpp + sortBlock.cpp + StorageID.cpp + SubqueryForSet.cpp + SyntaxAnalyzer.cpp + SystemLog.cpp + TableJoin.cpp + TablesStatus.cpp + TextLog.cpp + ThreadStatusExt.cpp + TraceLog.cpp + TranslateQualifiedNamesVisitor.cpp +) + +END() diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index f5b9afde4b3..eac96e50cb1 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -1,9 +1,12 @@ #pragma once -#include "config_core.h" #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace DB { diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make new file mode 100644 index 00000000000..942124c2c7a --- /dev/null +++ b/src/Parsers/ya.make @@ -0,0 +1,113 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + ASTAlterQuery.cpp + ASTAsterisk.cpp + ASTColumnDeclaration.cpp + ASTColumnsMatcher.cpp + ASTConstraintDeclaration.cpp + ASTCreateQuery.cpp + ASTCreateQuotaQuery.cpp + ASTCreateRoleQuery.cpp + ASTCreateRowPolicyQuery.cpp + ASTCreateSettingsProfileQuery.cpp + ASTCreateUserQuery.cpp + ASTDictionary.cpp + ASTDictionaryAttributeDeclaration.cpp + ASTDropAccessEntityQuery.cpp + ASTDropQuery.cpp + ASTExpressionList.cpp + ASTExtendedRoleSet.cpp + ASTFunction.cpp + ASTFunctionWithKeyValueArguments.cpp + ASTGrantQuery.cpp + ASTIdentifier.cpp + ASTInsertQuery.cpp + ASTKillQueryQuery.cpp + ASTLiteral.cpp + ASTOptimizeQuery.cpp + ASTOrderByElement.cpp + ASTPartition.cpp + ASTQualifiedAsterisk.cpp + ASTQueryParameter.cpp + ASTQueryWithOnCluster.cpp + ASTQueryWithOutput.cpp + ASTQueryWithTableAndOutput.cpp + ASTSampleRatio.cpp + ASTSelectQuery.cpp + ASTSelectWithUnionQuery.cpp + ASTSetRoleQuery.cpp + ASTSettingsProfileElement.cpp + ASTShowCreateAccessEntityQuery.cpp + ASTShowGrantsQuery.cpp + ASTShowQuotasQuery.cpp + ASTShowRowPoliciesQuery.cpp + ASTShowTablesQuery.cpp + ASTSubquery.cpp + ASTSystemQuery.cpp + ASTTablesInSelectQuery.cpp + ASTTTLElement.cpp + ASTWithAlias.cpp + CommonParsers.cpp + ExpressionElementParsers.cpp + ExpressionListParsers.cpp + formatAST.cpp + IAST.cpp + iostream_debug_helpers.cpp + IParserBase.cpp + Lexer.cpp + makeASTForLogicalFunction.cpp + parseDatabaseAndTableName.cpp + parseIdentifierOrStringLiteral.cpp + parseIntervalKind.cpp + parseQuery.cpp + ParserAlterQuery.cpp + ParserCase.cpp + ParserCheckQuery.cpp + ParserCreateQuery.cpp + ParserCreateQuotaQuery.cpp + ParserCreateRoleQuery.cpp + ParserCreateRowPolicyQuery.cpp + ParserCreateSettingsProfileQuery.cpp + ParserCreateUserQuery.cpp + ParserDescribeTableQuery.cpp + ParserDictionary.cpp + ParserDictionaryAttributeDeclaration.cpp + ParserDropAccessEntityQuery.cpp + ParserDropQuery.cpp + ParserExtendedRoleSet.cpp + ParserGrantQuery.cpp + ParserInsertQuery.cpp + ParserKillQueryQuery.cpp + ParserOptimizeQuery.cpp + ParserPartition.cpp + ParserQuery.cpp + ParserQueryWithOutput.cpp + ParserRenameQuery.cpp + ParserSampleRatio.cpp + ParserSelectQuery.cpp + ParserSelectWithUnionQuery.cpp + ParserSetQuery.cpp + ParserSetRoleQuery.cpp + ParserSettingsProfileElement.cpp + ParserShowCreateAccessEntityQuery.cpp + ParserShowGrantsQuery.cpp + ParserShowQuotasQuery.cpp + ParserShowRowPoliciesQuery.cpp + ParserShowTablesQuery.cpp + ParserSystemQuery.cpp + ParserTablePropertiesQuery.cpp + ParserTablesInSelectQuery.cpp + ParserUnionQueryElement.cpp + ParserUseQuery.cpp + ParserWatchQuery.cpp + parseUserName.cpp + queryToString.cpp + TokenIterator.cpp +) + +END() diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 029b2c8329e..8dd36722efc 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -1,11 +1,13 @@ #pragma once -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include namespace DB { diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index ae8ebbf28f4..f216af01d80 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -1,13 +1,15 @@ #pragma once -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include +# include namespace google diff --git a/src/Processors/ya.make b/src/Processors/ya.make new file mode 100644 index 00000000000..9822344aec1 --- /dev/null +++ b/src/Processors/ya.make @@ -0,0 +1,100 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/msgpack + contrib/libs/protobuf_std +) + +SRCS( + Chunk.cpp + ConcatProcessor.cpp + DelayedPortsProcessor.cpp + Executors/ParallelPipelineExecutor.cpp + Executors/PipelineExecutor.cpp + Executors/SequentialPipelineExecutor.cpp + Executors/TreeExecutorBlockInputStream.cpp + ForkProcessor.cpp + Formats/IInputFormat.cpp + Formats/Impl/BinaryRowInputFormat.cpp + Formats/Impl/BinaryRowOutputFormat.cpp + Formats/Impl/ConstantExpressionTemplate.cpp + Formats/Impl/CSVRowInputFormat.cpp + Formats/Impl/CSVRowOutputFormat.cpp + Formats/Impl/JSONCompactEachRowRowInputFormat.cpp + Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp + Formats/Impl/JSONCompactRowOutputFormat.cpp + Formats/Impl/JSONEachRowRowInputFormat.cpp + Formats/Impl/JSONEachRowRowOutputFormat.cpp + Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp + Formats/Impl/JSONRowOutputFormat.cpp + Formats/Impl/MsgPackRowInputFormat.cpp + Formats/Impl/MsgPackRowOutputFormat.cpp + Formats/Impl/MySQLOutputFormat.cpp + Formats/Impl/NativeFormat.cpp + Formats/Impl/NullFormat.cpp + Formats/Impl/ODBCDriver2BlockOutputFormat.cpp + Formats/Impl/ODBCDriverBlockOutputFormat.cpp + Formats/Impl/PrettyBlockOutputFormat.cpp + Formats/Impl/PrettyCompactBlockOutputFormat.cpp + Formats/Impl/PrettySpaceBlockOutputFormat.cpp + Formats/Impl/ProtobufRowInputFormat.cpp + Formats/Impl/ProtobufRowOutputFormat.cpp + Formats/Impl/RegexpRowInputFormat.cpp + Formats/Impl/TabSeparatedRowInputFormat.cpp + Formats/Impl/TabSeparatedRowOutputFormat.cpp + Formats/Impl/TemplateBlockOutputFormat.cpp + Formats/Impl/TemplateRowInputFormat.cpp + Formats/Impl/TSKVRowInputFormat.cpp + Formats/Impl/TSKVRowOutputFormat.cpp + Formats/Impl/ValuesBlockInputFormat.cpp + Formats/Impl/ValuesRowOutputFormat.cpp + Formats/Impl/VerticalRowOutputFormat.cpp + Formats/Impl/XMLRowOutputFormat.cpp + Formats/IOutputFormat.cpp + Formats/IRowInputFormat.cpp + Formats/IRowOutputFormat.cpp + Formats/LazyOutputFormat.cpp + Formats/OutputStreamToOutputFormat.cpp + Formats/RowInputFormatWithDiagnosticInfo.cpp + IAccumulatingTransform.cpp + IInflatingTransform.cpp + IProcessor.cpp + ISimpleTransform.cpp + ISink.cpp + ISource.cpp + LimitTransform.cpp + Pipe.cpp + Port.cpp + QueryPipeline.cpp + ResizeProcessor.cpp + Sources/SinkToOutputStream.cpp + Sources/SourceFromInputStream.cpp + Sources/SourceWithProgress.cpp + Transforms/AddingMissedTransform.cpp + Transforms/AggregatingTransform.cpp + Transforms/ConvertingTransform.cpp + Transforms/CreatingSetsTransform.cpp + Transforms/CubeTransform.cpp + Transforms/DistinctTransform.cpp + Transforms/ExpressionTransform.cpp + Transforms/ExtremesTransform.cpp + Transforms/FillingTransform.cpp + Transforms/FilterTransform.cpp + Transforms/FinishSortingTransform.cpp + Transforms/InflatingExpressionTransform.cpp + Transforms/LimitByTransform.cpp + Transforms/LimitsCheckingTransform.cpp + Transforms/MaterializingTransform.cpp + Transforms/MergeSortingTransform.cpp + Transforms/MergingAggregatedMemoryEfficientTransform.cpp + Transforms/MergingAggregatedTransform.cpp + Transforms/MergingSortedTransform.cpp + Transforms/PartialSortingTransform.cpp + Transforms/ReverseTransform.cpp + Transforms/RollupTransform.cpp + Transforms/SortingTransform.cpp + Transforms/TotalsHavingTransform.cpp +) + +END() diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 5a792080370..fc69123e696 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -126,8 +126,13 @@ Block IStorage::getSampleBlockForColumns(const Names & column_names) const namespace { - using NamesAndTypesMap = ::google::dense_hash_map; - using UniqueStrings = ::google::dense_hash_set; +#if !defined(ARCADIA_BUILD) + using NamesAndTypesMap = google::dense_hash_map; + using UniqueStrings = google::dense_hash_set; +#else + using NamesAndTypesMap = google::sparsehash::dense_hash_map; + using UniqueStrings = google::sparsehash::dense_hash_set; +#endif String listOfColumns(const NamesAndTypesList & available_columns) { diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index e42ac942362..6887b7c0d9d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -208,8 +208,7 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx else if (element.function == RPNElement::FUNCTION_EQUALS || element.function == RPNElement::FUNCTION_NOT_EQUALS) { - rpn_stack.emplace_back( - granule->bloom_filters[element.key_column].contains(*element.bloom_filter), true); + rpn_stack.emplace_back(granule->bloom_filters[element.key_column].contains(*element.bloom_filter), true); if (element.function == RPNElement::FUNCTION_NOT_EQUALS) rpn_stack.back() = !rpn_stack.back(); diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 3bde1cab4c7..8b98536e4d7 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -1,13 +1,16 @@ #pragma once -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_MYSQL -#include +# include -#include -#include -#include +# include +# include +# include namespace DB diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp b/src/Storages/System/StorageSystemBuildOptions.cpp index 73d1c91f422..4dd4ff90572 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp +++ b/src/Storages/System/StorageSystemBuildOptions.cpp @@ -18,11 +18,15 @@ NamesAndTypesList StorageSystemBuildOptions::getNamesAndTypes() void StorageSystemBuildOptions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { +#if !defined(ARCADIA_BUILD) for (auto it = auto_config_build; *it; it += 2) { res_columns[0]->insert(it[0]); res_columns[1]->insert(it[1]); } +#else + UNUSED(res_columns); +#endif } } diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index f5fab52285d..5ad26b70803 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -1,6 +1,3 @@ -#include -#include "config_core.h" - #include #include diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index 63a758f5b38..c9874551073 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -1,6 +1,9 @@ #pragma once -#include -#include "config_core.h" + +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif namespace DB { diff --git a/src/Storages/ya.make b/src/Storages/ya.make new file mode 100644 index 00000000000..6a032d82bdc --- /dev/null +++ b/src/Storages/ya.make @@ -0,0 +1,167 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/sparsehash +) + +SRCS( + Distributed/DirectoryMonitor.cpp + Distributed/DistributedBlockOutputStream.cpp + LiveView/StorageLiveView.cpp + MergeTree/ActiveDataPartSet.cpp + MergeTree/AllMergeSelector.cpp + MergeTree/BackgroundProcessingPool.cpp + MergeTree/BoolMask.cpp + MergeTree/checkDataPart.cpp + MergeTree/DataPartsExchange.cpp + MergeTree/EphemeralLockInZooKeeper.cpp + MergeTree/IMergedBlockOutputStream.cpp + MergeTree/IMergeTreeDataPart.cpp + MergeTree/IMergeTreeDataPartWriter.cpp + MergeTree/IMergeTreeReader.cpp + MergeTree/KeyCondition.cpp + MergeTree/LevelMergeSelector.cpp + MergeTree/localBackup.cpp + MergeTree/MergedBlockOutputStream.cpp + MergeTree/MergedColumnOnlyOutputStream.cpp + MergeTree/MergeList.cpp + MergeTree/MergeTreeBaseSelectProcessor.cpp + MergeTree/MergeTreeBlockOutputStream.cpp + MergeTree/MergeTreeBlockReadUtils.cpp + MergeTree/MergeTreeData.cpp + MergeTree/MergeTreeDataMergerMutator.cpp + MergeTree/MergeTreeDataPartChecksum.cpp + MergeTree/MergeTreeDataPartCompact.cpp + MergeTree/MergeTreeDataPartTTLInfo.cpp + MergeTree/MergeTreeDataPartType.cpp + MergeTree/MergeTreeDataPartWide.cpp + MergeTree/MergeTreeDataPartWriterCompact.cpp + MergeTree/MergeTreeDataPartWriterWide.cpp + MergeTree/MergeTreeDataSelectExecutor.cpp + MergeTree/MergeTreeDataWriter.cpp + MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp + MergeTree/MergeTreeIndexBloomFilter.cpp + MergeTree/MergeTreeIndexConditionBloomFilter.cpp + MergeTree/MergeTreeIndexFullText.cpp + MergeTree/MergeTreeIndexGranularity.cpp + MergeTree/MergeTreeIndexGranularityInfo.cpp + MergeTree/MergeTreeIndexGranuleBloomFilter.cpp + MergeTree/MergeTreeIndexMinMax.cpp + MergeTree/MergeTreeIndexReader.cpp + MergeTree/MergeTreeIndexSet.cpp + MergeTree/MergeTreeIndices.cpp + MergeTree/MergeTreeMarksLoader.cpp + MergeTree/MergeTreeMutationEntry.cpp + MergeTree/MergeTreePartInfo.cpp + MergeTree/MergeTreePartition.cpp + MergeTree/MergeTreePartsMover.cpp + MergeTree/MergeTreeRangeReader.cpp + MergeTree/MergeTreeReaderCompact.cpp + MergeTree/MergeTreeReaderStream.cpp + MergeTree/MergeTreeReaderWide.cpp + MergeTree/MergeTreeReadPool.cpp + MergeTree/MergeTreeReverseSelectProcessor.cpp + MergeTree/MergeTreeSelectProcessor.cpp + MergeTree/MergeTreeSequentialBlockInputStream.cpp + MergeTree/MergeTreeSettings.cpp + MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp + MergeTree/MergeTreeWhereOptimizer.cpp + MergeTree/registerStorageMergeTree.cpp + MergeTree/ReplicatedMergeTreeAddress.cpp + MergeTree/ReplicatedMergeTreeAltersSequence.cpp + MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp + MergeTree/ReplicatedMergeTreeCleanupThread.cpp + MergeTree/ReplicatedMergeTreeLogEntry.cpp + MergeTree/ReplicatedMergeTreeMutationEntry.cpp + MergeTree/ReplicatedMergeTreePartCheckThread.cpp + MergeTree/ReplicatedMergeTreePartHeader.cpp + MergeTree/ReplicatedMergeTreeQueue.cpp + MergeTree/ReplicatedMergeTreeRestartingThread.cpp + MergeTree/ReplicatedMergeTreeTableMetadata.cpp + MergeTree/SimpleMergeSelector.cpp + MergeTree/TTLMergeSelector.cpp + System/attachSystemTables.cpp + System/StorageSystemAggregateFunctionCombinators.cpp + System/StorageSystemAsynchronousMetrics.cpp + System/StorageSystemBuildOptions.cpp + System/StorageSystemClusters.cpp + System/StorageSystemCollations.cpp + System/StorageSystemColumns.cpp + System/StorageSystemContributors.cpp + System/StorageSystemContributors.generated.cpp + System/StorageSystemDatabases.cpp + System/StorageSystemDataTypeFamilies.cpp + System/StorageSystemDetachedParts.cpp + System/StorageSystemDictionaries.cpp + System/StorageSystemDisks.cpp + System/StorageSystemEvents.cpp + System/StorageSystemFormats.cpp + System/StorageSystemFunctions.cpp + System/StorageSystemGraphite.cpp + System/StorageSystemMacros.cpp + System/StorageSystemMerges.cpp + System/StorageSystemMergeTreeSettings.cpp + System/StorageSystemMetrics.cpp + System/StorageSystemModels.cpp + System/StorageSystemMutations.cpp + System/StorageSystemNumbers.cpp + System/StorageSystemOne.cpp + System/StorageSystemParts.cpp + System/StorageSystemPartsBase.cpp + System/StorageSystemPartsColumns.cpp + System/StorageSystemProcesses.cpp + System/StorageSystemQuotas.cpp + System/StorageSystemQuotaUsage.cpp + System/StorageSystemReplicas.cpp + System/StorageSystemReplicationQueue.cpp + System/StorageSystemRowPolicies.cpp + System/StorageSystemSettings.cpp + System/StorageSystemStackTrace.cpp + System/StorageSystemStoragePolicies.cpp + System/StorageSystemTableEngines.cpp + System/StorageSystemTableFunctions.cpp + System/StorageSystemTables.cpp + System/StorageSystemZeros.cpp + System/StorageSystemZooKeeper.cpp + AlterCommands.cpp + ColumnDefault.cpp + ColumnsDescription.cpp + ConstraintsDescription.cpp + getStructureOfRemoteTable.cpp + IndicesDescription.cpp + IStorage.cpp + MutationCommands.cpp + PartitionCommands.cpp + ReadInOrderOptimizer.cpp + registerStorages.cpp + StorageBuffer.cpp + StorageDictionary.cpp + StorageDistributed.cpp + StorageFactory.cpp + StorageFile.cpp + StorageGenerateRandom.cpp + StorageInMemoryMetadata.cpp + StorageInput.cpp + StorageJoin.cpp + StorageLog.cpp + StorageLogSettings.cpp + StorageMaterializedView.cpp + StorageMemory.cpp + StorageMerge.cpp + StorageMergeTree.cpp + StorageMySQL.cpp + StorageNull.cpp + StorageReplicatedMergeTree.cpp + StorageSet.cpp + StorageStripeLog.cpp + StorageTinyLog.cpp + StorageURL.cpp + StorageValues.cpp + StorageView.cpp + StorageXDBC.cpp + transformQueryForExternalDatabase.cpp + VirtualColumnUtils.cpp +) + +END() diff --git a/src/TableFunctions/ITableFunctionXDBC.h b/src/TableFunctions/ITableFunctionXDBC.h index 262c237bac2..547fda3f734 100644 --- a/src/TableFunctions/ITableFunctionXDBC.h +++ b/src/TableFunctions/ITableFunctionXDBC.h @@ -4,7 +4,10 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index be707c3520d..0afc74f163e 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -1,25 +1,27 @@ -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_MYSQL +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include "registerTableFunctions.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "registerTableFunctions.h" - -#include +# include namespace DB diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 3cde28685cc..8c54913676a 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -1,6 +1,9 @@ #pragma once -#include -#include "config_core.h" + +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif namespace DB { diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make new file mode 100644 index 00000000000..4d5d53ead88 --- /dev/null +++ b/src/TableFunctions/ya.make @@ -0,0 +1,26 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + ITableFunction.cpp + ITableFunctionFileLike.cpp + ITableFunctionXDBC.cpp + parseColumnsListForTableFunction.cpp + registerTableFunctions.cpp + TableFunctionFactory.cpp + TableFunctionFile.cpp + TableFunctionGenerateRandom.cpp + TableFunctionInput.cpp + TableFunctionMerge.cpp + TableFunctionMySQL.cpp + TableFunctionNumbers.cpp + TableFunctionRemote.cpp + TableFunctionURL.cpp + TableFunctionValues.cpp + TableFunctionZeros.cpp +) + +END() diff --git a/src/ya.make b/src/ya.make new file mode 100644 index 00000000000..95e5914bbe7 --- /dev/null +++ b/src/ya.make @@ -0,0 +1,26 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Access + clickhouse/src/AggregateFunctions + clickhouse/src/Client + clickhouse/src/Columns + clickhouse/src/Common + clickhouse/src/Compression + clickhouse/src/Core + clickhouse/src/Databases + clickhouse/src/DataStreams + clickhouse/src/DataTypes + clickhouse/src/Formats + clickhouse/src/Dictionaries + clickhouse/src/Disks + clickhouse/src/Functions + clickhouse/src/Interpreters + clickhouse/src/IO + clickhouse/src/Parsers + clickhouse/src/Processors + clickhouse/src/Storages + clickhouse/src/TableFunctions +) + +END() diff --git a/ya.make b/ya.make index 84f1a2163a4..7f7c1bbce2f 100644 --- a/ya.make +++ b/ya.make @@ -7,4 +7,6 @@ OWNER(g:clickhouse) RECURSE( base + programs + src ) From 2ef123bc0eaf3be734d7cc36c57c9e7c9ee77c32 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 16 Apr 2020 15:53:07 +0300 Subject: [PATCH 287/743] Replace one benchmark link --- website/templates/index/performance.html | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/website/templates/index/performance.html b/website/templates/index/performance.html index 9f31ad7a3ad..de54b05dd2d 100644 --- a/website/templates/index/performance.html +++ b/website/templates/index/performance.html @@ -38,12 +38,11 @@ rel="external nofollow noreferrer" target="_blank" class="text-reset">Geospatial processing with Clickhouse
Carto
  • ClickHouse and Vertica comparison
    zhtsh (machine translation from Chinese)
  • -
  • ClickHouse and InfiniDB comparison
    RamboLau (machine translation from Chinese)
  • +
  • MySQL vs ColumnStore vs ClickHouse
    Mafiree (machine translation from Chinese)
  • - - + {## TODO: ##} From 03c575934b4b818872704fe118e6945efe6dc88b Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 16 Apr 2020 15:55:32 +0300 Subject: [PATCH 288/743] Minor docs content improvements (#10304) * Add link to flink-clickhouse-sink + fixes nearby * Add link to dbal-doctrine * cleanup ru toc root * Sort out russian docs ToC a little bit * Add link collection of commercial support services --- docs/en/commercial/cloud.md | 5 ++ docs/en/commercial/support.md | 21 ++++++ docs/en/development/index.md | 2 +- .../en/interfaces/third-party/integrations.md | 8 ++- .../es/interfaces/third-party/integrations.md | 8 ++- .../fa/interfaces/third-party/integrations.md | 3 + .../fr/interfaces/third-party/integrations.md | 3 + .../ja/interfaces/third-party/integrations.md | 3 + docs/redirects.txt | 1 + docs/ru/commercial/index.md | 4 +- docs/ru/engines/index.md | 2 +- .../getting_started/example_datasets/index.md | 5 ++ docs/ru/getting_started/index.md | 7 ++ docs/ru/guides/index.md | 6 ++ docs/ru/index.md | 5 ++ docs/ru/interfaces/index.md | 6 ++ docs/ru/interfaces/third-party/index.md | 2 +- .../ru/interfaces/third-party/integrations.md | 70 ++++++++++--------- docs/ru/introduction/index.md | 2 +- docs/ru/operations/index.md | 6 ++ .../aggregate_functions/index.md | 6 ++ docs/ru/sql_reference/data_types/index.md | 6 ++ docs/ru/sql_reference/index.md | 7 ++ docs/ru/sql_reference/statements/alter.md | 5 ++ docs/ru/sql_reference/statements/create.md | 5 ++ docs/ru/sql_reference/statements/index.md | 2 +- .../sql_reference/statements/insert_into.md | 5 ++ docs/ru/sql_reference/statements/select.md | 5 ++ .../ru/sql_reference/table_functions/index.md | 6 ++ docs/ru/{ => whats_new}/extended_roadmap.md | 0 docs/ru/whats_new/index.md | 2 +- .../tr/interfaces/third-party/integrations.md | 3 + .../zh/interfaces/third-party/integrations.md | 61 ++++++++-------- 33 files changed, 212 insertions(+), 70 deletions(-) create mode 100644 docs/en/commercial/support.md rename docs/ru/{ => whats_new}/extended_roadmap.md (100%) diff --git a/docs/en/commercial/cloud.md b/docs/en/commercial/cloud.md index 3f7ead86219..5255d254100 100644 --- a/docs/en/commercial/cloud.md +++ b/docs/en/commercial/cloud.md @@ -1,3 +1,8 @@ +--- +toc_title: Cloud +toc_priority: 1 +--- + # ClickHouse Cloud Service Providers {#clickhouse-cloud-service-providers} !!! info "Info" diff --git a/docs/en/commercial/support.md b/docs/en/commercial/support.md new file mode 100644 index 00000000000..e7b4503a534 --- /dev/null +++ b/docs/en/commercial/support.md @@ -0,0 +1,21 @@ +--- +toc_title: Support +toc_priority: 3 +--- + +# ClickHouse Commercial Support Service Providers {#clickhouse-commercial-support-service-providers} + +!!! info "Info" + If you have launched a ClickHouse commercial support service, feel free to [open a pull-request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md) adding it to the following list. + +## Altinity {#altinity} + +[Service description](https://www.altinity.com/24x7-support) + +## Mafiree {#mafiree} + +[Service description](http://mafiree.com/clickhouse-analytics-services.php) + +## MinervaDB {#minervadb} + +[Service description](https://minervadb.com/index.php/clickhouse-consulting-and-support-by-minervadb/) diff --git a/docs/en/development/index.md b/docs/en/development/index.md index bb4158554d3..34329853509 100644 --- a/docs/en/development/index.md +++ b/docs/en/development/index.md @@ -1,5 +1,5 @@ --- -toc_folder_title: Development +toc_folder_title: Разработка toc_hidden: true toc_priority: 58 toc_title: hidden diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index ff0b7fe2e19..9f4b0f2fa65 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -24,7 +24,10 @@ toc_title: Integrations - [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator) - Message queues - [Kafka](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/kshvakov/clickhouse/)) + - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/)) +- Stream processing + - [Flink](https://flink.apache.org) + - [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink) - Object storages - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup) @@ -72,6 +75,9 @@ toc_title: Integrations - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pandas](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) +- PHP + - [Doctrine](https://www.doctrine-project.org/) + - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) - R - [dplyr](https://db.rstudio.com/dplyr/) - [RClickHouse](https://github.com/IMSMWU/RClickHouse) (uses [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) diff --git a/docs/es/interfaces/third-party/integrations.md b/docs/es/interfaces/third-party/integrations.md index 00be86b187d..d706278a280 100644 --- a/docs/es/interfaces/third-party/integrations.md +++ b/docs/es/interfaces/third-party/integrations.md @@ -26,7 +26,10 @@ toc_title: "Integraci\xF3n" - [Método de codificación de datos:](https://github.com/zlzforever/ClickHouseMigrator) - Colas de mensajes - [Kafka](https://kafka.apache.org) - - [Método de codificación de datos:](https://github.com/housepower/clickhouse_sinker) (utilizar [Ir cliente](https://github.com/kshvakov/clickhouse/)) + - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (usos [Go client](https://github.com/ClickHouse/clickhouse-go/)) +- Procesamiento de flujo + - [Flink](https://flink.apache.org) + - [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink) - Almacenamiento de objetos - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - [Haga clic en el botón de copia de seguridad](https://github.com/AlexAkulov/clickhouse-backup) @@ -74,6 +77,9 @@ toc_title: "Integraci\xF3n" - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (utilizar [InformaciónSistema abierto.](https://github.com/Infinidat/infi.clickhouse_orm)) - [pandas](https://pandas.pydata.org) - [Pandahouse](https://github.com/kszucs/pandahouse) +- PHP + - [Doctrine](https://www.doctrine-project.org/) + - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) - R - [Dplyr](https://db.rstudio.com/dplyr/) - [Bienvenidos al Portal de Licitación Electrónica de Licitación Electrónica](https://github.com/IMSMWU/RClickhouse) (utilizar [Bienvenidos](https://github.com/artpaul/clickhouse-cpp)) diff --git a/docs/fa/interfaces/third-party/integrations.md b/docs/fa/interfaces/third-party/integrations.md index ddbdffad169..0ad13d1fd9e 100644 --- a/docs/fa/interfaces/third-party/integrations.md +++ b/docs/fa/interfaces/third-party/integrations.md @@ -74,6 +74,9 @@ toc_title: "\u06CC\u06A9\u067E\u0627\u0631\u0686\u06AF\u06CC" - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (استفاده [اطالعات.کلیک \_شورم](https://github.com/Infinidat/infi.clickhouse_orm)) - [پانداها](https://pandas.pydata.org) - [پانداهاوس](https://github.com/kszucs/pandahouse) +- PHP + - [Doctrine](https://www.doctrine-project.org/) + - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) - R - [هواپیمای دوباله](https://db.rstudio.com/dplyr/) - [خانه روستایی](https://github.com/IMSMWU/RClickhouse) (استفاده [صفحه اصلی](https://github.com/artpaul/clickhouse-cpp)) diff --git a/docs/fr/interfaces/third-party/integrations.md b/docs/fr/interfaces/third-party/integrations.md index 76648d8613d..565be2b9604 100644 --- a/docs/fr/interfaces/third-party/integrations.md +++ b/docs/fr/interfaces/third-party/integrations.md @@ -74,6 +74,9 @@ toc_title: "Int\xE9gration" - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (utiliser [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [Panda](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) +- PHP + - [Doctrine](https://www.doctrine-project.org/) + - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) - R - [dplyr](https://db.rstudio.com/dplyr/) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) (utiliser [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) diff --git a/docs/ja/interfaces/third-party/integrations.md b/docs/ja/interfaces/third-party/integrations.md index 7f5bfb93325..74d0a51e7de 100644 --- a/docs/ja/interfaces/third-party/integrations.md +++ b/docs/ja/interfaces/third-party/integrations.md @@ -74,6 +74,9 @@ toc_title: "\u7D71\u5408" - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (用途 [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [パンダ](https://pandas.pydata.org) - [パンダハウス](https://github.com/kszucs/pandahouse) +- PHP + - [Doctrine](https://www.doctrine-project.org/) + - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) - R - [dplyr](https://db.rstudio.com/dplyr/) - [Rクリックハウス](https://github.com/IMSMWU/RClickhouse) (用途 [クリックハウス-cpp](https://github.com/artpaul/clickhouse-cpp)) diff --git a/docs/redirects.txt b/docs/redirects.txt index 95df6773359..1a9455355db 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -43,6 +43,7 @@ dicts/external_dicts_dict_sources.md query_language/dicts/external_dicts_dict_so dicts/external_dicts_dict_structure.md query_language/dicts/external_dicts_dict_structure.md dicts/index.md query_language/dicts/index.md dicts/internal_dicts.md query_language/dicts/internal_dicts.md +extended_roadmap.md whats_new/extended_roadmap.md formats.md interfaces/formats.md formats/capnproto.md interfaces/formats.md formats/csv.md interfaces/formats.md diff --git a/docs/ru/commercial/index.md b/docs/ru/commercial/index.md index f9065c7cd50..dc78f6f4899 100644 --- a/docs/ru/commercial/index.md +++ b/docs/ru/commercial/index.md @@ -1,7 +1,7 @@ --- -toc_folder_title: Commercial +toc_folder_title: Коммерческие услуги toc_priority: 70 -toc_title: Commercial +toc_title: Коммерческие услуги --- diff --git a/docs/ru/engines/index.md b/docs/ru/engines/index.md index c4b0b299858..a3d85155cbb 100644 --- a/docs/ru/engines/index.md +++ b/docs/ru/engines/index.md @@ -1,5 +1,5 @@ --- -toc_folder_title: Engines +toc_folder_title: Движки toc_priority: 25 --- diff --git a/docs/ru/getting_started/example_datasets/index.md b/docs/ru/getting_started/example_datasets/index.md index c01f73ddc4d..5fbe0843f4b 100644 --- a/docs/ru/getting_started/example_datasets/index.md +++ b/docs/ru/getting_started/example_datasets/index.md @@ -1,3 +1,8 @@ +--- +toc_folder_title: Тестовые массивы данных +toc_priority: 12 +toc_title: Обзор +--- # Тестовые массивы данных Этот раздел описывает как получить тестовые массивы данных и загрузить их в ClickHouse. diff --git a/docs/ru/getting_started/index.md b/docs/ru/getting_started/index.md index aa126f7b697..7711d6b94ee 100644 --- a/docs/ru/getting_started/index.md +++ b/docs/ru/getting_started/index.md @@ -1,3 +1,10 @@ +--- +toc_folder_title: Начало работы +toc_hidden: true +toc_priority: 8 +toc_title: hidden +--- + # Начало работы {#nachalo-raboty} Если вы новичок в ClickHouse и хотите вживую оценить его производительность, прежде всего нужно пройти через [процесс установки](install.md). diff --git a/docs/ru/guides/index.md b/docs/ru/guides/index.md index 1ece4b22b85..2ba84e9afc2 100644 --- a/docs/ru/guides/index.md +++ b/docs/ru/guides/index.md @@ -1,3 +1,9 @@ +--- +toc_folder_title: Руководства +toc_priority: 38 +toc_title: Обзор +--- + # Руководства {#rukovodstva} Подробные пошаговые инструкции, которые помогут вам решать различные задачи с помощью ClickHouse. diff --git a/docs/ru/index.md b/docs/ru/index.md index c49c33adf6f..4636c2cb5df 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -1,3 +1,8 @@ +--- +toc_priority: 0 +toc_title: Обзор +--- + # Что такое ClickHouse {#chto-takoe-clickhouse} ClickHouse - столбцовая система управления базами данных (СУБД) для онлайн обработки аналитических запросов (OLAP). diff --git a/docs/ru/interfaces/index.md b/docs/ru/interfaces/index.md index 8f5a2e5f073..be49e640132 100644 --- a/docs/ru/interfaces/index.md +++ b/docs/ru/interfaces/index.md @@ -1,3 +1,9 @@ +--- +toc_folder_title: Интерфейсы +toc_priority: 14 +toc_title: Введение +--- + # Интерфейсы {#interfaces} ClickHouse предоставляет два сетевых интерфейса (оба могут быть дополнительно обернуты в TLS для дополнительной безопасности): diff --git a/docs/ru/interfaces/third-party/index.md b/docs/ru/interfaces/third-party/index.md index 16d315d059c..8e768ad9d8e 100644 --- a/docs/ru/interfaces/third-party/index.md +++ b/docs/ru/interfaces/third-party/index.md @@ -1,5 +1,5 @@ --- -toc_folder_title: Third-Party +toc_folder_title: От сторонних разработчиков toc_priority: 24 --- diff --git a/docs/ru/interfaces/third-party/integrations.md b/docs/ru/interfaces/third-party/integrations.md index c05ff4d062f..d16404e2f27 100644 --- a/docs/ru/interfaces/third-party/integrations.md +++ b/docs/ru/interfaces/third-party/integrations.md @@ -7,66 +7,72 @@ - Реляционные системы управления базами данных - [MySQL](https://www.mysql.com) - - [ProxySQL](https://github.com/sysown/proxysql/wiki/ClickHouse-Support) - - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) - - [horgh-replicator](https://github.com/larsnovikov/horgh-replicator) + - [ProxySQL](https://github.com/sysown/proxysql/wiki/ClickHouse-Support) + - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) + - [horgh-replicator](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (использует [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - - [pg2ch](https://github.com/mkabilov/pg2ch) - - [clickhouse\_fdw](https://github.com/adjust/clickhouse_fdw) + - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (использует [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [pg2ch](https://github.com/mkabilov/pg2ch) + - [clickhouse\_fdw](https://github.com/adjust/clickhouse_fdw) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - - [ClickHouseMightrator](https://github.com/zlzforever/ClickHouseMigrator) + - [ClickHouseMightrator](https://github.com/zlzforever/ClickHouseMigrator) - Очереди сообщений - [Kafka](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (использует [Go client](https://github.com/kshvakov/clickhouse/)) + - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (использует [Go client](https://github.com/ClickHouse/clickhouse-go/)) +- Потоковая обработка + - [Flink](https://flink.apache.org) + - [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink) - Хранилища объектов - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - - [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup) + - [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup) - Оркестрация контейнеров - [Kubernetes](https://kubernetes.io) - - [clickhouse-operator](https://github.com/Altinity/clickhouse-operator) + - [clickhouse-operator](https://github.com/Altinity/clickhouse-operator) - Системы управления конфигурацией - [puppet](https://puppet.com) - - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) - - [mfedotov/clickhouse](https://forge.puppet.com/mfedotov/clickhouse) + - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) + - [mfedotov/clickhouse](https://forge.puppet.com/mfedotov/clickhouse) - Мониторинг - [Graphite](https://graphiteapp.org) - - [graphouse](https://github.com/yandex/graphouse) - - [carbon-clickhouse](https://github.com/lomik/carbon-clickhouse) + - - [graphite-clickhouse](https://github.com/lomik/graphite-clickhouse) - - [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer) - оптимизирует партиции таблиц [\*GraphiteMergeTree](../../engines/table_engines/mergetree_family/graphitemergetree.md#graphitemergetree) согласно правилам в [конфигурации rollup](../../engines/table_engines/mergetree_family/graphitemergetree.md#rollup-configuration) + - [graphouse](https://github.com/yandex/graphouse) + - [carbon-clickhouse](https://github.com/lomik/carbon-clickhouse) + + - [graphite-clickhouse](https://github.com/lomik/graphite-clickhouse) + - [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer) - оптимизирует партиции таблиц [\*GraphiteMergeTree](../../engines/table_engines/mergetree_family/graphitemergetree.md#graphitemergetree) согласно правилам в [конфигурации rollup](../../engines/table_engines/mergetree_family/graphitemergetree.md#rollup-configuration) - [Grafana](https://grafana.com/) - - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) + - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - [Prometheus](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) - - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (использует [Go client](https://github.com/kshvakov/clickhouse/)) + - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [PromHouse](https://github.com/Percona-Lab/PromHouse) + - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (использует [Go client](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - - [clickhouse-zabbix-template](https://github.com/Altinity/clickhouse-zabbix-template) + - [clickhouse-zabbix-template](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) - - [clickhouse интеграция](https://github.com/sematext/sematext-agent-integrations/tree/master/clickhouse) + - [clickhouse интеграция](https://github.com/sematext/sematext-agent-integrations/tree/master/clickhouse) - Логирование - [rsyslog](https://www.rsyslog.com/) - - [omclickhouse](https://www.rsyslog.com/doc/master/configuration/modules/omclickhouse.html) + - [omclickhouse](https://www.rsyslog.com/doc/master/configuration/modules/omclickhouse.html) - [fluentd](https://www.fluentd.org) - - [loghouse](https://github.com/flant/loghouse) (для [Kubernetes](https://kubernetes.io)) - - [logagent](https://www.sematext.com/logagent) - - [logagent output-plugin-clickhouse](https://sematext.com/docs/logagent/output-plugin-clickhouse/) + - [loghouse](https://github.com/flant/loghouse) (для [Kubernetes](https://kubernetes.io)) + - [Sematext](https://www.sematext.com/logagent) + - [logagent output-plugin-clickhouse](https://sematext.com/docs/logagent/output-plugin-clickhouse/) - Гео - [MaxMind](https://dev.maxmind.com/geoip/) - - [clickhouse-maxmind-geoip](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip) + - [clickhouse-maxmind-geoip](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip) ## Экосистемы вокруг языков программирования {#ekosistemy-vokrug-iazykov-programmirovaniia} - Python - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (использует [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (использует [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [pandas](https://pandas.pydata.org) - - [pandahouse](https://github.com/kszucs/pandahouse) + - [pandahouse](https://github.com/kszucs/pandahouse) +- PHP + - [Doctrine](https://www.doctrine-project.org/) + - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) - R - [dplyr](https://db.rstudio.com/dplyr/) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) (использует [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) diff --git a/docs/ru/introduction/index.md b/docs/ru/introduction/index.md index ba80f9c2640..5479b37428f 100644 --- a/docs/ru/introduction/index.md +++ b/docs/ru/introduction/index.md @@ -1,5 +1,5 @@ --- -toc_folder_title: Introduction +toc_folder_title: Введение toc_priority: 1 --- diff --git a/docs/ru/operations/index.md b/docs/ru/operations/index.md index 3df5dbb1f7e..babb22cd8b4 100644 --- a/docs/ru/operations/index.md +++ b/docs/ru/operations/index.md @@ -1,3 +1,9 @@ +--- +toc_folder_title: Эксплуатация +toc_priority: 41 +toc_title: Введение +--- + # Эксплуатация {#ekspluatatsiia} Руководство по эксплуатации ClickHouse состоит из следующих основных разделов: diff --git a/docs/ru/sql_reference/aggregate_functions/index.md b/docs/ru/sql_reference/aggregate_functions/index.md index e0c07a1c95e..d28e105f6fc 100644 --- a/docs/ru/sql_reference/aggregate_functions/index.md +++ b/docs/ru/sql_reference/aggregate_functions/index.md @@ -1,3 +1,9 @@ +--- +toc_folder_title: Агрегатные функции +toc_priority: 33 +toc_title: Введение +--- + # Агрегатные функции {#aggregate-functions} Агрегатные функции работают в [привычном](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) для специалистов по базам данных смысле. diff --git a/docs/ru/sql_reference/data_types/index.md b/docs/ru/sql_reference/data_types/index.md index a362b4708e6..1fd52b9bf9e 100644 --- a/docs/ru/sql_reference/data_types/index.md +++ b/docs/ru/sql_reference/data_types/index.md @@ -1,3 +1,9 @@ +--- +toc_folder_title: Типы данных +toc_priority: 37 +toc_title: Введение +--- + # Типы данных {#data_types} ClickHouse может сохранять в ячейках таблиц данные различных типов. diff --git a/docs/ru/sql_reference/index.md b/docs/ru/sql_reference/index.md index ca35c14a547..ca21b0df3ed 100644 --- a/docs/ru/sql_reference/index.md +++ b/docs/ru/sql_reference/index.md @@ -1,3 +1,10 @@ +--- +toc_folder_title: Справка по SQL +toc_hidden: true +toc_priority: 28 +toc_title: hidden +--- + # Справка по SQL {#spravka-po-sql} - [SELECT](statements/select.md) diff --git a/docs/ru/sql_reference/statements/alter.md b/docs/ru/sql_reference/statements/alter.md index bc06fe074d2..5c08d68623e 100644 --- a/docs/ru/sql_reference/statements/alter.md +++ b/docs/ru/sql_reference/statements/alter.md @@ -1,3 +1,8 @@ +--- +toc_priority: 36 +toc_title: ALTER +--- + ## ALTER {#query_language_queries_alter} Запрос `ALTER` поддерживается только для таблиц типа `*MergeTree`, а также `Merge` и `Distributed`. Запрос имеет несколько вариантов. diff --git a/docs/ru/sql_reference/statements/create.md b/docs/ru/sql_reference/statements/create.md index 7a5e3c9dc08..e09877d3495 100644 --- a/docs/ru/sql_reference/statements/create.md +++ b/docs/ru/sql_reference/statements/create.md @@ -1,3 +1,8 @@ +--- +toc_priority: 35 +toc_title: CREATE +--- + ## CREATE DATABASE {#query-language-create-database} Создает базу данных. diff --git a/docs/ru/sql_reference/statements/index.md b/docs/ru/sql_reference/statements/index.md index 1adf93a153e..5b461b73c92 100644 --- a/docs/ru/sql_reference/statements/index.md +++ b/docs/ru/sql_reference/statements/index.md @@ -1,5 +1,5 @@ --- -toc_folder_title: Statements +toc_folder_title: Выражения toc_priority: 31 --- diff --git a/docs/ru/sql_reference/statements/insert_into.md b/docs/ru/sql_reference/statements/insert_into.md index ed07d6d3b1c..e9c80466a58 100644 --- a/docs/ru/sql_reference/statements/insert_into.md +++ b/docs/ru/sql_reference/statements/insert_into.md @@ -1,3 +1,8 @@ +--- +toc_priority: 34 +toc_title: INSERT INTO +--- + ## INSERT {#insert} Добавление данных. diff --git a/docs/ru/sql_reference/statements/select.md b/docs/ru/sql_reference/statements/select.md index bd9dc21e2aa..2ceeb3489c2 100644 --- a/docs/ru/sql_reference/statements/select.md +++ b/docs/ru/sql_reference/statements/select.md @@ -1,3 +1,8 @@ +--- +toc_priority: 33 +toc_title: SELECT +--- + # Синтаксис запросов SELECT {#sintaksis-zaprosov-select} `SELECT` осуществляет выборку данных. diff --git a/docs/ru/sql_reference/table_functions/index.md b/docs/ru/sql_reference/table_functions/index.md index f0e465c5cb0..70c701d1713 100644 --- a/docs/ru/sql_reference/table_functions/index.md +++ b/docs/ru/sql_reference/table_functions/index.md @@ -1,3 +1,9 @@ +--- +toc_folder_title: Табличные функции +toc_priority: 34 +toc_title: Введение +--- + # Табличные функции {#tablichnye-funktsii} Табличные функции — это метод создания таблиц. diff --git a/docs/ru/extended_roadmap.md b/docs/ru/whats_new/extended_roadmap.md similarity index 100% rename from docs/ru/extended_roadmap.md rename to docs/ru/whats_new/extended_roadmap.md diff --git a/docs/ru/whats_new/index.md b/docs/ru/whats_new/index.md index 0901166b887..b97b3623a9e 100644 --- a/docs/ru/whats_new/index.md +++ b/docs/ru/whats_new/index.md @@ -1,5 +1,5 @@ --- -toc_folder_title: What's New +toc_folder_title: Что нового? toc_priority: 72 --- diff --git a/docs/tr/interfaces/third-party/integrations.md b/docs/tr/interfaces/third-party/integrations.md index ecb6662d0e8..2216e68a4c4 100644 --- a/docs/tr/interfaces/third-party/integrations.md +++ b/docs/tr/interfaces/third-party/integrations.md @@ -74,6 +74,9 @@ toc_title: Entegrasyonlar - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (kullanma [ınfi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [Pandalar](https://pandas.pydata.org) - [pandahouse](https://github.com/kszucs/pandahouse) +- PHP + - [Doctrine](https://www.doctrine-project.org/) + - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) - R - [dplyr](https://db.rstudio.com/dplyr/) - [RClickHouse](https://github.com/IMSMWU/RClickHouse) (kullanma [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) diff --git a/docs/zh/interfaces/third-party/integrations.md b/docs/zh/interfaces/third-party/integrations.md index aac3d7a1b11..128a4060c2d 100644 --- a/docs/zh/interfaces/third-party/integrations.md +++ b/docs/zh/interfaces/third-party/integrations.md @@ -1,4 +1,3 @@ - # 第三方集成库 {#di-san-fang-ji-cheng-ku} !!! warning "声明" @@ -8,18 +7,21 @@ - 关系数据库管理系统 - [MySQL](https://www.mysql.com) - - [ProxySQL](https://github.com/sysown/proxysql/wiki/ClickHouse-Support) - - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) - - [horgh-复制器](https://github.com/larsnovikov/horgh-replicator) + - [ProxySQL](https://github.com/sysown/proxysql/wiki/ClickHouse-Support) + - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) + - [horgh-复制器](https://github.com/larsnovikov/horgh-replicator) - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (使用 [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - - [pg2ch](https://github.com/mkabilov/pg2ch) + - [clickhousedb\_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) + - [infi.clickhouse\_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (使用 [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [pg2ch](https://github.com/mkabilov/pg2ch) - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - - [ClickHouseMightrator](https://github.com/zlzforever/ClickHouseMigrator) + - [ClickHouseMightrator](https://github.com/zlzforever/ClickHouseMigrator) - 消息队列 - [卡夫卡](https://kafka.apache.org) - - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (使用 [去客户](https://github.com/kshvakov/clickhouse/)) + - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (使用 [去客户](https://github.com/ClickHouse/clickhouse-go/)) +- 流处理 + - [Flink](https://flink.apache.org) + - [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink) - 对象存储 - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - [ツ环板backupョツ嘉ッツ偲](https://github.com/AlexAkulov/clickhouse-backup) @@ -32,41 +34,44 @@ - [mfedotov/clickhouse](https://forge.puppet.com/mfedotov/clickhouse) - 监控 - [石墨](https://graphiteapp.org) - - [graphouse](https://github.com/yandex/graphouse) - - [ツ暗ェツ氾环催ツ団](https://github.com/lomik/carbon-clickhouse) + - - [ツ环板-ョツ嘉ッツ偲](https://github.com/lomik/graphite-clickhouse) - - [石墨-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer) -优化静态分区 [\*GraphiteMergeTree](../../engines/table_engines/mergetree_family/graphitemergetree.md#graphitemergetree) 如果从规则 [汇总配置](../../engines/table_engines/mergetree_family/graphitemergetree.md#rollup-configuration) 可以应用 + - [graphouse](https://github.com/yandex/graphouse) + - [ツ暗ェツ氾环催ツ団](https://github.com/lomik/carbon-clickhouse) + + - [ツ环板-ョツ嘉ッツ偲](https://github.com/lomik/graphite-clickhouse) + - [石墨-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer) -优化静态分区 [\*GraphiteMergeTree](../../engines/table_engines/mergetree_family/graphitemergetree.md#graphitemergetree) 如果从规则 [汇总配置](../../engines/table_engines/mergetree_family/graphitemergetree.md#rollup-configuration) 可以应用 - [Grafana](https://grafana.com/) - - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) + - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - [普罗米修斯号](https://prometheus.io/) - - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) - - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (用途 [去客户](https://github.com/kshvakov/clickhouse/)) + - [clickhouse\_exporter](https://github.com/f1yegor/clickhouse_exporter) + - [PromHouse](https://github.com/Percona-Lab/PromHouse) + - [clickhouse\_exporter](https://github.com/hot-wifi/clickhouse_exporter) (用途 [去客户](https://github.com/kshvakov/clickhouse/)) - [Nagios](https://www.nagios.org/) - - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) + - [check\_clickhouse](https://github.com/exogroup/check_clickhouse/) + - [check\_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - [Zabbix](https://www.zabbix.com) - - [ツ暗ェツ氾环催ツ団ツ法ツ人](https://github.com/Altinity/clickhouse-zabbix-template) + - [ツ暗ェツ氾环催ツ団ツ法ツ人](https://github.com/Altinity/clickhouse-zabbix-template) - [Sematext](https://sematext.com/) - - [clickhouse积分](https://github.com/sematext/sematext-agent-integrations/tree/master/clickhouse) + - [clickhouse积分](https://github.com/sematext/sematext-agent-integrations/tree/master/clickhouse) - 记录 - [rsyslog](https://www.rsyslog.com/) - - [鹿茫house om omhousehousehousehouse酶](https://www.rsyslog.com/doc/master/configuration/modules/omclickhouse.html) + - [鹿茫house omhousee酶](https://www.rsyslog.com/doc/master/configuration/modules/omclickhouse.html) - [fluentd](https://www.fluentd.org) - - [loghouse](https://github.com/flant/loghouse) (对于 [Kubernetes](https://kubernetes.io)) - - [logagent](https://www.sematext.com/logagent) - - [logagent输出-插件-clickhouse](https://sematext.com/docs/logagent/output-plugin-clickhouse/) + - [loghouse](https://github.com/flant/loghouse) (对于 [Kubernetes](https://kubernetes.io)) + - [Sematext](https://www.sematext.com/logagent) + - [logagent输出-插件-clickhouse](https://sematext.com/docs/logagent/output-plugin-clickhouse/) - 地理 - [MaxMind](https://dev.maxmind.com/geoip/) - - [ツ环板-ョツ嘉ッツ偲青clickシツ氾カツ鉄ツ工ツ渉](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip) + - [ツ环板-ョツ嘉ッツ偲青clickシツ氾カツ鉄ツ工ツ渉](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip) ## 编程语言生态系统 {#bian-cheng-yu-yan-sheng-tai-xi-tong} - Python - [SQLAlchemy](https://www.sqlalchemy.org) - - [ツ暗ェツ氾环催ツ団ツ法ツ人](https://github.com/cloudflare/sqlalchemy-clickhouse) (使用 [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) + - [ツ暗ェツ氾环催ツ団ツ法ツ人](https://github.com/cloudflare/sqlalchemy-clickhouse) (使用 [infi.clickhouse\_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - [熊猫](https://pandas.pydata.org) - - [pandahouse](https://github.com/kszucs/pandahouse) + - [pandahouse](https://github.com/kszucs/pandahouse) +- PHP + - [Doctrine](https://www.doctrine-project.org/) + - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) - R - [dplyr](https://db.rstudio.com/dplyr/) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) (使用 [ツ暗ェツ氾环催ツ団](https://github.com/artpaul/clickhouse-cpp)) From dc247ed01141d5b7fac05811d5e95c71fd298347 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Apr 2020 16:16:50 +0300 Subject: [PATCH 289/743] Add special key to dupload which allows to push .deb package to several branches in repo --- utils/release/push_packages | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/release/push_packages b/utils/release/push_packages index 4a0548bb4ca..68d72bb39fe 100755 --- a/utils/release/push_packages +++ b/utils/release/push_packages @@ -152,7 +152,7 @@ def transfer_packages_dupload(ssh_key, path, repo_user, repo_url, incoming_direc } with DebRelease(config, repo_user, ssh_key): logging.info("Duploading") - subprocess.check_call("dupload --nomail --to {repo} {path}".format(repo=repo_short_name, path=path), shell=True) + subprocess.check_call("dupload -f --nomail --to {repo} {path}".format(repo=repo_short_name, path=path), shell=True) logging.info("Dupload finished") From d6db27e3727ffb51e6302db0791dec0e98fd37ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 16 Apr 2020 17:37:38 +0300 Subject: [PATCH 290/743] Modify context. --- programs/server/Server.cpp | 3 ++- src/Interpreters/Context.cpp | 15 ++++++++++++++- src/Interpreters/Context.h | 5 ++++- 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5322514e7c2..0173203d263 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -227,7 +227,8 @@ int Server::main(const std::vector & /*args*/) /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases... */ - global_context = std::make_unique(Context::createGlobal()); + auto shared_context = Context::createShared(); + global_context = std::make_unique(Context::createGlobal(shared_context.get())); global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::SERVER); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6e30792277f..6c2fd349155 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -445,10 +445,23 @@ Context & Context::operator=(const Context &) = default; Context Context::createGlobal() { Context res; - res.shared = std::make_shared(); + res.shared_holder = std::make_shared(); + res.shared = res.shared_holder.get(); return res; } +Context Context::createGlobal(ContextShared * shared) +{ + Context res; + res.shared = shared; + return res; +} + +std::unique_ptr Context::createShared() +{ + return std::make_unique(); +} + Context::~Context() = default; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f68b08bf9f0..9eea948ba95 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -138,7 +138,8 @@ class Context { private: using Shared = std::shared_ptr; - Shared shared; + Shared shared_holder; + ContextShared * shared; ClientInfo client_info; ExternalTablesInitializer external_tables_initializer_callback; @@ -191,6 +192,8 @@ private: public: /// Create initial Context with ContextShared and etc. static Context createGlobal(); + static Context createGlobal(ContextShared * shared); + static std::unique_ptr createShared(); Context(const Context &); Context & operator=(const Context &); From 025093d35436f98727bf1157e76f6a868be0f740 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 16 Apr 2020 17:51:33 +0300 Subject: [PATCH 291/743] Fix build. --- programs/server/Server.cpp | 2 +- src/Interpreters/Context.cpp | 9 +++++++-- src/Interpreters/Context.h | 10 +++++++++- 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 0173203d263..20d7fc49fba 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -228,7 +228,7 @@ int Server::main(const std::vector & /*args*/) * settings, available functions, data types, aggregate functions, databases... */ auto shared_context = Context::createShared(); - global_context = std::make_unique(Context::createGlobal(shared_context.get())); + global_context = std::make_unique(Context::createGlobal(shared_context.shared.get())); global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::SERVER); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6c2fd349155..c227ebd2f4c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -441,6 +441,9 @@ Context::Context() = default; Context::Context(const Context &) = default; Context & Context::operator=(const Context &) = default; +SharedContextHolder::SharedContextHolder() = default; +SharedContextHolder::SharedContextHolder(SharedContextHolder &&) = default; +SharedContextHolder::~SharedContextHolder() = default; Context Context::createGlobal() { @@ -457,9 +460,11 @@ Context Context::createGlobal(ContextShared * shared) return res; } -std::unique_ptr Context::createShared() +SharedContextHolder Context::createShared() { - return std::make_unique(); + SharedContextHolder holder; + holder.shared = std::make_unique(); + return holder; } Context::~Context() = default; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 9eea948ba95..91b2ee3242b 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -128,6 +128,14 @@ struct IHostContext using IHostContextPtr = std::shared_ptr; +struct SharedContextHolder +{ + std::unique_ptr shared; + ~SharedContextHolder(); + SharedContextHolder(); + SharedContextHolder(SharedContextHolder &&); +}; + /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -193,7 +201,7 @@ public: /// Create initial Context with ContextShared and etc. static Context createGlobal(); static Context createGlobal(ContextShared * shared); - static std::unique_ptr createShared(); + static SharedContextHolder createShared(); Context(const Context &); Context & operator=(const Context &); From 605f3b211999e8bc4dddb764782e0d3d5ee4e496 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Apr 2020 18:30:18 +0300 Subject: [PATCH 292/743] fix DROP vs OPTIMIZE race in ReplicatedMergeTree --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 3 +++ src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 3 +++ src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++++++-- ...00993_system_parts_race_condition_drop_zookeeper.sh | 8 ++++---- 4 files changed, 18 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index bebb6ee79c3..72044ab832c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int UNEXPECTED_NODE_IN_ZOOKEEPER; extern const int UNFINISHED; + extern const int ABORTED; } @@ -426,6 +427,8 @@ bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo & void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback) { std::lock_guard lock(pull_logs_to_queue_mutex); + if (pull_log_blocker.isCancelled()) + throw Exception("Log pulling is cancelled", ErrorCodes::ABORTED); String index_str = zookeeper->get(replica_path + "/log_pointer"); UInt64 index; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index fcb3dfb4b86..2191104a291 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -356,6 +356,9 @@ public: /// A blocker that stops selects from the queue ActionBlocker actions_blocker; + /// A blocker that stops pulling entries from replication log to queue + ActionBlocker pull_log_blocker; + /// Adds a subscriber SubscriberHandler addSubscriber(SubscriberCallBack && callback); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a19a424c643..3f907541a3c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -300,8 +300,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } createNewZooKeeperNodes(); - - } @@ -2905,6 +2903,7 @@ void StorageReplicatedMergeTree::shutdown() fetcher.blocker.cancelForever(); merger_mutator.merges_blocker.cancelForever(); parts_mover.moves_blocker.cancelForever(); + queue.pull_log_blocker.cancelForever(); restarting_thread.shutdown(); @@ -3641,7 +3640,11 @@ void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &) LOG_INFO(log, "Removing replica " << replica_path); replica_is_active_node = nullptr; + /// It may left some garbage if replica_path subtree are concurently modified zookeeper->tryRemoveRecursive(replica_path); + if (zookeeper->exists(replica_path)) + LOG_ERROR(log, "Replica was not completely removed from ZooKeeper, " + << replica_path << " still exists and may contain some garbage."); /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line. Strings replicas; @@ -3649,6 +3652,9 @@ void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &) { LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)"); zookeeper->tryRemoveRecursive(zookeeper_path); + if (zookeeper->exists(zookeeper_path)) + LOG_ERROR(log, "Table was not completely removed from ZooKeeper, " + << zookeeper_path << " still exists and may contain some garbage."); } } diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index 2d458ba7e28..01127a3ef7b 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -72,28 +72,28 @@ timeout $TIMEOUT bash -c thread2 2> /dev/null & timeout $TIMEOUT bash -c thread3 2> /dev/null & timeout $TIMEOUT bash -c thread4 2> /dev/null & timeout $TIMEOUT bash -c thread5 2> /dev/null & -timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from ZooKeeper" & timeout $TIMEOUT bash -c thread1 2> /dev/null & timeout $TIMEOUT bash -c thread2 2> /dev/null & timeout $TIMEOUT bash -c thread3 2> /dev/null & timeout $TIMEOUT bash -c thread4 2> /dev/null & timeout $TIMEOUT bash -c thread5 2> /dev/null & -timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from ZooKeeper" & timeout $TIMEOUT bash -c thread1 2> /dev/null & timeout $TIMEOUT bash -c thread2 2> /dev/null & timeout $TIMEOUT bash -c thread3 2> /dev/null & timeout $TIMEOUT bash -c thread4 2> /dev/null & timeout $TIMEOUT bash -c thread5 2> /dev/null & -timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from ZooKeeper" & timeout $TIMEOUT bash -c thread1 2> /dev/null & timeout $TIMEOUT bash -c thread2 2> /dev/null & timeout $TIMEOUT bash -c thread3 2> /dev/null & timeout $TIMEOUT bash -c thread4 2> /dev/null & timeout $TIMEOUT bash -c thread5 2> /dev/null & -timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from ZooKeeper" & wait From 3de1a36b33e91327b67bf9e13c5629d0fdaa4822 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 18:55:39 +0300 Subject: [PATCH 293/743] Update ya.make --- src/Common/ya.make | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ya.make b/src/Common/ya.make index 53155ac471f..eece911d2b8 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -22,9 +22,9 @@ CFLAGS (GLOBAL -DDBMS_NAME=\"ClickHouse\") CFLAGS (GLOBAL -DDBMS_VERSION_MAJOR=0) CFLAGS (GLOBAL -DDBMS_VERSION_MINOR=0) CFLAGS (GLOBAL -DDBMS_VERSION_PATCH=0) -CFLAGS (GLOBAL -DVERSION_FULL=\"Clickhouse\") +CFLAGS (GLOBAL -DVERSION_FULL=\"ClickHouse\") CFLAGS (GLOBAL -DVERSION_INTEGER=0) -CFLAGS (GLOBAL -DVERSION_NAME=\"Clickhouse\") +CFLAGS (GLOBAL -DVERSION_NAME=\"ClickHouse\") CFLAGS (GLOBAL -DVERSION_OFFICIAL=\"\\\(arcadia\\\)\") CFLAGS (GLOBAL -DVERSION_REVISION=0) CFLAGS (GLOBAL -DVERSION_STRING=\"Unknown\") From 62442708096fed060b5e1ada0467addd458bc7f0 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Mon, 6 Apr 2020 11:25:25 +0000 Subject: [PATCH 294/743] Add SimpleAggregateFunction support for sumMap --- .../aggregate_functions/reference.md | 26 +++++++++------- .../AggregateFunctionSumMap.cpp | 30 ++++++++++++++----- .../AggregateFunctionSumMap.h | 12 ++++++++ .../DataTypeCustomSimpleAggregateFunction.cpp | 2 +- .../0_stateless/00502_sum_map.reference | 1 + tests/queries/0_stateless/00502_sum_map.sql | 1 + .../00915_simple_aggregate_function.reference | 6 ++-- .../00915_simple_aggregate_function.sql | 13 ++++---- 8 files changed, 62 insertions(+), 29 deletions(-) diff --git a/docs/en/sql_reference/aggregate_functions/reference.md b/docs/en/sql_reference/aggregate_functions/reference.md index d7bc8e963e2..f20e32db14e 100644 --- a/docs/en/sql_reference/aggregate_functions/reference.md +++ b/docs/en/sql_reference/aggregate_functions/reference.md @@ -330,9 +330,10 @@ Computes the sum of the numbers, using the same data type for the result as for Only works for numbers. -## sumMap(key, value) {#agg_functions-summap} +## sumMap(key, value), sumMap(Tuple(key, value)) {#agg_functions-summap} Totals the ‘value’ array according to the keys specified in the ‘key’ array. +Passing tuple of keys and values arrays is synonymical to passing two arrays of keys and values. The number of elements in ‘key’ and ‘value’ must be the same for each row that is totaled. Returns a tuple of two arrays: keys in sorted order, and values ​​summed for the corresponding keys. @@ -345,25 +346,28 @@ CREATE TABLE sum_map( statusMap Nested( status UInt16, requests UInt64 - ) + ), + statusMapTuple Tuple(Array(Int32), Array(Int32)) ) ENGINE = Log; INSERT INTO sum_map VALUES - ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10]), - ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10]), - ('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10]), - ('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10]); + ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10], ([1, 2, 3], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10], ([3, 4, 5], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10], ([4, 5, 6], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10], ([6, 7, 8], [10, 10, 10])); + SELECT timeslot, - sumMap(statusMap.status, statusMap.requests) + sumMap(statusMap.status, statusMap.requests), + sumMap(statusMapTuple) FROM sum_map GROUP BY timeslot ``` ``` text -┌────────────timeslot─┬─sumMap(statusMap.status, statusMap.requests)─┐ -│ 2000-01-01 00:00:00 │ ([1,2,3,4,5],[10,10,20,10,10]) │ -│ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │ -└─────────────────────┴──────────────────────────────────────────────┘ +┌────────────timeslot─┬─sumMap(statusMap.status, statusMap.requests)─┬─sumMap(statusMapTuple)─────────┐ +│ 2000-01-01 00:00:00 │ ([1,2,3,4,5],[10,10,20,10,10]) │ ([1,2,3,4,5],[10,10,20,10,10]) │ +│ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │ ([4,5,6,7,8],[10,10,20,10,10]) │ +└─────────────────────┴──────────────────────────────────────────────┴────────────────────────────────┘ ``` ## skewPop {#skewpop} diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 5bedf72c39b..6191b26a855 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -52,23 +52,37 @@ using SumMapArgs = std::pair; SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) { - if (arguments.size() < 2) - throw Exception("Aggregate function " + name + " requires at least two arguments of Array type.", + DataTypes args; + + if (arguments.size() == 1) + { + const auto * tuple_type = checkAndGetDataType(arguments[0].get()); + if (!tuple_type) + throw Exception("When function " + name + " gets one argument it must be a tuple", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto elems = tuple_type->getElements(); + args.insert(args.end(), elems.begin(), elems.end()); + } + else + args.insert(args.end(), arguments.begin(), arguments.end()); + + if (args.size() < 2) + throw Exception("Aggregate function " + name + " requires at least two arguments of Array type or one argument of tuple of two arrays", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - const auto * array_type = checkAndGetDataType(arguments[0].get()); + const auto * array_type = checkAndGetDataType(args[0].get()); if (!array_type) - throw Exception("First argument for function " + name + " must be an array.", + throw Exception("First argument for function " + name + " must be an array, not " + args[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - DataTypePtr keys_type = array_type->getNestedType(); DataTypes values_types; - values_types.reserve(arguments.size() - 1); - for (size_t i = 1; i < arguments.size(); ++i) + values_types.reserve(args.size() - 1); + for (size_t i = 1; i < args.size(); ++i) { - array_type = checkAndGetDataType(arguments[i].get()); + array_type = checkAndGetDataType(args[i].get()); if (!array_type) throw Exception("Argument #" + toString(i) + " for function " + name + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 88f99b73841..88ee10f4627 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -80,6 +80,18 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override { + // Check if tuple + std::unique_ptr tuple_columns; + auto tuple_col = checkAndGetColumn(columns[0]); + if (tuple_col) + { + tuple_columns.reset(new const IColumn*[tuple_col->tupleSize()]); + for (size_t i = 0; i < tuple_col->tupleSize(); i++) + tuple_columns.get()[i] = &const_cast(tuple_col->getColumn(i)); + + columns = tuple_columns.get(); + } + // Column 0 contains array of keys of known type Field key_field; const ColumnArray & array_column0 = assert_cast(*columns[0]); diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index 3072e4a40c1..bf22845a5f6 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -30,7 +30,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static const std::vector supported_functions{"any", "anyLast", "min", "max", "sum", "groupBitAnd", "groupBitOr", "groupBitXor"}; +static const std::vector supported_functions{"any", "anyLast", "min", "max", "sum", "groupBitAnd", "groupBitOr", "groupBitXor", "sumMap"}; String DataTypeCustomSimpleAggregateFunction::getName() const diff --git a/tests/queries/0_stateless/00502_sum_map.reference b/tests/queries/0_stateless/00502_sum_map.reference index 6cddf662424..0002c43945a 100644 --- a/tests/queries/0_stateless/00502_sum_map.reference +++ b/tests/queries/0_stateless/00502_sum_map.reference @@ -4,6 +4,7 @@ 2000-01-01 2000-01-01 00:01:00 [6,7,8] [10,10,10] ([1,2,3,4,5,6,7,8],[10,10,20,20,20,20,10,10]) ([1,2,3,4,5,6,7,8],[10,10,20,20,20,20,10,10]) +([1,2,3,4,5,6,7,8],[10,10,20,20,20,20,10,10]) 2000-01-01 00:00:00 ([1,2,3,4,5],[10,10,20,10,10]) 2000-01-01 00:01:00 ([4,5,6,7,8],[10,10,20,10,10]) 2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10] diff --git a/tests/queries/0_stateless/00502_sum_map.sql b/tests/queries/0_stateless/00502_sum_map.sql index dba8bb5549f..6a4035a3782 100644 --- a/tests/queries/0_stateless/00502_sum_map.sql +++ b/tests/queries/0_stateless/00502_sum_map.sql @@ -7,6 +7,7 @@ INSERT INTO sum_map VALUES ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, SELECT * FROM sum_map ORDER BY timeslot; SELECT sumMap(statusMap.status, statusMap.requests) FROM sum_map; +SELECT sumMap((statusMap.status, statusMap.requests)) FROM sum_map; SELECT sumMapMerge(s) FROM (SELECT sumMapState(statusMap.status, statusMap.requests) AS s FROM sum_map); SELECT timeslot, sumMap(statusMap.status, statusMap.requests) FROM sum_map GROUP BY timeslot ORDER BY timeslot; SELECT timeslot, sumMap(statusMap.status, statusMap.requests).1, sumMap(statusMap.status, statusMap.requests).2 FROM sum_map GROUP BY timeslot ORDER BY timeslot; diff --git a/tests/queries/0_stateless/00915_simple_aggregate_function.reference b/tests/queries/0_stateless/00915_simple_aggregate_function.reference index 6fb5e7b3744..d9e0a92cb01 100644 --- a/tests/queries/0_stateless/00915_simple_aggregate_function.reference +++ b/tests/queries/0_stateless/00915_simple_aggregate_function.reference @@ -39,6 +39,6 @@ SimpleAggregateFunction(sum, Float64) 7 14 8 16 9 18 -1 1 2 2.2.2.2 3 -10 2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222 20 20.20.20.20 5 -SimpleAggregateFunction(anyLast, Nullable(String)) SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) SimpleAggregateFunction(anyLast, IPv4) SimpleAggregateFunction(groupBitOr, UInt32) +1 1 2 2.2.2.2 3 ([1,2,3],[2,1,1]) +10 2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222 20 20.20.20.20 5 ([2,3,4],[2,1,1]) +SimpleAggregateFunction(anyLast, Nullable(String)) SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) SimpleAggregateFunction(anyLast, IPv4) SimpleAggregateFunction(groupBitOr, UInt32) SimpleAggregateFunction(sumMap, Tuple(Array(Int32), Array(Int64))) diff --git a/tests/queries/0_stateless/00915_simple_aggregate_function.sql b/tests/queries/0_stateless/00915_simple_aggregate_function.sql index 030893e3ea1..1866e2bc8c5 100644 --- a/tests/queries/0_stateless/00915_simple_aggregate_function.sql +++ b/tests/queries/0_stateless/00915_simple_aggregate_function.sql @@ -24,16 +24,17 @@ create table simple ( nullable_str SimpleAggregateFunction(anyLast,Nullable(String)), low_str SimpleAggregateFunction(anyLast,LowCardinality(Nullable(String))), ip SimpleAggregateFunction(anyLast,IPv4), - status SimpleAggregateFunction(groupBitOr, UInt32) + status SimpleAggregateFunction(groupBitOr, UInt32), + tup SimpleAggregateFunction(sumMap, Tuple(Array(Int32), Array(Int64))) ) engine=AggregatingMergeTree order by id; -insert into simple values(1,'1','1','1.1.1.1', 1); -insert into simple values(1,null,'2','2.2.2.2', 2); +insert into simple values(1,'1','1','1.1.1.1', 1, ([1,2], [1,1])); +insert into simple values(1,null,'2','2.2.2.2', 2, ([1,3], [1,1])); -- String longer then MAX_SMALL_STRING_SIZE (actual string length is 100) -insert into simple values(10,'10','10','10.10.10.10', 4); -insert into simple values(10,'2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222','20','20.20.20.20', 1); +insert into simple values(10,'10','10','10.10.10.10', 4, ([2,3], [1,1])); +insert into simple values(10,'2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222','20','20.20.20.20', 1, ([2, 4], [1,1])); select * from simple final; -select toTypeName(nullable_str),toTypeName(low_str),toTypeName(ip),toTypeName(status) from simple limit 1; +select toTypeName(nullable_str),toTypeName(low_str),toTypeName(ip),toTypeName(status), toTypeName(tup) from simple limit 1; optimize table simple final; From fcacca68d7ee3ae7729db23a474057ee574b1c81 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Thu, 16 Apr 2020 17:42:19 +0200 Subject: [PATCH 295/743] Update src/AggregateFunctions/AggregateFunctionSumMap.cpp Co-Authored-By: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> --- src/AggregateFunctions/AggregateFunctionSumMap.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 6191b26a855..3fc24f4c064 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -56,6 +56,9 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) if (arguments.size() == 1) { + // sumMap is a transitive function, so it can be stored in SimpleAggregateFunction columns. + // There is a caveat: it must support sumMap(sumMap(...)), e.g. it must be able to accept its + // own output as an input. This is why we also support Tuple(keys, values) as an argument. const auto * tuple_type = checkAndGetDataType(arguments[0].get()); if (!tuple_type) throw Exception("When function " + name + " gets one argument it must be a tuple", From 15dc45b9b25246f03e11138fd711f3f423f55461 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Apr 2020 19:44:49 +0300 Subject: [PATCH 296/743] Update extended_roadmap.md --- docs/ru/whats_new/extended_roadmap.md | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/docs/ru/whats_new/extended_roadmap.md b/docs/ru/whats_new/extended_roadmap.md index 135a49ca8fb..219eb188e8f 100644 --- a/docs/ru/whats_new/extended_roadmap.md +++ b/docs/ru/whats_new/extended_roadmap.md @@ -15,9 +15,7 @@ Задача «normalized z-Order curve» в перспективе может быть полезна для БК и Метрики, так как позволяет смешивать OrderID и PageID и избежать дублирования данных. В задаче также вводится способ индексации путём обращения функции нескольких аргументов на интервале, что имеет смысл для дальнейшего развития. -Изначально делал [Андрей Чулков](https://github.com/achulkov2), ВШЭ, теперь (не) доделывает [Ольга Хвостикова](https://github.com/stavrolia), но сроки немного сдвинуты из-за задачи 25.9. Будем надеятся на лучшее. - -Upd. Доделывать будет другой человек. Приоритет не высокий. +[Андрей Чулков](https://github.com/achulkov2), ВШЭ. ### 1.2. Wait-free каталог баз данных. {#wait-free-katalog-baz-dannykh} @@ -869,8 +867,6 @@ Upd. Нас заставляют переписать эту библиотек ### 10.14. Поддержка всех типов в функции transform. {#podderzhka-vsekh-tipov-v-funktsii-transform} -Задачу взяла Ольга Хвостикова. Upd. Статус неизвестен. - ### 10.15. Использование словарей как специализированного layout для Join. {#ispolzovanie-slovarei-kak-spetsializirovannogo-layout-dlia-join} ### 10.16. Словари на локальном SSD. {#slovari-na-lokalnom-ssd} @@ -1414,8 +1410,6 @@ N.Vartolomei. ### 22.3. Защита от абсурдно заданных пользователем кодеков. {#zashchita-ot-absurdno-zadannykh-polzovatelem-kodekov} -В очереди, скорее всего [Ольга Хвостикова](https://github.com/stavrolia). - ### 22.4. Исправление оставшихся deadlocks в табличных RWLock-ах. {#ispravlenie-ostavshikhsia-deadlocks-v-tablichnykh-rwlock-akh} Александр Казаков. Нужно для Яндекс.Метрики и Datalens. Задача постепенно тащится и исправлениями в соседних местах стала менее актуальна. From c7be8c226cd0ba0610cd784ed6f4d4b743824ba8 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 16 Apr 2020 19:51:29 +0300 Subject: [PATCH 297/743] add more test cases as discussed on tg --- .../0_stateless/01101_literal_column_clash.reference | 7 +++++++ tests/queries/0_stateless/01101_literal_column_clash.sql | 9 +++++++++ tests/queries/bugs/join_constants_on.sql | 2 -- 3 files changed, 16 insertions(+), 2 deletions(-) delete mode 100644 tests/queries/bugs/join_constants_on.sql diff --git a/tests/queries/0_stateless/01101_literal_column_clash.reference b/tests/queries/0_stateless/01101_literal_column_clash.reference index 094b70dd0b7..47458541f79 100644 --- a/tests/queries/0_stateless/01101_literal_column_clash.reference +++ b/tests/queries/0_stateless/01101_literal_column_clash.reference @@ -5,3 +5,10 @@ xyzabc 2 1 3 1 2 0 0 +1 0 0 3 +\N 1 2 \N 0 +\N 1 0 \N 3 +1 +1 +1 hello\0\0\0 +1 hello\0\0\0 diff --git a/tests/queries/0_stateless/01101_literal_column_clash.sql b/tests/queries/0_stateless/01101_literal_column_clash.sql index 75d3ffa693c..2c665d3833b 100644 --- a/tests/queries/0_stateless/01101_literal_column_clash.sql +++ b/tests/queries/0_stateless/01101_literal_column_clash.sql @@ -11,4 +11,13 @@ with 3 as "1" select 1, "1"; -- https://github.com/ClickHouse/ClickHouse/issues/9953 select 1, * from (select 2 x) a left join (select 1, 3 y) b on y = x; +select 1, * from (select 2 x, 1) a right join (select 3 y) b on y = x; +select null, isConstant(null), * from (select 2 x) a left join (select null, 3 y) b on y = x; +select null, isConstant(null), * from (select 2 x, null) a right join (select 3 y) b on y = x; + +-- other cases with joins and constants + +select cast(1, 'UInt8') from (select arrayJoin([1, 2]) as a) t1 left join (select 1 as b) t2 on b = ignore('UInt8'); + +select isConstant('UInt8'), toFixedString('hello', toUInt8(substring('UInt8', 5, 1))) from (select arrayJoin([1, 2]) as a) t1 left join (select 1 as b) t2 on b = ignore('UInt8'); diff --git a/tests/queries/bugs/join_constants_on.sql b/tests/queries/bugs/join_constants_on.sql deleted file mode 100644 index ae967e07adb..00000000000 --- a/tests/queries/bugs/join_constants_on.sql +++ /dev/null @@ -1,2 +0,0 @@ -select cast(1, 'UInt8') from (select arrayJoin([1, 2]) as a) t1 left join (select 1 as b) t2 on b = ignore('UInt8'); -select isConstant('UInt8'), toFixedString('hello', toUInt8(substring('UInt8', 5, 1))) from (select arrayJoin([1, 2]) as a) t1 left join (select 1 as b) t2 on b = ignore('UInt8'); From a9b1b36907b106ce7fcd8cf9d376937be84fc82c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 16 Apr 2020 10:48:49 +0300 Subject: [PATCH 298/743] Use separate pool for buffer flushes (background_buffer_flush_schedule_pool_size) --- src/Core/Settings.h | 1 + src/Interpreters/Context.cpp | 10 +++++ src/Interpreters/Context.h | 1 + src/Storages/StorageBuffer.cpp | 70 +++++++++++++++++++++------------- src/Storages/StorageBuffer.h | 12 +++--- 5 files changed, 62 insertions(+), 32 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 325abc16f3f..73fc4578ce5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -78,6 +78,7 @@ struct Settings : public SettingsCollection M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \ M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ + M(SettingUInt64, background_buffer_flush_schedule_pool_size, 16, "Number of threads performing background flush for tables with Buffer engine. Only has meaning at server startup.", 0) \ M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ M(SettingUInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \ M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.", 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6e30792277f..5d8a0e53276 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -317,6 +317,7 @@ struct ContextShared MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. + std::optional buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. std::optional background_pool; /// The thread pool for the background work performed by the tables. std::optional background_move_pool; /// The thread pool for the background moves performed by the tables. std::optional schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) @@ -413,6 +414,7 @@ struct ContextShared embedded_dictionaries.reset(); external_dictionaries_loader.reset(); external_models_loader.reset(); + buffer_flush_schedule_pool.reset(); background_pool.reset(); background_move_pool.reset(); schedule_pool.reset(); @@ -1330,6 +1332,14 @@ BackgroundProcessingPool & Context::getBackgroundMovePool() return *shared->background_move_pool; } +BackgroundSchedulePool & Context::getBufferFlushSchedulePool() +{ + auto lock = getLock(); + if (!shared->buffer_flush_schedule_pool) + shared->buffer_flush_schedule_pool.emplace(settings.background_buffer_flush_schedule_pool_size); + return *shared->buffer_flush_schedule_pool; +} + BackgroundSchedulePool & Context::getSchedulePool() { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f68b08bf9f0..c9402b6e32a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -468,6 +468,7 @@ public: */ void dropCaches() const; + BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundProcessingPool & getBackgroundPool(); BackgroundProcessingPool & getBackgroundMovePool(); BackgroundSchedulePool & getSchedulePool(); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 1765e663902..4f098b46ff5 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -76,6 +75,7 @@ StorageBuffer::StorageBuffer( , destination_id(destination_id_) , allow_materialized(allow_materialized_) , log(&Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) + , bg_pool(global_context.getBufferFlushSchedulePool()) { setColumns(columns_); setConstraints(constraints_); @@ -83,12 +83,7 @@ StorageBuffer::StorageBuffer( StorageBuffer::~StorageBuffer() { - // Should not happen if shutdown was called - if (flush_thread.joinable()) - { - shutdown_event.set(); - flush_thread.join(); - } + flush_handle->deactivate(); } @@ -397,6 +392,9 @@ public: least_busy_lock = std::unique_lock(least_busy_buffer->mutex); } insertIntoBuffer(block, *least_busy_buffer); + least_busy_lock.unlock(); + + storage.reschedule(); } private: StorageBuffer & storage; @@ -458,16 +456,15 @@ void StorageBuffer::startup() << " Set appropriate system_profile to fix this."); } - flush_thread = ThreadFromGlobalPool(&StorageBuffer::flushThread, this); + + flush_handle = bg_pool.createTask(log->name() + "/Bg", [this]{ flushBack(); }); + flush_handle->activateAndSchedule(); } void StorageBuffer::shutdown() { - shutdown_event.set(); - - if (flush_thread.joinable()) - flush_thread.join(); + flush_handle->deactivate(); try { @@ -595,7 +592,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc ProfileEvents::increment(ProfileEvents::StorageBufferFlush); - LOG_TRACE(log, "Flushing buffer with " << rows << " rows, " << bytes << " bytes, age " << time_passed << " seconds."); + LOG_TRACE(log, "Flushing buffer with " << rows << " rows, " << bytes << " bytes, age " << time_passed << " seconds " << (check_thresholds ? "(bg)" : "(direct)") << "."); if (!destination_id) return; @@ -697,21 +694,42 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl } -void StorageBuffer::flushThread() +void StorageBuffer::flushBack() { - setThreadName("BufferFlush"); - - do + try { - try - { - flushAllBuffers(true); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } while (!shutdown_event.tryWait(1000)); + flushAllBuffers(true); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + reschedule(); +} + +void StorageBuffer::reschedule() +{ + time_t min_first_write_time = std::numeric_limits::max(); + time_t rows = 0; + + for (auto & buffer : buffers) + { + std::lock_guard lock(buffer.mutex); + min_first_write_time = buffer.first_write_time; + rows += buffer.data.rows(); + } + + /// will be rescheduled via INSERT + if (!rows) + return; + + time_t current_time = time(nullptr); + time_t time_passed = current_time - min_first_write_time; + + size_t min = std::max(min_thresholds.time - time_passed, 1); + size_t max = std::max(max_thresholds.time - time_passed, 1); + flush_handle->scheduleAfter(std::min(min, max) * 1000); } void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 93f95692b18..4c6c911e339 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -118,10 +118,6 @@ private: Poco::Logger * log; - Poco::Event shutdown_event; - /// Resets data by timeout. - ThreadFromGlobalPool flush_thread; - void flushAllBuffers(bool check_thresholds = true); /// Reset the buffer. If check_thresholds is set - resets only if thresholds are exceeded. void flushBuffer(Buffer & buffer, bool check_thresholds, bool locked = false); @@ -131,7 +127,11 @@ private: /// `table` argument is passed, as it is sometimes evaluated beforehand. It must match the `destination`. void writeBlockToDestination(const Block & block, StoragePtr table); - void flushThread(); + void flushBack(); + void reschedule(); + + BackgroundSchedulePool & bg_pool; + BackgroundSchedulePoolTaskHolder flush_handle; protected: /** num_shards - the level of internal parallelism (the number of independent buffers) From 602baf62f00c653975432bd9a9f32e9ad70774fa Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Thu, 16 Apr 2020 17:26:17 +0000 Subject: [PATCH 299/743] Improve implementation of sumMap for tuples --- src/AggregateFunctions/AggregateFunctionSumMap.h | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 88ee10f4627..be58a82c0c8 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -81,17 +81,16 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override { // Check if tuple - std::unique_ptr tuple_columns; auto tuple_col = checkAndGetColumn(columns[0]); if (tuple_col) - { - tuple_columns.reset(new const IColumn*[tuple_col->tupleSize()]); - for (size_t i = 0; i < tuple_col->tupleSize(); i++) - tuple_columns.get()[i] = &const_cast(tuple_col->getColumn(i)); - - columns = tuple_columns.get(); - } + addImpl(place, tuple_col->getColumns(), row_num); + else + addImpl(place, columns, row_num); + } + template + void addImpl(AggregateDataPtr place, TColumns & columns, const size_t row_num) const + { // Column 0 contains array of keys of known type Field key_field; const ColumnArray & array_column0 = assert_cast(*columns[0]); From 2afa3cbaa3672be53ebb6852941c0c87d71db915 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 16 Apr 2020 20:23:12 +0300 Subject: [PATCH 300/743] Add a test for Buffer flushing (direct, background min/max time) --- .../0_stateless/01246_buffer_flush.reference | 10 +++++ .../0_stateless/01246_buffer_flush.sql | 44 +++++++++++++++++++ 2 files changed, 54 insertions(+) create mode 100644 tests/queries/0_stateless/01246_buffer_flush.reference create mode 100644 tests/queries/0_stateless/01246_buffer_flush.sql diff --git a/tests/queries/0_stateless/01246_buffer_flush.reference b/tests/queries/0_stateless/01246_buffer_flush.reference new file mode 100644 index 00000000000..a877e94b919 --- /dev/null +++ b/tests/queries/0_stateless/01246_buffer_flush.reference @@ -0,0 +1,10 @@ +min +0 +5 +max +5 +10 +direct +20 +drop +30 diff --git a/tests/queries/0_stateless/01246_buffer_flush.sql b/tests/queries/0_stateless/01246_buffer_flush.sql new file mode 100644 index 00000000000..efe0adf703a --- /dev/null +++ b/tests/queries/0_stateless/01246_buffer_flush.sql @@ -0,0 +1,44 @@ +drop table if exists data_01256; +drop table if exists buffer_01256; + +create table data_01256 as system.numbers Engine=Memory(); + +select 'min'; +create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, + 2, 100, /* time */ + 4, 100, /* rows */ + 1, 1e6 /* bytes */ +); +insert into buffer_01256 select * from system.numbers limit 5; +select count() from data_01256; +-- sleep 2 (min time) + 1 (round up) + bias (1) = 4 +select sleepEachRow(2) from numbers(2) FORMAT Null; +select count() from data_01256; +drop table buffer_01256; + +select 'max'; +create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, + 100, 2, /* time */ + 0, 100, /* rows */ + 0, 1e6 /* bytes */ +); +insert into buffer_01256 select * from system.numbers limit 5; +select count() from data_01256; +-- sleep 2 (min time) + 1 (round up) + bias (1) = 4 +select sleepEachRow(2) from numbers(2) FORMAT Null; +select count() from data_01256; +drop table buffer_01256; + +select 'direct'; +create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, + 100, 100, /* time */ + 0, 9, /* rows */ + 0, 1e6 /* bytes */ +); +insert into buffer_01256 select * from system.numbers limit 10; +select count() from data_01256; + +select 'drop'; +insert into buffer_01256 select * from system.numbers limit 10; +drop table if exists buffer_01256; +select count() from data_01256; From ce372f464bf21efe1207a0bc246072344667ad41 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Thu, 16 Apr 2020 21:03:27 +0300 Subject: [PATCH 301/743] Fix finalize chain logic in ExpressionAnalysisResult (#10302) --- src/Interpreters/ExpressionAnalyzer.cpp | 8 ++++--- .../01117_chain_finalize_bug.reference | 9 +++++++ .../0_stateless/01117_chain_finalize_bug.sql | 24 +++++++++++++++++++ 3 files changed, 38 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01117_chain_finalize_bug.reference create mode 100644 tests/queries/0_stateless/01117_chain_finalize_bug.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 9ec32737fdc..bb2553c76a4 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -963,13 +963,15 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( auto finalize_chain = [&](ExpressionActionsChain & chain) { + chain.finalize(); + if (!finalized) { - chain.finalize(); finalize(chain, context, where_step_num); - chain.clear(); + finalized = true; } - finalized = true; + + chain.clear(); }; { diff --git a/tests/queries/0_stateless/01117_chain_finalize_bug.reference b/tests/queries/0_stateless/01117_chain_finalize_bug.reference new file mode 100644 index 00000000000..5704c399b5c --- /dev/null +++ b/tests/queries/0_stateless/01117_chain_finalize_bug.reference @@ -0,0 +1,9 @@ +1 0 +1 1 +2 0 +2 1 +1 0 +1 1 +2 0 +2 1 +200000 diff --git a/tests/queries/0_stateless/01117_chain_finalize_bug.sql b/tests/queries/0_stateless/01117_chain_finalize_bug.sql new file mode 100644 index 00000000000..f79f82b8d4d --- /dev/null +++ b/tests/queries/0_stateless/01117_chain_finalize_bug.sql @@ -0,0 +1,24 @@ +SELECT arrayJoin(arrayMap(i -> (i + 1), range(2))) AS index, number +FROM numbers(2) +GROUP BY number +ORDER BY index, number; + +SET max_bytes_before_external_group_by = 1; + +SELECT arrayJoin(arrayMap(i -> (i + 1), range(2))) AS index, number +FROM numbers(2) +GROUP BY number +ORDER BY index, number; + +SET group_by_two_level_threshold = 2; + +SELECT count() FROM +( + SELECT + arrayJoin(arrayMap(i -> (i + 1), range(2))) AS index, + number + FROM numbers_mt(100000) + GROUP BY number + ORDER BY index ASC + SETTINGS max_block_size = 100000, max_threads = 2 +); From 80873d79e3d7935b05626fff479febd387d4cdf8 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 16 Apr 2020 21:03:52 +0300 Subject: [PATCH 302/743] [docs] tuning of
    tag, search menu colours and hidden pages (#10313) * improve hidden page title * adjust docsearch menu colors * enable pymdownx.details * some
    styling * remove useless example --- docs/en/index.md | 42 ----------------------------- docs/es/index.md | 42 ----------------------------- docs/fa/index.md | 48 --------------------------------- docs/fr/index.md | 42 ----------------------------- docs/ja/index.md | 42 ----------------------------- docs/ru/index.md | 42 ----------------------------- docs/tools/build.py | 1 + docs/tr/index.md | 42 ----------------------------- docs/zh/index.md | 48 --------------------------------- website/css/docs.css | 21 ++++++++++++++- website/templates/docs/nav.html | 6 ++++- 11 files changed, 26 insertions(+), 350 deletions(-) diff --git a/docs/en/index.md b/docs/en/index.md index e38a89abd49..67c4ca4d53b 100644 --- a/docs/en/index.md +++ b/docs/en/index.md @@ -78,48 +78,6 @@ See the difference? For example, the query “count the number of records for each advertising platform” requires reading one “advertising platform ID” column, which takes up 1 byte uncompressed. If most of the traffic was not from advertising platforms, you can expect at least 10-fold compression of this column. When using a quick compression algorithm, data decompression is possible at a speed of at least several gigabytes of uncompressed data per second. In other words, this query can be processed at a speed of approximately several billion rows per second on a single server. This speed is actually achieved in practice. -
    - -Example - -``` bash -$ clickhouse-client -ClickHouse client version 0.0.52053. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.52053. -``` - -``` sql -SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 -``` - -``` text -┌─CounterID─┬──count()─┐ -│ 114208 │ 56057344 │ -│ 115080 │ 51619590 │ -│ 3228 │ 44658301 │ -│ 38230 │ 42045932 │ -│ 145263 │ 42042158 │ -│ 91244 │ 38297270 │ -│ 154139 │ 26647572 │ -│ 150748 │ 24112755 │ -│ 242232 │ 21302571 │ -│ 338158 │ 13507087 │ -│ 62180 │ 12229491 │ -│ 82264 │ 12187441 │ -│ 232261 │ 12148031 │ -│ 146272 │ 11438516 │ -│ 168777 │ 11403636 │ -│ 4120072 │ 11227824 │ -│ 10938808 │ 10519739 │ -│ 74088 │ 9047015 │ -│ 115079 │ 8837972 │ -│ 337234 │ 8205961 │ -└───────────┴──────────┘ -``` - -
    - ### CPU {#cpu} Since executing a query requires processing a large number of rows, it helps to dispatch all operations for entire vectors instead of for separate rows, or to implement the query engine so that there is almost no dispatching cost. If you don’t do this, with any half-decent disk subsystem, the query interpreter inevitably stalls the CPU. It makes sense to both store data in columns and process it, when possible, by columns. diff --git a/docs/es/index.md b/docs/es/index.md index 2cbe375e8ee..1348ef7013d 100644 --- a/docs/es/index.md +++ b/docs/es/index.md @@ -80,48 +80,6 @@ Ver la diferencia? Por ejemplo, la consulta “count the number of records for each advertising platform” requiere leer uno “advertising platform ID” columna, que ocupa 1 byte sin comprimir. Si la mayor parte del tráfico no proviene de plataformas publicitarias, puede esperar al menos una compresión de 10 veces de esta columna. Cuando se utiliza un algoritmo de compresión rápida, la descompresión de datos es posible a una velocidad de al menos varios gigabytes de datos sin comprimir por segundo. En otras palabras, esta consulta se puede procesar a una velocidad de aproximadamente varios miles de millones de filas por segundo en un único servidor. Esta velocidad se logra realmente en la práctica. -
    - -Ejemplo - -``` bash -$ clickhouse-client -ClickHouse client version 0.0.52053. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.52053. -``` - -``` sql -SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 -``` - -``` text -┌─CounterID─┬──count()─┐ -│ 114208 │ 56057344 │ -│ 115080 │ 51619590 │ -│ 3228 │ 44658301 │ -│ 38230 │ 42045932 │ -│ 145263 │ 42042158 │ -│ 91244 │ 38297270 │ -│ 154139 │ 26647572 │ -│ 150748 │ 24112755 │ -│ 242232 │ 21302571 │ -│ 338158 │ 13507087 │ -│ 62180 │ 12229491 │ -│ 82264 │ 12187441 │ -│ 232261 │ 12148031 │ -│ 146272 │ 11438516 │ -│ 168777 │ 11403636 │ -│ 4120072 │ 11227824 │ -│ 10938808 │ 10519739 │ -│ 74088 │ 9047015 │ -│ 115079 │ 8837972 │ -│ 337234 │ 8205961 │ -└───────────┴──────────┘ -``` - -
    - ### CPU {#cpu} Dado que la ejecución de una consulta requiere procesar un gran número de filas, ayuda enviar todas las operaciones para vectores completos en lugar de para filas separadas, o implementar el motor de consultas para que casi no haya costo de envío. Si no hace esto, con cualquier subsistema de disco medio decente, el intérprete de consultas inevitablemente detiene la CPU. Tiene sentido almacenar datos en columnas y procesarlos, cuando sea posible, por columnas. diff --git a/docs/fa/index.md b/docs/fa/index.md index a8f1c201e37..f2b8f433edb 100644 --- a/docs/fa/index.md +++ b/docs/fa/index.md @@ -79,54 +79,6 @@ ClickHouse یک مدیریت دیتابیس (DBMS) ستون گرا برای پر برای مثال، query «تعداد رکوردها به ازای هر بستر نیازمندی» نیازمند خواندن ستون «آیدی بستر آگهی»، که 1 بایت بدون فشرده طول می کشد، خواهد بود. اگر بیشتر ترافیک مربوط به بستر های نیازمندی نبود، شما می توانید انتظار حداقل 10 برابر فشرده سازی این ستون را داشته باشید. زمانی که از الگوریتم فشرده سازی quick استفاده می کنید، عملیات decompression داده ها با سرعت حداقل چندین گیگابایت در ثانیه انجام می شود. به عبارت دیگر، این query توانایی پردازش تقریبا چندین میلیارد رکورد در ثانیه به ازای یک سرور را دارد. این سرعت در عمل واقعی و دست یافتنی است. -
    - -مثال - - $ clickhouse-client - ClickHouse client version 0.0.52053. - Connecting to localhost:9000. - Connected to ClickHouse server version 0.0.52053. - - :) SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 - - SELECT - CounterID, - count() - FROM hits - GROUP BY CounterID - ORDER BY count() DESC - LIMIT 20 - - ┌─CounterID─┬──count()─┐ - │ 114208 │ 56057344 │ - │ 115080 │ 51619590 │ - │ 3228 │ 44658301 │ - │ 38230 │ 42045932 │ - │ 145263 │ 42042158 │ - │ 91244 │ 38297270 │ - │ 154139 │ 26647572 │ - │ 150748 │ 24112755 │ - │ 242232 │ 21302571 │ - │ 338158 │ 13507087 │ - │ 62180 │ 12229491 │ - │ 82264 │ 12187441 │ - │ 232261 │ 12148031 │ - │ 146272 │ 11438516 │ - │ 168777 │ 11403636 │ - │ 4120072 │ 11227824 │ - │ 10938808 │ 10519739 │ - │ 74088 │ 9047015 │ - │ 115079 │ 8837972 │ - │ 337234 │ 8205961 │ - └───────────┴──────────┘ - - 20 rows in set. Elapsed: 0.153 sec. Processed 1.00 billion rows, 4.00 GB (6.53 billion rows/s., 26.10 GB/s.) - - :) - -
    - ### CPU {#cpu} از آنجایی که اجرای یک query نیازمند پردازش تعداد زیادی سطر می باشد، این کمک می کند تا تمام عملیات ها به جای ارسال به سطرهای جداگانه، برای کل بردار ارسال شود، یا برای ترکیب query engine به طوری که هیچ هزینه ی ارسالی وجود ندارد. اگر این کار رو نکنید، با هر half-decent disk subsystem، تفسیرگر query ناگزیر است که CPU را متوقف کند. این منطقی است که که در صورت امکان هر دو کار ذخیره سازی داده در ستون ها و پردازش ستون ها با هم انجام شود. diff --git a/docs/fr/index.md b/docs/fr/index.md index 0e53167306e..bcdeda0a693 100644 --- a/docs/fr/index.md +++ b/docs/fr/index.md @@ -80,48 +80,6 @@ Vous voyez la différence? Par exemple, la requête “count the number of records for each advertising platform” nécessite la lecture d'un “advertising platform ID” colonne, qui prend 1 octet non compressé. Si la majeure partie du trafic ne provenait pas de plates-formes publicitaires, vous pouvez vous attendre à une compression d'au moins 10 fois de cette colonne. Lors de l'utilisation d'un algorithme de compression rapide, la décompression des données est possible à une vitesse d'au moins plusieurs gigaoctets de données non compressées par seconde. En d'autres termes, cette requête ne peut être traitée qu'à une vitesse d'environ plusieurs milliards de lignes par seconde sur un seul serveur. Cette vitesse est effectivement atteinte dans la pratique. -
    - -Exemple - -``` bash -$ clickhouse-client -ClickHouse client version 0.0.52053. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.52053. -``` - -``` sql -SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 -``` - -``` text -┌─CounterID─┬──count()─┐ -│ 114208 │ 56057344 │ -│ 115080 │ 51619590 │ -│ 3228 │ 44658301 │ -│ 38230 │ 42045932 │ -│ 145263 │ 42042158 │ -│ 91244 │ 38297270 │ -│ 154139 │ 26647572 │ -│ 150748 │ 24112755 │ -│ 242232 │ 21302571 │ -│ 338158 │ 13507087 │ -│ 62180 │ 12229491 │ -│ 82264 │ 12187441 │ -│ 232261 │ 12148031 │ -│ 146272 │ 11438516 │ -│ 168777 │ 11403636 │ -│ 4120072 │ 11227824 │ -│ 10938808 │ 10519739 │ -│ 74088 │ 9047015 │ -│ 115079 │ 8837972 │ -│ 337234 │ 8205961 │ -└───────────┴──────────┘ -``` - -
    - ### CPU {#cpu} Étant donné que l'exécution d'une requête nécessite le traitement d'un grand nombre de lignes, il est utile de répartir toutes les opérations pour des vecteurs entiers au lieu de lignes séparées, ou d'implémenter le moteur de requête de sorte qu'il n'y ait presque aucun coût d'expédition. Si vous ne le faites pas, avec un sous-système de disque à moitié décent, l'interpréteur de requête bloque inévitablement le processeur. Il est logique de stocker des données dans des colonnes et de les traiter, si possible, par des colonnes. diff --git a/docs/ja/index.md b/docs/ja/index.md index 07c2724eb0e..4dc166d7f5c 100644 --- a/docs/ja/index.md +++ b/docs/ja/index.md @@ -82,48 +82,6 @@ OLAPシナリオは、他の一般的なシナリオ(OLTPやKey-Valueアクセ たとえば、「各広告プラットフォームのレコード数をカウントする」クエリでは、1つの「広告プラットフォームID」列を読み取る必要がありますが、これは非圧縮では1バイトの領域を要します。トラフィックのほとんどが広告プラットフォームからのものではない場合、この列は少なくとも10倍の圧縮が期待できます。高速な圧縮アルゴリズムを使用すれば、1秒あたり少なくとも非圧縮データに換算して数ギガバイトの速度でデータを展開できます。つまり、このクエリは、単一のサーバーで1秒あたり約数十億行の速度で処理できます。この速度はまさに実際に達成されます。 -
    - -Example - -``` bash -$ clickhouse-client -ClickHouse client version 0.0.52053. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.52053. -``` - -``` sql -SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 -``` - -``` text -┌─CounterID─┬──count()─┐ -│ 114208 │ 56057344 │ -│ 115080 │ 51619590 │ -│ 3228 │ 44658301 │ -│ 38230 │ 42045932 │ -│ 145263 │ 42042158 │ -│ 91244 │ 38297270 │ -│ 154139 │ 26647572 │ -│ 150748 │ 24112755 │ -│ 242232 │ 21302571 │ -│ 338158 │ 13507087 │ -│ 62180 │ 12229491 │ -│ 82264 │ 12187441 │ -│ 232261 │ 12148031 │ -│ 146272 │ 11438516 │ -│ 168777 │ 11403636 │ -│ 4120072 │ 11227824 │ -│ 10938808 │ 10519739 │ -│ 74088 │ 9047015 │ -│ 115079 │ 8837972 │ -│ 337234 │ 8205961 │ -└───────────┴──────────┘ -``` - -
    - ### CPU {#cpu} クエリを実行するには大量の行を処理する必要があるため、個別の行ではなくベクター全体のすべての操作をディスパッチするか、ディスパッチコストがほとんどないようにクエリエンジンを実装すると効率的です。 適切なディスクサブシステムでこれを行わないと、クエリインタープリターが必然的にCPUを失速させます。 diff --git a/docs/ru/index.md b/docs/ru/index.md index 4636c2cb5df..e537c240cb6 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -82,48 +82,6 @@ ClickHouse - столбцовая система управления базам Например, для запроса «посчитать количество записей для каждой рекламной системы», требуется прочитать один столбец «идентификатор рекламной системы», который занимает 1 байт в несжатом виде. Если большинство переходов было не с рекламных систем, то можно рассчитывать хотя бы на десятикратное сжатие этого столбца. При использовании быстрого алгоритма сжатия, возможно разжатие данных со скоростью более нескольких гигабайт несжатых данных в секунду. То есть, такой запрос может выполняться со скоростью около нескольких миллиардов строк в секунду на одном сервере. На практике, такая скорость действительно достигается. -
    - -Пример - -``` bash -$ clickhouse-client -ClickHouse client version 0.0.52053. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.52053. -``` - -``` sql -SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 -``` - -``` text -┌─CounterID─┬──count()─┐ -│ 114208 │ 56057344 │ -│ 115080 │ 51619590 │ -│ 3228 │ 44658301 │ -│ 38230 │ 42045932 │ -│ 145263 │ 42042158 │ -│ 91244 │ 38297270 │ -│ 154139 │ 26647572 │ -│ 150748 │ 24112755 │ -│ 242232 │ 21302571 │ -│ 338158 │ 13507087 │ -│ 62180 │ 12229491 │ -│ 82264 │ 12187441 │ -│ 232261 │ 12148031 │ -│ 146272 │ 11438516 │ -│ 168777 │ 11403636 │ -│ 4120072 │ 11227824 │ -│ 10938808 │ 10519739 │ -│ 74088 │ 9047015 │ -│ 115079 │ 8837972 │ -│ 337234 │ 8205961 │ -└───────────┴──────────┘ -``` - -
    - ### По вычислениям {#po-vychisleniiam} Так как для выполнения запроса надо обработать достаточно большое количество строк, становится актуальным диспетчеризовывать все операции не для отдельных строк, а для целых векторов, или реализовать движок выполнения запроса так, чтобы издержки на диспетчеризацию были примерно нулевыми. Если этого не делать, то при любой не слишком плохой дисковой подсистеме, интерпретатор запроса неизбежно упрётся в CPU. diff --git a/docs/tools/build.py b/docs/tools/build.py index 4f3a5378b34..4876c6e7566 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -111,6 +111,7 @@ def build_for_lang(lang, args): 'codehilite', 'nl2br', 'sane_lists', + 'pymdownx.details', 'pymdownx.magiclink', 'pymdownx.superfences', 'extra', diff --git a/docs/tr/index.md b/docs/tr/index.md index fb65930b39c..5cbf9330750 100644 --- a/docs/tr/index.md +++ b/docs/tr/index.md @@ -78,48 +78,6 @@ Farkı görüyor musunuz? Örneğin, sorgu “count the number of records for each advertising platform” bir okuma gerektirir “advertising platform ID” 1 bayt sıkıştırılmamış kadar alır sütun. Trafiğin çoğu reklam platformlarından değilse, bu sütunun en az 10 kat sıkıştırılmasını bekleyebilirsiniz. Hızlı bir sıkıştırma algoritması kullanırken, saniyede en az birkaç gigabayt sıkıştırılmamış veri hızında veri dekompresyonu mümkündür. Başka bir deyişle, bu sorgu, tek bir sunucuda saniyede yaklaşık birkaç milyar satır hızında işlenebilir. Bu hız aslında pratikte elde edilir. -
    - -Örnek - -``` bash -$ clickhouse-client -ClickHouse client version 0.0.52053. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.52053. -``` - -``` sql -SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 -``` - -``` text -┌─CounterID─┬──count()─┐ -│ 114208 │ 56057344 │ -│ 115080 │ 51619590 │ -│ 3228 │ 44658301 │ -│ 38230 │ 42045932 │ -│ 145263 │ 42042158 │ -│ 91244 │ 38297270 │ -│ 154139 │ 26647572 │ -│ 150748 │ 24112755 │ -│ 242232 │ 21302571 │ -│ 338158 │ 13507087 │ -│ 62180 │ 12229491 │ -│ 82264 │ 12187441 │ -│ 232261 │ 12148031 │ -│ 146272 │ 11438516 │ -│ 168777 │ 11403636 │ -│ 4120072 │ 11227824 │ -│ 10938808 │ 10519739 │ -│ 74088 │ 9047015 │ -│ 115079 │ 8837972 │ -│ 337234 │ 8205961 │ -└───────────┴──────────┘ -``` - -
    - ### CPU {#cpu} Bir sorguyu yürütmek çok sayıda satırı işlemeyi gerektirdiğinden, ayrı satırlar yerine tüm vektörler için tüm işlemlerin gönderilmesine veya sorgu motorunun neredeyse hiç gönderim maliyeti olmaması için uygulanmasına yardımcı olur. Bunu yapmazsanız, yarı iyi bir disk alt sistemi ile, sorgu yorumlayıcısı kaçınılmaz olarak CPU'yu durdurur. Hem verileri sütunlarda depolamak hem de mümkün olduğunda sütunlarla işlemek mantıklıdır. diff --git a/docs/zh/index.md b/docs/zh/index.md index cb9ccf0420a..455a578456b 100644 --- a/docs/zh/index.md +++ b/docs/zh/index.md @@ -81,54 +81,6 @@ ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS) 例如,查询«统计每个广告平台的记录数量»需要读取«广告平台ID»这一列,它在未压缩的情况下需要1个字节进行存储。如果大部分流量不是来自广告平台,那么这一列至少可以以十倍的压缩率被压缩。当采用快速压缩算法,它的解压速度最少在十亿字节(未压缩数据)每秒。换句话说,这个查询可以在单个服务器上以每秒大约几十亿行的速度进行处理。这实际上是当前实现的速度。 -
    - -示例 - - $ clickhouse-client - ClickHouse client version 0.0.52053. - Connecting to localhost:9000. - Connected to ClickHouse server version 0.0.52053. - - :) SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 - - SELECT - CounterID, - count() - FROM hits - GROUP BY CounterID - ORDER BY count() DESC - LIMIT 20 - - ┌─CounterID─┬──count()─┐ - │ 114208 │ 56057344 │ - │ 115080 │ 51619590 │ - │ 3228 │ 44658301 │ - │ 38230 │ 42045932 │ - │ 145263 │ 42042158 │ - │ 91244 │ 38297270 │ - │ 154139 │ 26647572 │ - │ 150748 │ 24112755 │ - │ 242232 │ 21302571 │ - │ 338158 │ 13507087 │ - │ 62180 │ 12229491 │ - │ 82264 │ 12187441 │ - │ 232261 │ 12148031 │ - │ 146272 │ 11438516 │ - │ 168777 │ 11403636 │ - │ 4120072 │ 11227824 │ - │ 10938808 │ 10519739 │ - │ 74088 │ 9047015 │ - │ 115079 │ 8837972 │ - │ 337234 │ 8205961 │ - └───────────┴──────────┘ - - 20 rows in set. Elapsed: 0.153 sec. Processed 1.00 billion rows, 4.00 GB (6.53 billion rows/s., 26.10 GB/s.) - - :) - -
    - ### CPU {#cpu} 由于执行一个查询需要处理大量的行,因此在整个向量上执行所有操作将比在每一行上执行所有操作更加高效。同时这将有助于实现一个几乎没有调用成本的查询引擎。如果你不这样做,使用任何一个机械硬盘,查询引擎都不可避免的停止CPU进行等待。所以,在数据按列存储并且按列执行是很有意义的。 diff --git a/website/css/docs.css b/website/css/docs.css index f4c4857bc7e..7e4e1040848 100644 --- a/website/css/docs.css +++ b/website/css/docs.css @@ -1,3 +1,14 @@ +details { + background: #444451; + padding: 1rem; + margin-bottom: 1rem; + margin-top: 1rem; +} + +summary { + font-weight: bold; +} + #sidebar { position: fixed; z-index: 50; @@ -109,6 +120,15 @@ border-color: #333; } +.algolia-autocomplete .algolia-docsearch-suggestion--content { + background-color: #333; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--content:hover, +.algolia-autocomplete .ds-dropdown-menu .ds-suggestion.ds-cursor .algolia-docsearch-suggestion:not(.suggestion-layout-simple) .algolia-docsearch-suggestion--content { + background-color: #444451 !important; +} + .algolia-autocomplete .algolia-docsearch-suggestion--category-header, .algolia-autocomplete .algolia-docsearch-suggestion--subcategory-column, .algolia-autocomplete .algolia-docsearch-suggestion--title, @@ -184,4 +204,3 @@ color: #bbb; } } - diff --git a/website/templates/docs/nav.html b/website/templates/docs/nav.html index 42808819ef3..adc7231658f 100644 --- a/website/templates/docs/nav.html +++ b/website/templates/docs/nav.html @@ -6,7 +6,11 @@ From 85fbf413c161e33e14daea741395234711142a9a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 16 Apr 2020 21:47:20 +0300 Subject: [PATCH 303/743] server crashes, just to save --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 5 ++-- .../MergeTree/MergeTreeDataMergerMutator.h | 4 +-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 26 +++++++++++++++---- .../MergeTree/ReplicatedMergeTreeQueue.h | 14 +++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 25 +++++------------- 5 files changed, 42 insertions(+), 32 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index fc0a60f6fb9..3332a3eee5d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -204,8 +204,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, - String * out_disable_reason, - const AllowedSingleMergePredicate & single_merge) + String * out_disable_reason) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); const auto data_settings = data.getSettings(); @@ -233,7 +232,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( * So we have to check if this part is currently being inserted with quorum and so on and so forth. * Obviously we have to check it manually only for the first part * of each partition because it will be automatically checked for a pair of parts. */ - if (!single_merge(part, nullptr)) + if (!can_merge_callback(nullptr, part, nullptr)) continue; const String & partition_id = part->info.partition_id; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index e40d63bbded..f2f4ea95fd0 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -50,7 +50,6 @@ class MergeTreeDataMergerMutator { public: using AllowedMergingPredicate = std::function; - using AllowedSingleMergePredicate = std::function; MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size); @@ -81,8 +80,7 @@ public: bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge, - String * out_disable_reason = nullptr, - const AllowedSingleMergePredicate & single_merge = [](const MergeTreeData::DataPartPtr &, String *) -> bool { return true; }); + String * out_disable_reason = nullptr); /** Select all the parts in the specified partition for merge, if possible. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 82343cbd879..962ee7bdf0d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1645,8 +1645,24 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( } bool ReplicatedMergeTreeMergePredicate::operator()( - const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, - String * out_reason) const + const MergeTreeData::DataPartPtr & left, + const MergeTreeData::DataPartPtr & right, + String * out_reason) const +{ + if (!left) + throw; + + if (left) + return canMergeTwoParts(left, right, out_reason); + else + return canMergeSinglePart(right, out_reason); +} + + +bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( + const MergeTreeData::DataPartPtr & left, + const MergeTreeData::DataPartPtr & right, + String * out_reason) const { /// A sketch of a proof of why this method actually works: /// @@ -1781,10 +1797,10 @@ bool ReplicatedMergeTreeMergePredicate::operator()( return true; } -bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const +bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( + const MergeTreeData::DataPartPtr & part, + String * out_reason) const { - LOG_FATAL(&Poco::Logger::get("ReplicatedMergeTreeMergePredicate::operator()"), "begin"); - if (part->name == inprogress_quorum_part) { LOG_FATAL(&Poco::Logger::get("ReplicatedMergeTreeMergePredicate"), "operator()"); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index e31c0eb7ab1..f7765c6b55d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -405,13 +405,21 @@ class ReplicatedMergeTreeMergePredicate public: ReplicatedMergeTreeMergePredicate(ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper); + /// Depending on the existence of left part checks a merge predicate for two parts or for single part. + bool operator()(const MergeTreeData::DataPartPtr & left, + const MergeTreeData::DataPartPtr & right, + String * out_reason = nullptr) const; + /// Can we assign a merge with these two parts? /// (assuming that no merge was assigned after the predicate was constructed) /// If we can't and out_reason is not nullptr, set it to the reason why we can't merge. - bool operator()( - const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, - String * out_reason = nullptr) const; + bool canMergeTwoParts(const MergeTreeData::DataPartPtr & left, + const MergeTreeData::DataPartPtr & right, + String * out_reason = nullptr) const; + /// Can we assign a merge this part and some other part? + /// For example a merge of a part and itself is needed for TTL. + /// This predicate is checked for the first part of each partitition. bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; /// Return nonempty optional of desired mutation version and alter version. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 077ffa35b70..8ec9fefab81 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2213,13 +2213,8 @@ void StorageReplicatedMergeTree::mergeSelectingTask() UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); FutureMergedMutatedPart future_merged_part; - auto merge_pred_for_single_part = - [&merge_pred] (const MergeTreeData::DataPartPtr & part, String * explain) -> bool - { - return merge_pred.canMergeSinglePart(part, explain); - }; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, nullptr, merge_pred_for_single_part)) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, nullptr)) { success = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, force_ttl); @@ -2708,9 +2703,6 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id, while (true) { - Coordination::Requests ops; - Coordination::Responses responses; - Coordination::Stat added_parts_stat; String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); @@ -2720,15 +2712,12 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id, parts_with_quorum.fromString(old_added_parts); /// Delete information about particular partition. - - /// Since c++20. if (!parts_with_quorum.added_parts.contains(partition_id)) { /// There is no information about interested part. break; } - /// De Morgan's law if (part_name.empty() || parts_with_quorum.added_parts[partition_id] == part_name) parts_with_quorum.added_parts.erase(partition_id); else @@ -3606,9 +3595,6 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPt String partition_id = getPartitionIDFromQuery(partition, query_context); - if (query_context.getSettingsRef().insert_quorum) - cleanLastPartNode(partition_id); - LogEntry entry; if (dropPartsInPartition(*zookeeper, partition_id, entry, detach)) { @@ -3621,6 +3607,9 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPt waitForAllReplicasToProcessLogEntry(entry); } } + + /// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper. + cleanLastPartNode(partition_id); } @@ -5203,14 +5192,14 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta log_znode_path = dynamic_cast(*op_results.back()).path_created; entry_delete.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - if (query_context.getSettingsRef().insert_quorum) - cleanLastPartNode(partition_id); - if (query_context.getSettingsRef().replication_alter_partitions_sync > 1) { lock1.release(); waitForAllReplicasToProcessLogEntry(entry_delete); } + + /// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper. + cleanLastPartNode(partition_id); } void StorageReplicatedMergeTree::getCommitPartOps( From d5b3b2c7b68a0b493384caabcb6f4a2549449b92 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Thu, 16 Apr 2020 21:56:21 +0300 Subject: [PATCH 304/743] Updated RWLockImpl: Fixed suboptimacy (readers queuing up) + minor tweaks to names and comments (#10303) * Readers queue correction * Comments, renamings and cosmetics --- src/Common/RWLock.cpp | 29 ++++++++--------------------- src/Common/RWLock.h | 12 +++++++++--- 2 files changed, 17 insertions(+), 24 deletions(-) diff --git a/src/Common/RWLock.cpp b/src/Common/RWLock.cpp index d6b8cbd244f..a8dba490fac 100644 --- a/src/Common/RWLock.cpp +++ b/src/Common/RWLock.cpp @@ -154,23 +154,17 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c writers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) } else if (readers_queue.empty() || - (rdlock_owner == readers_queue.begin() && !writers_queue.empty())) + (rdlock_owner == readers_queue.begin() && readers_queue.size() == 1 && !writers_queue.empty())) { readers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) } GroupsContainer::iterator it_group = (type == Type::Write) ? std::prev(writers_queue.end()) : std::prev(readers_queue.end()); + /// Lock is free to acquire if (rdlock_owner == readers_queue.end() && wrlock_owner == writers_queue.end()) { - if (type == Type::Read) - { - rdlock_owner = it_group; /// SM2: nothrow - } - else - { - wrlock_owner = it_group; /// SM2: nothrow - } + (type == Read ? rdlock_owner : wrlock_owner) = it_group; /// SM2: nothrow } else { @@ -192,17 +186,10 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Step 3a. Check if we must handle timeout and exit if (!wait_result) /// Wait timed out! { + /// Rollback(SM1): nothrow if (it_group->requests == 0) { - /// Roll back SM1 - if (type == Read) - { - readers_queue.erase(it_group); /// Rollback(SM1): nothrow - } - else - { - writers_queue.erase(it_group); /// Rollback(SM1): nothrow - } + (type == Read ? readers_queue : writers_queue).erase(it_group); } return nullptr; @@ -224,7 +211,7 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Methods std::list<>::emplace_back() and std::unordered_map<>::emplace() provide strong exception safety /// We only need to roll back the changes to these objects: owner_queries and the readers/writers queue if (it_group->requests == 0) - eraseGroup(it_group); /// Rollback(SM1): nothrow + dropOwnerGroupAndPassOwnership(it_group); /// Rollback(SM1): nothrow throw; } @@ -272,11 +259,11 @@ void RWLockImpl::unlock(GroupsContainer::iterator group_it, const String & query /// If we are the last remaining referrer, remove this QNode and notify the next one if (--group_it->requests == 0) /// SM: nothrow - eraseGroup(group_it); + dropOwnerGroupAndPassOwnership(group_it); } -void RWLockImpl::eraseGroup(GroupsContainer::iterator group_it) noexcept +void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept { rdlock_owner = readers_queue.end(); wrlock_owner = writers_queue.end(); diff --git a/src/Common/RWLock.h b/src/Common/RWLock.h index 43366192cf8..ad0a3f139fc 100644 --- a/src/Common/RWLock.h +++ b/src/Common/RWLock.h @@ -19,9 +19,15 @@ class RWLockImpl; using RWLock = std::shared_ptr; -/// Implements shared lock with FIFO service +/// Implements Readers-Writers locking algorithm that serves requests in "Phase Fair" order. /// (Phase Fair RWLock as suggested in https://www.cs.unc.edu/~anderson/papers/rtsj10-for-web.pdf) -/// Can be acquired recursively (for the same query) in Read mode +/// It is used for synchronizing access to various objects on query level (i.e. Storages). +/// +/// In general, ClickHouse processes queries by multiple threads of execution in parallel. +/// As opposed to the standard OS synchronization primitives (mutexes), this implementation allows +/// unlock() to be called by a thread other than the one, that called lock(). +/// It is also possible to acquire RWLock in Read mode without waiting (FastPath) by multiple threads, +/// that execute the same query (share the same query_id). /// /// NOTE: it is important to allow acquiring the same lock in Read mode without waiting if it is already /// acquired by another thread of the same query. Otherwise the following deadlock is possible: @@ -82,6 +88,6 @@ private: private: RWLockImpl() = default; void unlock(GroupsContainer::iterator group_it, const String & query_id) noexcept; - void eraseGroup(GroupsContainer::iterator group_it) noexcept; + void dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept; }; } From c153cc8fe51c621f73330c67567ace954fa24651 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 16 Apr 2020 22:54:11 +0300 Subject: [PATCH 305/743] fixed :) --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 5 +---- src/Storages/StorageMergeTree.cpp | 6 +++++- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 962ee7bdf0d..4058b24896d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1649,9 +1649,6 @@ bool ReplicatedMergeTreeMergePredicate::operator()( const MergeTreeData::DataPartPtr & right, String * out_reason) const { - if (!left) - throw; - if (left) return canMergeTwoParts(left, right, out_reason); else @@ -1801,9 +1798,9 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( const MergeTreeData::DataPartPtr & part, String * out_reason) const { + LOG_FATAL(&Poco::Logger::get("ReplicatedMergeTreeMergePredicate"), "operator()"); if (part->name == inprogress_quorum_part) { - LOG_FATAL(&Poco::Logger::get("ReplicatedMergeTreeMergePredicate"), "operator()"); if (out_reason) *out_reason = "Quorum insert for part " + part->name + " is currently in progress"; return false; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1aac6717728..5110258834e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -552,8 +552,12 @@ bool StorageMergeTree::merge( { std::lock_guard lock(currently_processing_in_background_mutex); - auto can_merge = [this, &lock] (const DataPartPtr & left, const DataPartPtr & right, String *) + auto can_merge = [this, &lock] (const DataPartPtr & left, const DataPartPtr & right, String *) -> bool { + /// This predicate is checked for the first part of each partition. + /// (left = nullptr, right = "first part of partition") + if (!left) + return !currently_merging_mutating_parts.count(right); return !currently_merging_mutating_parts.count(left) && !currently_merging_mutating_parts.count(right) && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); }; From 6b58998a363b94a891eead84632f68c4f669d58d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 16 Apr 2020 22:57:39 +0300 Subject: [PATCH 306/743] better --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 4058b24896d..262fd73762c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1798,7 +1798,6 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( const MergeTreeData::DataPartPtr & part, String * out_reason) const { - LOG_FATAL(&Poco::Logger::get("ReplicatedMergeTreeMergePredicate"), "operator()"); if (part->name == inprogress_quorum_part) { if (out_reason) From 831c48c5149bab38fc5e88d59149802a50997bd5 Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 16 Apr 2020 23:10:42 +0300 Subject: [PATCH 307/743] Add MarkdownRowOutput format --- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatFactory.h | 1 + .../Formats/Impl/MarkdownRowOutputFormat.cpp | 62 +++++++++++++++++++ .../Formats/Impl/MarkdownRowOutputFormat.h | 39 ++++++++++++ .../01231_markdown_format.reference | 5 ++ .../0_stateless/01231_markdown_format.sql | 6 ++ 6 files changed, 114 insertions(+) create mode 100644 src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/MarkdownRowOutputFormat.h create mode 100644 tests/queries/0_stateless/01231_markdown_format.reference create mode 100644 tests/queries/0_stateless/01231_markdown_format.sql diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 7d741004766..f944a32c4b3 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -377,6 +377,7 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); registerOutputFormatProcessorMySQLWrite(*this); + registerOutputFormatProcessorMarkdown(*this); } FormatFactory & FormatFactory::instance() diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 9199ed89890..39d6eb0cc15 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -198,6 +198,7 @@ void registerOutputFormatProcessorODBCDriver(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); void registerOutputFormatProcessorNull(FormatFactory & factory); void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory); +void registerOutputFormatProcessorMarkdown(FormatFactory & factory); /// Input only formats. void registerInputFormatProcessorCapnProto(FormatFactory & factory); diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp new file mode 100644 index 00000000000..54aecc7b43b --- /dev/null +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp @@ -0,0 +1,62 @@ +#include +#include + +namespace DB +{ + +MarkdownRowOutputFormat::MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) {} + +void MarkdownRowOutputFormat::writePrefix() +{ + auto & header = getPort(PortKind::Main).getHeader(); + size_t columns = header.columns(); + + writeChar('|', out); + for (size_t i = 0; i < columns; ++i) + { + writeEscapedString(header.safeGetByPosition(i).name, out); + writeChar('|', out); + } + writeCString("\n|", out); + String format = ":-:|"; + for (size_t i = 0; i < columns; ++i) + { + writeString(format, out); + } + writeChar('\n', out); +} + +void MarkdownRowOutputFormat::writeRowStartDelimiter() +{ + writeChar('|', out); +} + +void MarkdownRowOutputFormat::writeFieldDelimiter() +{ + writeChar('|', out); +} + +void MarkdownRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("|\n", out); +} + +void MarkdownRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + type.serializeAsTextEscaped(column, row_num, out, format_settings); +} + +void registerOutputFormatProcessorMarkdown(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("Markdown", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, callback, settings); + }); +} + +} \ No newline at end of file diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h new file mode 100644 index 00000000000..2f39cce0181 --- /dev/null +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +class MarkdownRowOutputFormat : public IRowOutputFormat +{ +public: + MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_); + + /// Write higher part of markdown table like this: + /// |columnName1|columnName2|...|columnNameN| + /// |:-:|:-:|...|:-:| + void writePrefix() override; + + /// Write '|' before each row + void writeRowStartDelimiter() override; + + /// Write '|' between values + void writeFieldDelimiter() override; + + /// Write '|\n' after each row + void writeRowEndDelimiter() override ; + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + String getName() const override { return "MarkdownRowOutputFormat"; } + +protected: + const FormatSettings format_settings; +}; + + +} diff --git a/tests/queries/0_stateless/01231_markdown_format.reference b/tests/queries/0_stateless/01231_markdown_format.reference new file mode 100644 index 00000000000..57ad6c51de8 --- /dev/null +++ b/tests/queries/0_stateless/01231_markdown_format.reference @@ -0,0 +1,5 @@ +|id|name|array| +|:-:|:-:|:-:| +|1|name1|[1,2,3]| +|2|name2|[4,5,6]| +|3|name3|[7,8,9]| diff --git a/tests/queries/0_stateless/01231_markdown_format.sql b/tests/queries/0_stateless/01231_markdown_format.sql new file mode 100644 index 00000000000..693664be1ab --- /dev/null +++ b/tests/queries/0_stateless/01231_markdown_format.sql @@ -0,0 +1,6 @@ +DROP TABLE IF EXISTS makrdown; +CREATE TABLE markdown (id UInt32, name String, array Array(Int8)) ENGINE = Memory; +INSERT INTO markdown VALUES (1, 'name1', [1,2,3]), (2, 'name2', [4,5,6]), (3, 'name3', [7,8,9]); + +SELECT * FROM markdown FORMAT Markdown; +DROP TABLE IF EXISTS markdown From 89d787dae3b1639cf24b51b407f531ef727aed7a Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 16 Apr 2020 23:11:52 +0300 Subject: [PATCH 308/743] Add new line at the end of file --- src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp index 54aecc7b43b..398934d22f3 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp @@ -59,4 +59,4 @@ void registerOutputFormatProcessorMarkdown(FormatFactory & factory) }); } -} \ No newline at end of file +} From c62c288e4cc748b03059f48d724b79e4becfd576 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Thu, 16 Apr 2020 23:28:23 +0300 Subject: [PATCH 309/743] fix COMMA and other JOIN mix (#10311) --- src/Interpreters/CrossToInnerJoinVisitor.cpp | 12 ++++++---- .../01117_comma_and_others_join_mix.reference | 23 +++++++++++++++++++ .../01117_comma_and_others_join_mix.sql | 20 ++++++++++++++++ 3 files changed, 51 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01117_comma_and_others_join_mix.reference create mode 100644 tests/queries/0_stateless/01117_comma_and_others_join_mix.sql diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index 903f561b8bb..af694fe634a 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -52,17 +52,21 @@ struct JoinedElement void rewriteCommaToCross() { - if (join) + if (join && join->kind == ASTTableJoin::Kind::Comma) join->kind = ASTTableJoin::Kind::Cross; } - void rewriteCrossToInner(ASTPtr on_expression) + bool rewriteCrossToInner(ASTPtr on_expression) { + if (join->kind != ASTTableJoin::Kind::Cross) + return false; + join->kind = ASTTableJoin::Kind::Inner; join->strictness = ASTTableJoin::Strictness::All; join->on_expression = on_expression; join->children.push_back(join->on_expression); + return true; } ASTPtr arrayJoin() const { return element.array_join; } @@ -329,8 +333,8 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da { if (visitor_data.matchAny(i)) { - joined_tables[i].rewriteCrossToInner(visitor_data.makeOnExpression(i)); - data.done = true; + if (joined_tables[i].rewriteCrossToInner(visitor_data.makeOnExpression(i))) + data.done = true; } } } diff --git a/tests/queries/0_stateless/01117_comma_and_others_join_mix.reference b/tests/queries/0_stateless/01117_comma_and_others_join_mix.reference new file mode 100644 index 00000000000..e50321284f4 --- /dev/null +++ b/tests/queries/0_stateless/01117_comma_and_others_join_mix.reference @@ -0,0 +1,23 @@ +0 0 0 +0 0 1 +0 0 2 +0 0 3 +1 1 0 +1 1 1 +1 1 2 +1 1 3 +- +0 0 0 +0 1 0 +1 0 1 +1 1 1 +2 0 \N +2 1 \N +- +0 0 0 +0 1 1 +0 2 2 +1 0 0 +1 1 1 +1 2 2 +\N \N 3 diff --git a/tests/queries/0_stateless/01117_comma_and_others_join_mix.sql b/tests/queries/0_stateless/01117_comma_and_others_join_mix.sql new file mode 100644 index 00000000000..5fe297956b8 --- /dev/null +++ b/tests/queries/0_stateless/01117_comma_and_others_join_mix.sql @@ -0,0 +1,20 @@ +SET join_use_nulls = 1; + +SELECT * +FROM numbers(2) AS n1 +JOIN numbers(3) AS n2 ON n1.number = n2.number, numbers(4) AS n3 +ORDER BY n1.number, n2.number, n3.number; + +SELECT '-'; + +SELECT * +FROM numbers(3) AS n1, numbers(2) AS n2 +LEFT JOIN numbers(2) AS n3 ON n1.number = n3.number +ORDER BY n1.number, n2.number, n3.number; + +SELECT '-'; + +SELECT * +FROM numbers(2) AS n1, numbers(3) AS n2 +RIGHT JOIN numbers(4) AS n3 ON n2.number = n3.number +ORDER BY n1.number, n2.number, n3.number; From 475ab6feef6f1215d962f8d79a91c78a235be886 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Apr 2020 23:30:34 +0300 Subject: [PATCH 310/743] Added results from Sergey Zakharov. --- website/benchmark_hardware.html | 54 ++++++++++++++++++++++++++++++++- 1 file changed, 53 insertions(+), 1 deletion(-) diff --git a/website/benchmark_hardware.html b/website/benchmark_hardware.html index b45ccb4875c..efb371c588f 100644 --- a/website/benchmark_hardware.html +++ b/website/benchmark_hardware.html @@ -2529,6 +2529,57 @@ var results = [0.022, 0.008, 0.006] ] }, + + { + "system": "AMD EPYC 7502P 32-Core Processor with HT (64 thread) / 512 Gb RAM / mdadm RAID1 SAMSUNG MZQLB3T8HALS-00007 + LVM", + "time": "2020-04-16 00:00:00", + "result": + [ +[0.007, 0.002, 0.002], +[0.022, 0.011, 0.011], +[0.028, 0.017, 0.017], +[0.050, 0.022, 0.022], +[0.098, 0.079, 0.078], +[0.149, 0.125, 0.126], +[0.020, 0.014, 0.014], +[0.015, 0.013, 0.013], +[0.169, 0.148, 0.148], +[0.237, 0.171, 0.168], +[0.103, 0.084, 0.082], +[0.099, 0.085, 0.084], +[0.262, 0.221, 0.221], +[0.312, 0.281, 0.282], +[0.274, 0.259, 0.255], +[0.255, 0.237, 0.237], +[0.616, 0.592, 0.589], +[0.398, 0.364, 0.359], +[1.358, 1.301, 1.292], +[0.056, 0.025, 0.022], +[0.485, 0.265, 0.263], +[0.510, 0.271, 0.255], +[0.938, 0.693, 0.674], +[1.262, 0.396, 0.367], +[0.144, 0.082, 0.081], +[0.088, 0.066, 0.065], +[0.141, 0.084, 0.082], +[0.488, 0.329, 0.325], +[0.441, 0.376, 0.374], +[1.054, 1.049, 1.068], +[0.222, 0.190, 0.189], +[0.386, 0.321, 0.319], +[1.703, 1.625, 1.660], +[1.272, 1.202, 1.207], +[1.276, 1.185, 1.204], +[0.398, 0.382, 0.382], +[0.171, 0.160, 0.160], +[0.071, 0.058, 0.059], +[0.059, 0.055, 0.053], +[0.364, 0.341, 0.354], +[0.028, 0.022, 0.027], +[0.027, 0.020, 0.018], +[0.010, 0.008, 0.005], + ] + }, ]; @@ -2959,7 +3010,8 @@ Results for AWS are from Wolf Kreuzerkrieg.
    Results for Huawei Taishan are from Peng Gao in sina.com.
    Results for Selectel and AMD EPYC 7402P are from Andrey Dudin.
    Results for ProLiant are from Denis Ustinov.
    -Results for AMD EPYC 7502P are from Kostiantyn Velychkovskyi.
    +Results for AMD EPYC 7502P 128GiB are from Kostiantyn Velychkovskyi.
    +Results for AMD EPYC 7502P 512GiB are from Sergey Zakharov.
    Results for Pinebook Pro are from Aleksey R. @kITerE.
    Results for AMD Ryzen are from Alexey Milovidov. Firefox was running in background.
    Results for Azure E32s are from Piotr Maśko.
    From 2672b71b2d7426c3568cab922cff40a6e0886d62 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 16 Apr 2020 23:47:01 +0300 Subject: [PATCH 311/743] Delete extra spaces --- src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp index 398934d22f3..13e03198668 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp @@ -3,7 +3,7 @@ namespace DB { - + MarkdownRowOutputFormat::MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & format_settings_) : IRowOutputFormat(header_, out_, callback), format_settings(format_settings_) {} From fa62be250f02db7128aa539c3a037d964cc22955 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 16 Apr 2020 23:54:52 +0300 Subject: [PATCH 312/743] performance comparison --- tests/performance/array_fill.xml | 6 +-- .../performance/mingroupby-orderbylimit1.xml | 37 +++++++++++++------ tests/performance/simple_join_query.xml | 2 +- 3 files changed, 30 insertions(+), 15 deletions(-) diff --git a/tests/performance/array_fill.xml b/tests/performance/array_fill.xml index d29e02730a3..b858ba03c8e 100644 --- a/tests/performance/array_fill.xml +++ b/tests/performance/array_fill.xml @@ -1,8 +1,8 @@ SELECT arraySlice(arrayFill(x -> ((x % 2) >= 0), range(100000000)), 1, 10) FORMAT Null - SELECT arraySlice(arrayFill(x -> (((x.1) % 2) >= 0), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10) FORMAT Null + SELECT arraySlice(arrayFill(x -> (((x.1) % 2) >= 0), arrayMap(x -> (x, toString(x)), range(10000000))), 1, 10) FORMAT Null SELECT arraySlice(arrayFill(x -> ((x % 2) >= 2), range(100000000)), 1, 10) FORMAT Null - SELECT arraySlice(arrayFill(x -> (((x.1) % 2) >= 2), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10) FORMAT Null + SELECT arraySlice(arrayFill(x -> (((x.1) % 2) >= 2), arrayMap(x -> (x, toString(x)), range(10000000))), 1, 10) FORMAT Null SELECT arraySlice(arrayFill(x -> ((x % 2) = 0), range(100000000)), 1, 10) FORMAT Null - SELECT arraySlice(arrayFill(x -> (((x.1) % 2) = 0), arrayMap(x -> (x, toString(x)), range(100000000))), 1, 10) FORMAT Null + SELECT arraySlice(arrayFill(x -> (((x.1) % 2) = 0), arrayMap(x -> (x, toString(x)), range(10000000))), 1, 10) FORMAT Null diff --git a/tests/performance/mingroupby-orderbylimit1.xml b/tests/performance/mingroupby-orderbylimit1.xml index 34cd992558b..306095a7e1c 100644 --- a/tests/performance/mingroupby-orderbylimit1.xml +++ b/tests/performance/mingroupby-orderbylimit1.xml @@ -12,53 +12,68 @@ - aggregationscale + aggregationscale_all + + 1111111 + 111111 + 11111 + + + + aggregationscale_small 111111 11111 + + aggregationscale_big + + 1111111 + 111111 + + - create table mingroupby_orderbylimit1_{aggregationscale}_tuple + create table mingroupby_orderbylimit1_{aggregationscale_all}_tuple Engine=MergeTree order by tuple() AS - select toUInt64( number % {aggregationscale} ) key, toUInt64(1) value + select toUInt64( number % {aggregationscale_all} ) key, toUInt64(1) value from numbers(2000000); - create table mingroupby_orderbylimit1_{aggregationscale}_key_value + create table mingroupby_orderbylimit1_{aggregationscale_all}_key_value Engine=MergeTree order by (key,value) AS - select toUInt64( number % {aggregationscale} ) key, toUInt64(1) value + select toUInt64( number % {aggregationscale_all} ) key, toUInt64(1) value from numbers(2000000); SELECT key, min(value) - FROM mingroupby_orderbylimit1_{aggregationscale}_tuple + FROM mingroupby_orderbylimit1_{aggregationscale_big}_tuple group by key format Null; SELECT key, min(value) - FROM mingroupby_orderbylimit1_{aggregationscale}_key_value + FROM mingroupby_orderbylimit1_{aggregationscale_big}_key_value group by key format Null; SELECT key, value - FROM mingroupby_orderbylimit1_{aggregationscale}_tuple + FROM mingroupby_orderbylimit1_{aggregationscale_small}_tuple order by key, value limit 1 by key format Null; SELECT key, value - FROM mingroupby_orderbylimit1_{aggregationscale}_key_value + FROM mingroupby_orderbylimit1_{aggregationscale_small}_key_value order by key, value limit 1 by key format Null; - DROP TABLE IF EXISTS mingroupby_orderbylimit1_{aggregationscale}_tuple - DROP TABLE IF EXISTS mingroupby_orderbylimit1_{aggregationscale}_key_value + DROP TABLE IF EXISTS mingroupby_orderbylimit1_{aggregationscale_all}_tuple + DROP TABLE IF EXISTS mingroupby_orderbylimit1_{aggregationscale_all}_key_value diff --git a/tests/performance/simple_join_query.xml b/tests/performance/simple_join_query.xml index 8f62ffdfd00..92fdfd23f93 100644 --- a/tests/performance/simple_join_query.xml +++ b/tests/performance/simple_join_query.xml @@ -12,7 +12,7 @@ CREATE TABLE join_table(A Int64, S0 String, S1 String, S2 String, S3 String) ENGINE = MergeTree ORDER BY A - INSERT INTO join_table SELECT number AS A, toString(arrayMap(x->x, range(100))) S0, S0 AS S1, S0 AS S2, S0 AS S3 from numbers(500000) + INSERT INTO join_table SELECT number AS A, toString(arrayMap(x->x, range(100))) S0, S0 AS S1, S0 AS S2, S0 AS S3 from numbers_mt(5000000) SELECT COUNT() FROM join_table LEFT JOIN join_table USING A SELECT COUNT() FROM join_table LEFT JOIN (SELECT A FROM join_table) AS right USING A From ae52067c308d16398fc113a75481951f77c5495a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 17 Apr 2020 00:16:45 +0300 Subject: [PATCH 313/743] Update docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md Co-Authored-By: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> --- .../nested_data_structures/simpleaggregatefunction.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md b/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md index 4fd4c37254b..e172986745d 100644 --- a/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md +++ b/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md @@ -2,9 +2,9 @@ Unlike [`AggregateFunction`](../aggregatefunction.md), which stores not the value of the aggregate function but it's state: -- `SimpleAggregateFunction` data type stores current value of aggregation and does not store aggregation state as [`AggregateFunction`](../aggregatefunction.md). -. -It supports simple stateless aggregate functions, including: +- `SimpleAggregateFunction` data type stores current value of the aggregate function, and does not store its full state as [`AggregateFunction`](../aggregatefunction.md) does. This optimization can be applied to functions for which the following property holds: the result of applying a function `f` to a row set `S1 UNION ALL S2` can be obtained by applying `f` to parts of the row set separately, and then again applying `f` to the results: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. This property guarantees that partial aggregation results are enough to compute the combined one, so we don't have to store and process any extra data. + +Currently, the following aggregate functions are supported: - [`any`](../../query_language/agg_functions/reference.md#agg_function-any) - [`anyLast`](../../query_language/agg_functions/reference.md#anylastx) From aef97164062d6358c053b040786f49c69513e628 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 17 Apr 2020 00:55:34 +0300 Subject: [PATCH 314/743] Update index.md --- docs/zh/index.md | 5 ----- 1 file changed, 5 deletions(-) diff --git a/docs/zh/index.md b/docs/zh/index.md index 455a578456b..c6cc5069b14 100644 --- a/docs/zh/index.md +++ b/docs/zh/index.md @@ -1,8 +1,3 @@ ---- -machine_translated: true -machine_translated_rev: b111334d6614a02564cf32f379679e9ff970d9b1 ---- - # 什么是ClickHouse? {#shi-yao-shi-clickhouse} ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS)。 From 9b66f33eb5ddd028d8cc134ef3a26c1cd84d53fc Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 17 Apr 2020 01:25:08 +0300 Subject: [PATCH 315/743] performance comparison --- docker/test/performance-comparison/entrypoint.sh | 3 --- docker/test/performance-comparison/perf.py | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index dc0480715d0..b125d624bfe 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -87,9 +87,6 @@ git -C ch diff --name-only "$SHA_TO_TEST" "$(git -C ch merge-base "$SHA_TO_TEST" # Set python output encoding so that we can print queries with Russian letters. export PYTHONIOENCODING=utf-8 -# Use a default number of runs if not told otherwise -export CHPC_RUNS=${CHPC_RUNS:-7} - # By default, use the main comparison script from the tested package, so that we # can change it in PRs. script_path="right/scripts" diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 55d93f89c6e..9be84fdc4b6 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -25,7 +25,7 @@ parser = argparse.ArgumentParser(description='Run performance test.') parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file') parser.add_argument('--host', nargs='*', default=['localhost'], help="Server hostname(s). Corresponds to '--port' options.") parser.add_argument('--port', nargs='*', default=[9000], help="Server port(s). Corresponds to '--host' options.") -parser.add_argument('--runs', type=int, default=int(os.environ.get('CHPC_RUNS', 7)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') +parser.add_argument('--runs', type=int, default=int(os.environ.get('CHPC_RUNS', 11)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') parser.add_argument('--no-long', type=bool, default=True, help='Skip the tests tagged as long.') args = parser.parse_args() From 1df5c7cedf9e752911fb952c32588ca1c5bcd48d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Apr 2020 01:28:08 +0300 Subject: [PATCH 316/743] Added generic variants of least and greatest functions #4767 --- src/Columns/IColumn.h | 2 +- src/Functions/LeastGreatestGeneric.h | 136 +++++++++++++++++++++++++++ src/Functions/greatest.cpp | 4 +- src/Functions/least.cpp | 4 +- 4 files changed, 143 insertions(+), 3 deletions(-) create mode 100644 src/Functions/LeastGreatestGeneric.h diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 090537d6770..4af593bb658 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -44,7 +44,7 @@ public: /// Name of a Column kind, without parameters (example: FixedString, Array). virtual const char * getFamilyName() const = 0; - /** If column isn't constant, returns nullptr (or itself). + /** If column isn't constant, returns itself. * If column is constant, transforms constant to full column (if column type allows such transform) and return it. */ virtual Ptr convertToFullColumnIfConst() const { return getPtr(); } diff --git a/src/Functions/LeastGreatestGeneric.h b/src/Functions/LeastGreatestGeneric.h new file mode 100644 index 00000000000..8e7eb555b3e --- /dev/null +++ b/src/Functions/LeastGreatestGeneric.h @@ -0,0 +1,136 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +enum class LeastGreatest +{ + Least, + Greatest +}; + + +template +class FunctionLeastGreatestGeneric : public IFunction +{ +public: + static constexpr auto name = kind == LeastGreatest::Least ? "least" : "greatest"; + static FunctionPtr create(const Context &) { return std::make_shared>(); } + +private: + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & types) const override + { + if (types.empty()) + throw Exception("Function " + getName() + " cannot be called without arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return getLeastSupertype(types); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + size_t num_arguments = arguments.size(); + if (1 == num_arguments) + { + block.getByPosition(result).column = block.getByPosition(arguments[0]).column; + return; + } + + auto result_type = block.getByPosition(result).type; + + Columns converted_columns(num_arguments); + for (size_t arg = 0; arg < num_arguments; ++arg) + converted_columns[arg] = castColumn(block.getByPosition(arguments[arg]), result_type)->convertToFullColumnIfConst(); + + auto result_column = result_type->createColumn(); + result_column->reserve(input_rows_count); + + for (size_t row_num = 0; row_num < input_rows_count; ++row_num) + { + size_t best_arg = 0; + for (size_t arg = 1; arg < num_arguments; ++arg) + { + auto cmp_result = converted_columns[arg]->compareAt(row_num, row_num, *converted_columns[best_arg], 1); + + if constexpr (kind == LeastGreatest::Least) + { + if (cmp_result < 0) + best_arg = arg; + } + else + { + if (cmp_result > 0) + best_arg = arg; + } + } + + result_column->insertFrom(*converted_columns[best_arg], row_num); + } + + block.getByPosition(result).column = std::move(result_column); + } +}; + + +template +class LeastGreatestOverloadResolver : public IFunctionOverloadResolverImpl +{ +public: + static constexpr auto name = kind == LeastGreatest::Least ? "least" : "greatest"; + + static FunctionOverloadResolverImplPtr create(const Context & context) + { + return std::make_unique>(context); + } + + explicit LeastGreatestOverloadResolver(const Context & context_) : context(context_) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + + FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + DataTypes argument_types; + + /// More efficient specialization for two numeric arguments. + if (arguments.size() == 2 && isNumber(arguments[0].type) && isNumber(arguments[1].type)) + return std::make_unique(SpecializedFunction::create(context), argument_types, return_type); + + return std::make_unique( + FunctionLeastGreatestGeneric::create(context), argument_types, return_type); + } + + DataTypePtr getReturnType(const DataTypes & types) const override + { + if (types.empty()) + throw Exception("Function " + getName() + " cannot be called without arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return getLeastSupertype(types); + } + +private: + const Context & context; +}; + +} + + diff --git a/src/Functions/greatest.cpp b/src/Functions/greatest.cpp index 9abf85e751b..63f08d0affe 100644 --- a/src/Functions/greatest.cpp +++ b/src/Functions/greatest.cpp @@ -1,6 +1,8 @@ #include #include #include +#include + namespace DB { @@ -57,7 +59,7 @@ using FunctionGreatest = FunctionBinaryArithmetic; void registerFunctionGreatest(FunctionFactory & factory) { - factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction>(FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/least.cpp b/src/Functions/least.cpp index f2e7c1f15d2..ba87e4bd7e4 100644 --- a/src/Functions/least.cpp +++ b/src/Functions/least.cpp @@ -1,6 +1,8 @@ #include #include #include +#include + namespace DB { @@ -57,7 +59,7 @@ using FunctionLeast = FunctionBinaryArithmetic; void registerFunctionLeast(FunctionFactory & factory) { - factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction>(FunctionFactory::CaseInsensitive); } } From 5b5de975e1758374ac071003a505f934419cf92b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Apr 2020 01:38:27 +0300 Subject: [PATCH 317/743] Added a test #4767 --- .../01246_least_greatest_generic.reference | 22 ++++++++++++ .../01246_least_greatest_generic.sql | 34 +++++++++++++++++++ 2 files changed, 56 insertions(+) create mode 100644 tests/queries/0_stateless/01246_least_greatest_generic.reference create mode 100644 tests/queries/0_stateless/01246_least_greatest_generic.sql diff --git a/tests/queries/0_stateless/01246_least_greatest_generic.reference b/tests/queries/0_stateless/01246_least_greatest_generic.reference new file mode 100644 index 00000000000..24c2233eed2 --- /dev/null +++ b/tests/queries/0_stateless/01246_least_greatest_generic.reference @@ -0,0 +1,22 @@ +hello +world + +z +hello +world +1 +\N +\N +nan +inf +-0 +123 +-1 +4294967295 +['world'] +[[[]]] +[[[],[]]] +[] +[NULL] +[0] +[NULL] diff --git a/tests/queries/0_stateless/01246_least_greatest_generic.sql b/tests/queries/0_stateless/01246_least_greatest_generic.sql new file mode 100644 index 00000000000..74d095a66e7 --- /dev/null +++ b/tests/queries/0_stateless/01246_least_greatest_generic.sql @@ -0,0 +1,34 @@ +SELECT least('hello', 'world'); +SELECT greatest('hello', 'world'); +SELECT least('hello', 'world', ''); +SELECT greatest('hello', 'world', 'z'); + +SELECT least('hello'); +SELECT greatest('world'); + +SELECT least(1, inf, nan); +SELECT least(1, inf, nan, NULL); +SELECT greatest(1, inf, nan, NULL); +SELECT greatest(1, inf, nan); +SELECT greatest(1, inf); + +SELECT least(0., -0.); +SELECT least(toNullable(123), 456); + +SELECT LEAST(-1, 18446744073709551615); -- { serverError 386 } +SELECT LEAST(-1., 18446744073709551615); -- { serverError 386 } +SELECT LEAST(-1., 18446744073709551615.); +SELECT greatest(-1, 1, 4294967295); + +SELECT greatest([], ['hello'], ['world']); + +SELECT least([[[], []]], [[[]]], [[[]], [[]]]); +SELECT greatest([[[], []]], [[[]]], [[[]], [[]]]); + +SELECT least([], [NULL]); +SELECT greatest([], [NULL]); + +SELECT LEAST([NULL], [0]); +SELECT GREATEST([NULL], [0]); + +SELECT Greatest(); -- { serverError 42 } From 67790a74e54f7af93a55375aa978683194011241 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Apr 2020 06:13:21 +0300 Subject: [PATCH 318/743] Fix test --- src/Functions/LeastGreatestGeneric.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/LeastGreatestGeneric.h b/src/Functions/LeastGreatestGeneric.h index 8e7eb555b3e..2d6d71b20c7 100644 --- a/src/Functions/LeastGreatestGeneric.h +++ b/src/Functions/LeastGreatestGeneric.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -124,6 +125,9 @@ public: if (types.empty()) throw Exception("Function " + getName() + " cannot be called without arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (types.size() == 2 && isNumber(types[0]) && isNumber(types[1])) + return SpecializedFunction::create(context)->getReturnTypeImpl(types); + return getLeastSupertype(types); } From fdc5688d2d955da65628c5df0c6c57b88389af00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Apr 2020 07:09:41 +0300 Subject: [PATCH 319/743] Added memory usage to AsynchronousMetrics --- src/Interpreters/AsynchronousMetrics.cpp | 20 +++++++++++++++++++ .../MergeTreeSequentialBlockInputStream.cpp | 2 ++ 2 files changed, 22 insertions(+) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 304db048f4f..5aa59bd901a 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -9,9 +9,12 @@ #include #include #include +#include +#include #include #include + #if !defined(ARCADIA_BUILD) # include "config_core.h" #endif @@ -130,6 +133,23 @@ void AsynchronousMetrics::update() set("Uptime", context.getUptimeSeconds()); + /// Process memory usage according to OS +#if defined(OS_LINUX) + { + char buf[1024]; + ReadBufferFromFile in("/proc/self/statm", 1024, -1, buf); + size_t memory_virtual = 0; + size_t memory_resident = 0; + readIntText(memory_virtual, in); + skipWhitespaceIfAny(in); + readIntText(memory_resident, in); + + static constexpr size_t PAGE_SIZE = 4096; + set("MemoryVirtual", memory_virtual * PAGE_SIZE); + set("MemoryResident", memory_resident * PAGE_SIZE); + } +#endif + { auto databases = DatabaseCatalog::instance().getDatabases(); diff --git a/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp index 9e0737810df..ed5eb5b4d2e 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp @@ -27,6 +27,8 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( message << "Reading " << data_part->getMarksCount() << " marks from part " << data_part->name << ", total " << data_part->rows_count << " rows starting from the beginning of the part"; + if (columns_to_read.size() == 1) /// Print column name but don't pollute logs in case of many columns. + message << ", column " << columns_to_read.front(); LOG_TRACE(log, message.rdbuf()); } From 22bc75d63187d88d46a975ba39852686750d654d Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Fri, 17 Apr 2020 07:46:34 +0000 Subject: [PATCH 320/743] Bump urllib3 from 1.25.8 to 1.25.9 in /docs/tools Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.25.8 to 1.25.9. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/master/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/1.25.8...1.25.9) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- docs/tools/translate/requirements.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 8414ae2c533..e6265b5e9e2 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -35,4 +35,4 @@ soupsieve==2.0 termcolor==1.1.0 tornado==5.1.1 Unidecode==1.1.1 -urllib3==1.25.8 +urllib3==1.25.9 diff --git a/docs/tools/translate/requirements.txt b/docs/tools/translate/requirements.txt index b0ea9603555..3c212ee8bc2 100644 --- a/docs/tools/translate/requirements.txt +++ b/docs/tools/translate/requirements.txt @@ -9,4 +9,4 @@ python-slugify==4.0.0 PyYAML==5.3.1 requests==2.23.0 text-unidecode==1.3 -urllib3==1.25.8 +urllib3==1.25.9 From f2136bc2861db318d0a5dd1ac5444e59e69c992d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 16 Apr 2020 12:01:19 +0300 Subject: [PATCH 321/743] improvements --- tests/integration/pytest.ini | 1 + utils/junit_to_html/junit_to_html | 56 ++++++++++++++++++++++++++----- 2 files changed, 49 insertions(+), 8 deletions(-) diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index bff275e3188..a7ca8c57da8 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -3,3 +3,4 @@ python_files = test*.py norecursedirs = _instances timeout = 300 junit_duration_report = call +junit_suite_name = integration diff --git a/utils/junit_to_html/junit_to_html b/utils/junit_to_html/junit_to_html index d6bebccbf9f..1fd63b5fc46 100755 --- a/utils/junit_to_html/junit_to_html +++ b/utils/junit_to_html/junit_to_html @@ -3,22 +3,62 @@ import os import sys import lxml.etree as etree +import json -def _convert_junit_to_html(junit_path, html_path): +def export_testcases_json(report, path): + with open(os.path.join(path, "cases.jer"), "w") as testcases_file: + for testsuite in report.getroot(): + for testcase in testsuite: + row = {} + row["hostname"] = testsuite.get("hostname") + row["hostname"] = testsuite.get("hostname") + row["timestamp"] = testsuite.get("timestamp") + row["testname"] = testcase.get("name") + row["classname"] = testcase.get("classname") + row["file"] = testcase.get("file") + row["line"] = testcase.get("line") + row["duration"] = testcase.get("time") + for el in testcase: + if el.tag == "system-err": + row["stderr"] = el.text + if el.tag == "system-out": + row["stdout"] = el.text + json.dump(row, testcases_file) + +def export_testsuites_json(report, path): + with open(os.path.join(path, "suites.jer"), "w") as testsuites_file: + for testsuite in report.getroot(): + row = {} + row["name"] = testsuite.get("name") + row["errors"] = testsuite.get("errors") + row["failures"] = testsuite.get("failures") + row["hostname"] = testsuite.get("hostname") + row["skipped"] = testsuite.get("skipped") + row["duration"] = testsuite.get("time") + row["timestamp"] = testsuite.get("timestamp") + json.dump(row, testsuites_file) + + +def _convert_junit_to_html(junit_path, result_path): with open(os.path.join(os.path.dirname(__file__), "junit-noframes.xsl")) as xslt_file: junit_to_html_xslt = etree.parse(xslt_file) + if not os.path.exists(result_path): + os.makedirs(result_path) + with open(junit_path) as junit_file: junit_xml = etree.parse(junit_file) + + export_testsuites_json(junit_xml, result_path) + export_testcases_json(junit_xml, result_path) transform = etree.XSLT(junit_to_html_xslt) html = etree.tostring(transform(junit_xml), encoding="utf-8") - html_dir = os.path.dirname(html_path) - if not os.path.exists(html_dir): - os.makedirs(html_dir) - with open(html_path, "w") as html_file: + + with open(os.path.join(result_path, "result.html"), "w") as html_file: html_file.write(html) if __name__ == "__main__": if len(sys.argv) < 3: - raise "Insufficient arguments: junit.xml result.html", level - junit_path, html_path = sys.argv[1] , sys.argv[2] - _convert_junit_to_html(junit_path, html_path) + raise "Insufficient arguments: junit.xml result_path ", level + junit_path, result_path = sys.argv[1] , sys.argv[2] + print "junit_path: {}, result_path: {}".format(junit_path, result_path) + _convert_junit_to_html(junit_path, result_path) From a312cb6b0de66fe04b9f141cc19b8cba1c8682a4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 16 Apr 2020 18:04:31 +0300 Subject: [PATCH 322/743] more features --- utils/junit_to_html/junit_to_html | 38 ++++++++++++++++++++++--------- 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/utils/junit_to_html/junit_to_html b/utils/junit_to_html/junit_to_html index 1fd63b5fc46..685a0d2bddc 100755 --- a/utils/junit_to_html/junit_to_html +++ b/utils/junit_to_html/junit_to_html @@ -1,9 +1,9 @@ #!/usr/bin/env python # -*- coding: utf-8 -*- import os -import sys import lxml.etree as etree import json +import argparse def export_testcases_json(report, path): with open(os.path.join(path, "cases.jer"), "w") as testcases_file: @@ -11,7 +11,8 @@ def export_testcases_json(report, path): for testcase in testsuite: row = {} row["hostname"] = testsuite.get("hostname") - row["hostname"] = testsuite.get("hostname") + row["suite"] = testsuite.get("name") + row["suite_duration"] = testsuite.get("time") row["timestamp"] = testsuite.get("timestamp") row["testname"] = testcase.get("name") row["classname"] = testcase.get("classname") @@ -24,12 +25,13 @@ def export_testcases_json(report, path): if el.tag == "system-out": row["stdout"] = el.text json.dump(row, testcases_file) + testcases_file.write("\n") def export_testsuites_json(report, path): with open(os.path.join(path, "suites.jer"), "w") as testsuites_file: for testsuite in report.getroot(): row = {} - row["name"] = testsuite.get("name") + row["suite"] = testsuite.get("name") row["errors"] = testsuite.get("errors") row["failures"] = testsuite.get("failures") row["hostname"] = testsuite.get("hostname") @@ -37,9 +39,10 @@ def export_testsuites_json(report, path): row["duration"] = testsuite.get("time") row["timestamp"] = testsuite.get("timestamp") json.dump(row, testsuites_file) + testsuites_file.write("\n") -def _convert_junit_to_html(junit_path, result_path): +def _convert_junit_to_html(junit_path, result_path, export_cases, export_suites): with open(os.path.join(os.path.dirname(__file__), "junit-noframes.xsl")) as xslt_file: junit_to_html_xslt = etree.parse(xslt_file) if not os.path.exists(result_path): @@ -48,8 +51,10 @@ def _convert_junit_to_html(junit_path, result_path): with open(junit_path) as junit_file: junit_xml = etree.parse(junit_file) - export_testsuites_json(junit_xml, result_path) - export_testcases_json(junit_xml, result_path) + if export_suites: + export_testsuites_json(junit_xml, result_path) + if export_cases: + export_testcases_json(junit_xml, result_path) transform = etree.XSLT(junit_to_html_xslt) html = etree.tostring(transform(junit_xml), encoding="utf-8") @@ -57,8 +62,19 @@ def _convert_junit_to_html(junit_path, result_path): html_file.write(html) if __name__ == "__main__": - if len(sys.argv) < 3: - raise "Insufficient arguments: junit.xml result_path ", level - junit_path, result_path = sys.argv[1] , sys.argv[2] - print "junit_path: {}, result_path: {}".format(junit_path, result_path) - _convert_junit_to_html(junit_path, result_path) + + parser = argparse.ArgumentParser(description='Convert JUnit XML.') + parser.add_argument('junit', help='path to junit.xml report') + parser.add_argument('result_dir', nargs='?', help='directory for result files. Default to junit.xml directory') + parser.add_argument('--export-cases', help='Export JSONEachRow result for testcases to upload in CI', action='store_true') + parser.add_argument('--export-suites', help='Export JSONEachRow result for testsuites to upload in CI', action='store_true') + + args = parser.parse_args() + + junit_path = args.junit + if args.result_dir: + result_path = args.result_dir + else: + result_path = os.path.dirname(junit_path) + print "junit_path: {}, result_path: {}, export cases:{}, export suites: {}".format(junit_path, result_path, args.export_cases, args.export_suites) + _convert_junit_to_html(junit_path, result_path, args.export_cases, args.export_suites) From ee8e855cbe0587eb75d54a8e6983ce20b9a0fd7b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 17 Apr 2020 11:41:57 +0300 Subject: [PATCH 323/743] add empty out and err columns --- utils/junit_to_html/junit_to_html | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/utils/junit_to_html/junit_to_html b/utils/junit_to_html/junit_to_html index 685a0d2bddc..cf50e7df00a 100755 --- a/utils/junit_to_html/junit_to_html +++ b/utils/junit_to_html/junit_to_html @@ -22,8 +22,14 @@ def export_testcases_json(report, path): for el in testcase: if el.tag == "system-err": row["stderr"] = el.text + else: + row["stderr"] = "" + if el.tag == "system-out": row["stdout"] = el.text + else: + row["stdout"] = "" + json.dump(row, testcases_file) testcases_file.write("\n") From b056dbce1ceb16f41c66f55baf2d49252f87d045 Mon Sep 17 00:00:00 2001 From: Avogar Date: Fri, 17 Apr 2020 12:35:38 +0300 Subject: [PATCH 324/743] Improve MsgPackRowInputFormat --- .../Formats/Impl/MsgPackRowInputFormat.cpp | 233 +++++++++--------- .../Formats/Impl/MsgPackRowInputFormat.h | 40 ++- tests/performance/select_format.xml | 2 +- 3 files changed, 153 insertions(+), 122 deletions(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index b7da335f0c5..0748af68523 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -24,14 +24,124 @@ namespace ErrorCodes } MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) - : IRowInputFormat(header_, in_, std::move(params_)), buf(in), ctx(&reference_func, nullptr, msgpack::unpack_limit()), data_types(header_.getDataTypes()) {} + : IRowInputFormat(header_, in_, std::move(params_)), buf(in), parser(visitor), data_types(header_.getDataTypes()) {} -int MsgPackRowInputFormat::unpack(msgpack::zone & zone, size_t & offset) +void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type) { - offset = 0; - ctx.init(); - ctx.user().set_zone(zone); - return ctx.execute(buf.position(), buf.buffer().end() - buf.position(), offset); + while (!info_stack.empty()) + { + info_stack.pop(); + } + info_stack.push(Info{column, type}); +} + +void MsgPackVisitor::insert_integer(UInt64 value) +{ + Info & info = info_stack.top(); + switch (info.type->getTypeId()) + { + case TypeIndex::UInt8: + { + assert_cast(info.column).insertValue(value); + break; + } + case TypeIndex::Date: [[fallthrough]]; + case TypeIndex::UInt16: + { + assert_cast(info.column).insertValue(value); + break; + } + case TypeIndex::DateTime: [[fallthrough]]; + case TypeIndex::UInt32: + { + assert_cast(info.column).insertValue(value); + break; + } + case TypeIndex::UInt64: + { + assert_cast(info.column).insertValue(value); + break; + } + case TypeIndex::Int8: + { + assert_cast(info.column).insertValue(value); + break; + } + case TypeIndex::Int16: + { + assert_cast(info.column).insertValue(value); + break; + } + case TypeIndex::Int32: + { + assert_cast(info.column).insertValue(value); + break; + } + case TypeIndex::Int64: + { + assert_cast(info.column).insertValue(value); + break; + } + case TypeIndex::DateTime64: + { + assert_cast(info.column).insertValue(value); + break; + } + default: + throw Exception("Type " + info.type->getName() + " is not supported for MsgPack input format", ErrorCodes::ILLEGAL_COLUMN); + } +} + +bool MsgPackVisitor::visit_positive_integer(UInt64 value) +{ + insert_integer(value); + return true; +} + +bool MsgPackVisitor::visit_negative_integer(Int64 value) +{ + insert_integer(value); + return true; +} + +bool MsgPackVisitor::visit_str(const char* value, size_t size) +{ + info_stack.top().column.insertData(value, size); + return true; +} + +bool MsgPackVisitor::visit_float32(Float32 value) +{ + assert_cast(info_stack.top().column).insertValue(value); + return true; +} + +bool MsgPackVisitor::visit_float64(Float64 value) +{ + assert_cast(info_stack.top().column).insertValue(value); + return true; +} + +bool MsgPackVisitor::start_array(size_t size) +{ + auto nested_type = assert_cast(*info_stack.top().type).getNestedType(); + ColumnArray & column_array = assert_cast(info_stack.top().column); + ColumnArray::Offsets & offsets = column_array.getOffsets(); + IColumn & nested_column = column_array.getData(); + offsets.push_back(offsets.back() + size); + info_stack.push(Info{nested_column, nested_type}); + return true; +} + +bool MsgPackVisitor::end_array() +{ + info_stack.pop(); + return true; +} + +void MsgPackVisitor::parse_error(size_t, size_t) +{ + throw Exception("Error occurred while parsing msgpack data.", ErrorCodes::INCORRECT_DATA); } bool MsgPackRowInputFormat::readObject() @@ -40,9 +150,8 @@ bool MsgPackRowInputFormat::readObject() return false; PeekableReadBufferCheckpoint checkpoint{buf}; - std::unique_ptr zone(new msgpack::zone); - size_t offset; - while (!unpack(*zone, offset)) + size_t offset = 0; + while (!parser.execute(buf.position(), buf.available(), offset)) { buf.position() = buf.buffer().end(); if (buf.eof()) @@ -52,123 +161,19 @@ bool MsgPackRowInputFormat::readObject() buf.rollbackToCheckpoint(); } buf.position() += offset; - object_handle = msgpack::object_handle(ctx.data(), std::move(zone)); return true; } -void MsgPackRowInputFormat::insertObject(IColumn & column, DataTypePtr data_type, const msgpack::object & object) -{ - switch (data_type->getTypeId()) - { - case TypeIndex::UInt8: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::Date: [[fallthrough]]; - case TypeIndex::UInt16: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::DateTime: [[fallthrough]]; - case TypeIndex::UInt32: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::UInt64: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::Int8: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::Int16: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::Int32: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::Int64: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::Float32: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::Float64: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::DateTime64: - { - assert_cast(column).insertValue(object.as()); - return; - } - case TypeIndex::FixedString: [[fallthrough]]; - case TypeIndex::String: - { - msgpack::object_str obj_str = object.via.str; - column.insertData(obj_str.ptr, obj_str.size); - return; - } - case TypeIndex::Array: - { - msgpack::object_array object_array = object.via.array; - auto nested_type = assert_cast(*data_type).getNestedType(); - ColumnArray & column_array = assert_cast(column); - ColumnArray::Offsets & offsets = column_array.getOffsets(); - IColumn & nested_column = column_array.getData(); - for (size_t i = 0; i != object_array.size; ++i) - { - insertObject(nested_column, nested_type, object_array.ptr[i]); - } - offsets.push_back(offsets.back() + object_array.size); - return; - } - case TypeIndex::Nullable: - { - auto nested_type = removeNullable(data_type); - ColumnNullable & column_nullable = assert_cast(column); - if (object.type == msgpack::type::NIL) - column_nullable.insertDefault(); - else - insertObject(column_nullable.getNestedColumn(), nested_type, object); - return; - } - case TypeIndex::Nothing: - { - // Nothing to insert, MsgPack object is nil. - return; - } - default: - break; - } - throw Exception("Type " + data_type->getName() + " is not supported for MsgPack input format", ErrorCodes::ILLEGAL_COLUMN); -} - bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { size_t column_index = 0; bool has_more_data = true; for (; column_index != columns.size(); ++column_index) { + visitor.set_info(*columns[column_index], data_types[column_index]); has_more_data = readObject(); if (!has_more_data) break; - insertObject(*columns[column_index], data_types[column_index], object_handle.get()); } if (!has_more_data) { diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index a426dc4950c..0a6f4ffaaf5 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -4,12 +4,43 @@ #include #include #include +#include namespace DB { class ReadBuffer; +class MsgPackVisitor : public msgpack::null_visitor { +public: + struct Info + { + IColumn & column; + DataTypePtr type; + }; + + /// These functions are called when parser meets corresponding object in parsed data + bool visit_positive_integer(UInt64 value); + bool visit_negative_integer(Int64 value); + bool visit_float32(Float32 value); + bool visit_float64(Float64 value); + bool visit_str(const char* value, size_t size); + bool start_array(size_t size); + bool end_array(); + + /// This function will be called if error occurs in parsing + void parse_error(size_t parsed_offset, size_t error_offset); + + /// Update info_stack + void set_info(IColumn & column, DataTypePtr type); + + void insert_integer(UInt64 value); + +private: + /// Stack is needed to process nested arrays + std::stack info_stack; +}; + class MsgPackRowInputFormat : public IRowInputFormat { public: @@ -19,15 +50,10 @@ public: String getName() const override { return "MagPackRowInputFormat"; } private: bool readObject(); - void insertObject(IColumn & column, DataTypePtr type, const msgpack::object & object); - int unpack(msgpack::zone & zone, size_t & offset); - - // msgpack makes a copy of object by default, this function tells unpacker not to copy. - static bool reference_func(msgpack::type::object_type, size_t, void *) { return true; } PeekableReadBuffer buf; - msgpack::object_handle object_handle; - msgpack::v1::detail::context ctx; + MsgPackVisitor visitor; + msgpack::detail::parse_helper parser; DataTypes data_types; }; diff --git a/tests/performance/select_format.xml b/tests/performance/select_format.xml index 2bdbde83c2d..e47d981c4d7 100644 --- a/tests/performance/select_format.xml +++ b/tests/performance/select_format.xml @@ -44,7 +44,7 @@ ODBCDriver2 MySQLWire Avro - + MsgPack From d51a9c551bd860bc1c8e5d7a5abfc000bdeb7121 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 12:47:40 +0300 Subject: [PATCH 325/743] Remove shared_ptr to SharedContext from Context. --- programs/benchmark/Benchmark.cpp | 4 +++- programs/client/Client.cpp | 3 ++- programs/copier/ClusterCopierApp.cpp | 3 ++- programs/local/LocalServer.cpp | 3 ++- programs/obfuscator/Obfuscator.cpp | 3 ++- programs/odbc-bridge/HandlerFactory.h | 4 ++-- programs/odbc-bridge/ODBCBridge.cpp | 5 +++-- programs/odbc-bridge/ODBCBridge.h | 2 -- programs/server/Server.cpp | 8 +++++++- programs/server/Server.h | 2 +- src/AggregateFunctions/IAggregateFunction.h | 4 ++++ src/Common/tests/gtest_global_context.h | 3 ++- .../tests/collapsing_sorted_stream.cpp | 3 ++- src/DataStreams/tests/expression_stream.cpp | 3 ++- src/DataStreams/tests/filter_stream.cpp | 3 ++- src/DataStreams/tests/union_stream2.cpp | 3 ++- src/Interpreters/Context.cpp | 16 ++++------------ src/Interpreters/Context.h | 15 +++++++++------ src/Interpreters/tests/create_query.cpp | 1 + src/Interpreters/tests/expression.cpp | 3 ++- src/Interpreters/tests/expression_analyzer.cpp | 3 ++- .../tests/in_join_subqueries_preprocessor.cpp | 3 ++- src/Interpreters/tests/select_query.cpp | 3 ++- src/Storages/tests/storage_log.cpp | 3 ++- src/Storages/tests/system_numbers.cpp | 3 ++- 25 files changed, 64 insertions(+), 42 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 624712504b7..ce59f5cac7f 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -64,7 +64,8 @@ public: concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_), cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_), json_path(json_path_), confidence(confidence_), query_id(query_id_), settings(settings_), - global_context(Context::createGlobal()), pool(concurrency) + shared_context(Context::createShared()), global_context(Context::createGlobal(shared_context.get())), + pool(concurrency) { const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable; size_t connections_cnt = std::max(ports_.size(), hosts_.size()); @@ -149,6 +150,7 @@ private: size_t confidence; std::string query_id; Settings settings; + SharedContextHolder shared_context; Context global_context; QueryProcessingStage::Enum query_processing_stage; diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e01eef98006..cf820334e69 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -138,7 +138,8 @@ private: bool has_vertical_output_suffix = false; /// Is \G present at the end of the query string? - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); /// Buffer that reads from stdin in batch mode. ReadBufferFromFileDescriptor std_in {STDIN_FILENO}; diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 14bfd3a3339..b70caf5f333 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -94,7 +94,8 @@ void ClusterCopierApp::mainImpl() << "path " << process_path << ", " << "revision " << ClickHouseRevision::get() << ")"); - auto context = std::make_unique(Context::createGlobal()); + SharedContextHolder shared_context = Context::createShared(); + auto context = std::make_unique(Context::createGlobal(shared_context.get())); context->makeGlobalContext(); SCOPE_EXIT(context->shutdown()); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 26752da5d87..ce4c608c310 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -148,7 +148,8 @@ try return Application::EXIT_OK; } - context = std::make_unique(Context::createGlobal()); + SharedContextHolder shared_context = Context::createShared(); + context = std::make_unique(Context::createGlobal(shared_context.get())); context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); tryInitPath(); diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 0352eba0a0a..9a80dc8d035 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1080,7 +1080,8 @@ try header.insert(std::move(column)); } - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); ReadBufferFromFileDescriptor file_in(STDIN_FILENO); diff --git a/programs/odbc-bridge/HandlerFactory.h b/programs/odbc-bridge/HandlerFactory.h index f1a6803f65b..35835de5dad 100644 --- a/programs/odbc-bridge/HandlerFactory.h +++ b/programs/odbc-bridge/HandlerFactory.h @@ -21,7 +21,7 @@ namespace DB class HandlerFactory : public Poco::Net::HTTPRequestHandlerFactory { public: - HandlerFactory(const std::string & name_, size_t keep_alive_timeout_, std::shared_ptr context_) + HandlerFactory(const std::string & name_, size_t keep_alive_timeout_, Context & context_) : log(&Poco::Logger::get(name_)), name(name_), keep_alive_timeout(keep_alive_timeout_), context(context_) { pool_map = std::make_shared(); @@ -33,7 +33,7 @@ private: Poco::Logger * log; std::string name; size_t keep_alive_timeout; - std::shared_ptr context; + Context & context; std::shared_ptr pool_map; }; } diff --git a/programs/odbc-bridge/ODBCBridge.cpp b/programs/odbc-bridge/ODBCBridge.cpp index e7e8aca7147..1cfba4b3aa8 100644 --- a/programs/odbc-bridge/ODBCBridge.cpp +++ b/programs/odbc-bridge/ODBCBridge.cpp @@ -176,8 +176,9 @@ int ODBCBridge::main(const std::vector & /*args*/) http_params->setTimeout(http_timeout); http_params->setKeepAliveTimeout(keep_alive_timeout); - context = std::make_shared(Context::createGlobal()); - context->makeGlobalContext(); + auto shared_context = Context::createShared(); + Context context(Context::createGlobal(shared_context.get())); + context.makeGlobalContext(); if (config().has("query_masking_rules")) { diff --git a/programs/odbc-bridge/ODBCBridge.h b/programs/odbc-bridge/ODBCBridge.h index 4ae11ad7301..9a0d37fa0f9 100644 --- a/programs/odbc-bridge/ODBCBridge.h +++ b/programs/odbc-bridge/ODBCBridge.h @@ -35,7 +35,5 @@ private: size_t keep_alive_timeout; Poco::Logger * log; - - std::shared_ptr context; /// need for settings only }; } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 20d7fc49fba..1578036ba46 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -228,7 +228,13 @@ int Server::main(const std::vector & /*args*/) * settings, available functions, data types, aggregate functions, databases... */ auto shared_context = Context::createShared(); - global_context = std::make_unique(Context::createGlobal(shared_context.shared.get())); + auto global_context_holder = Context::createGlobal(shared_context.get()); + + /// Global context is owned by Server only. Is is assumed that noboby can access context after server was stopped. + /// To check it, set global_context to nullptr at the and, to get explicit segfault if it is not true. + global_context = &global_context_holder; + SCOPE_EXIT(global_context = nullptr); + global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::SERVER); diff --git a/programs/server/Server.h b/programs/server/Server.h index 337d1551b70..a0233d7e123 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -56,7 +56,7 @@ protected: std::string getDefaultCorePath() const override; private: - std::unique_ptr global_context; + Context * global_context = nullptr; }; } diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 48d31793a5a..4387e05306a 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -97,6 +97,10 @@ public: /// Deserializes state. This function is called only for empty (just created) states. virtual void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const = 0; + /// Finalize state. This function is called once after all 'add' and 'merge' calls. Only is isFinalizationNeeded(). + virtual void finalize(AggregateDataPtr * /*places*/, size_t /*size*/) {} + virtual bool isFinalizationNeeded() const { return false; } + /// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()). virtual bool allocatesMemoryInArena() const { diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 3711ab67932..6f5a3c0e1a1 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -4,7 +4,8 @@ inline DB::Context createContext() { - auto context = DB::Context::createGlobal(); + static DB::SharedContextHolder shared_context = DB::Context::createShared(); + auto context = DB::Context::createGlobal(shared_context.get()); context.makeGlobalContext(); context.setPath("./"); return context; diff --git a/src/DataStreams/tests/collapsing_sorted_stream.cpp b/src/DataStreams/tests/collapsing_sorted_stream.cpp index a34d6a2fbd2..fbcbc0b8489 100644 --- a/src/DataStreams/tests/collapsing_sorted_stream.cpp +++ b/src/DataStreams/tests/collapsing_sorted_stream.cpp @@ -67,7 +67,8 @@ try //CollapsingSortedBlockInputStream collapsed(inputs, descr, "Sign", 1048576); CollapsingFinalBlockInputStream collapsed(inputs, descr, "Sign"); - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); WriteBufferFromFileDescriptor out_buf(STDERR_FILENO); BlockOutputStreamPtr output = context.getOutputFormat("TabSeparated", out_buf, block1); diff --git a/src/DataStreams/tests/expression_stream.cpp b/src/DataStreams/tests/expression_stream.cpp index bd4117f5aab..6c2b8373fc7 100644 --- a/src/DataStreams/tests/expression_stream.cpp +++ b/src/DataStreams/tests/expression_stream.cpp @@ -35,7 +35,8 @@ try ParserSelectQuery parser; ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); NamesAndTypesList source_columns = {{"number", std::make_shared()}}; diff --git a/src/DataStreams/tests/filter_stream.cpp b/src/DataStreams/tests/filter_stream.cpp index 5e324251440..73f583bf650 100644 --- a/src/DataStreams/tests/filter_stream.cpp +++ b/src/DataStreams/tests/filter_stream.cpp @@ -40,7 +40,8 @@ try formatAST(*ast, std::cerr); std::cerr << std::endl; - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); NamesAndTypesList source_columns = {{"number", std::make_shared()}}; diff --git a/src/DataStreams/tests/union_stream2.cpp b/src/DataStreams/tests/union_stream2.cpp index f3778543a6d..6e0997e7f72 100644 --- a/src/DataStreams/tests/union_stream2.cpp +++ b/src/DataStreams/tests/union_stream2.cpp @@ -23,7 +23,8 @@ using namespace DB; int main(int, char **) try { - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); Settings settings = context.getSettings(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c227ebd2f4c..2c123435850 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -441,17 +441,11 @@ Context::Context() = default; Context::Context(const Context &) = default; Context & Context::operator=(const Context &) = default; -SharedContextHolder::SharedContextHolder() = default; -SharedContextHolder::SharedContextHolder(SharedContextHolder &&) = default; +SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; SharedContextHolder::~SharedContextHolder() = default; +SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) + : shared(std::move(shared_context)) {} -Context Context::createGlobal() -{ - Context res; - res.shared_holder = std::make_shared(); - res.shared = res.shared_holder.get(); - return res; -} Context Context::createGlobal(ContextShared * shared) { @@ -462,9 +456,7 @@ Context Context::createGlobal(ContextShared * shared) SharedContextHolder Context::createShared() { - SharedContextHolder holder; - holder.shared = std::make_unique(); - return holder; + return SharedContextHolder(std::make_unique()); } Context::~Context() = default; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 91b2ee3242b..62d76fd19c9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -128,12 +128,18 @@ struct IHostContext using IHostContextPtr = std::shared_ptr; +/// A small class which owns ContextShared. +/// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete. struct SharedContextHolder { - std::unique_ptr shared; ~SharedContextHolder(); - SharedContextHolder(); - SharedContextHolder(SharedContextHolder &&); + SharedContextHolder(std::unique_ptr shared_context); + SharedContextHolder(SharedContextHolder &&) noexcept; + + ContextShared * get() const { return shared.get(); } + +private: + std::unique_ptr shared; }; /** A set of known objects that can be used in the query. @@ -145,8 +151,6 @@ struct SharedContextHolder class Context { private: - using Shared = std::shared_ptr; - Shared shared_holder; ContextShared * shared; ClientInfo client_info; @@ -199,7 +203,6 @@ private: public: /// Create initial Context with ContextShared and etc. - static Context createGlobal(); static Context createGlobal(ContextShared * shared); static SharedContextHolder createShared(); diff --git a/src/Interpreters/tests/create_query.cpp b/src/Interpreters/tests/create_query.cpp index 20a0bfcb062..5818a4367f6 100644 --- a/src/Interpreters/tests/create_query.cpp +++ b/src/Interpreters/tests/create_query.cpp @@ -78,6 +78,7 @@ try ParserCreateQuery parser; ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); + SharedContextHolder shared_context = Context::createShared(shared_context.get()); Context context = Context::createGlobal(); context.makeGlobalContext(); diff --git a/src/Interpreters/tests/expression.cpp b/src/Interpreters/tests/expression.cpp index 8d64b4f64ce..a2d58717d1e 100644 --- a/src/Interpreters/tests/expression.cpp +++ b/src/Interpreters/tests/expression.cpp @@ -46,7 +46,8 @@ int main(int argc, char ** argv) formatAST(*ast, std::cerr); std::cerr << std::endl; - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); NamesAndTypesList columns { diff --git a/src/Interpreters/tests/expression_analyzer.cpp b/src/Interpreters/tests/expression_analyzer.cpp index 2f8b6b2aef2..15a1612f0a6 100644 --- a/src/Interpreters/tests/expression_analyzer.cpp +++ b/src/Interpreters/tests/expression_analyzer.cpp @@ -95,7 +95,8 @@ int main() } }; - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); auto system_database = std::make_shared("system"); diff --git a/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index 9a17f03f32a..c000397d2fe 100644 --- a/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -1156,7 +1156,8 @@ static bool run() TestResult check(const TestEntry & entry) { - static DB::Context context = DB::Context::createGlobal(); + static DB::SharedContextHolder shared_context = DB::Context::createShared(); + static DB::Context context = DB::Context::createGlobal(shared_context.get()); context.makeGlobalContext(); try diff --git a/src/Interpreters/tests/select_query.cpp b/src/Interpreters/tests/select_query.cpp index 3775e5a3532..8ce65ede05f 100644 --- a/src/Interpreters/tests/select_query.cpp +++ b/src/Interpreters/tests/select_query.cpp @@ -30,7 +30,8 @@ try /// Pre-initialize the `DateLUT` so that the first initialization does not affect the measured execution speed. DateLUT::instance(); - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); context.setPath("./"); diff --git a/src/Storages/tests/storage_log.cpp b/src/Storages/tests/storage_log.cpp index 5df1b5764f1..4a7e9b221a0 100644 --- a/src/Storages/tests/storage_log.cpp +++ b/src/Storages/tests/storage_log.cpp @@ -26,7 +26,8 @@ try names_and_types.emplace_back("a", std::make_shared()); names_and_types.emplace_back("b", std::make_shared()); - auto context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + auto context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); context.setPath("./"); diff --git a/src/Storages/tests/system_numbers.cpp b/src/Storages/tests/system_numbers.cpp index 9fc84d76f5c..6955c90b74e 100644 --- a/src/Storages/tests/system_numbers.cpp +++ b/src/Storages/tests/system_numbers.cpp @@ -27,7 +27,8 @@ try WriteBufferFromOStream out_buf(std::cout); - auto context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + auto context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); From 523b0648abf1ae63a9d371816ecf8926b533abff Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Apr 2020 12:47:59 +0300 Subject: [PATCH 326/743] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0c5076021ce..8333b02342e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,6 @@ ## ClickHouse release v20.3 -### ClickHouse release origin/20.3 FIXME as compared to v20.3.5.21-stable +### ClickHouse release v20.3.6.40, 2020-04-16 #### New Feature From 3889e80bd76d8717a6350397df262fbb54f4417e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Apr 2020 12:52:07 +0300 Subject: [PATCH 327/743] Update CHANGELOG.md --- CHANGELOG.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8333b02342e..a16c5f94166 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,14 @@ ## ClickHouse release v20.3 +### ClickHouse release v20.3.7.46, 2020-04-17 + +#### Bug Fix + +* Fix 'Logical error: CROSS JOIN has expressions' error for queries with comma and names joins mix. [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix queries with `max_bytes_before_external_group_by`. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix move-to-prewhere optimization in presense of arrayJoin functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). + ### ClickHouse release v20.3.6.40, 2020-04-16 #### New Feature From 4efd70a0a63cb07e39fc1c1031931bff4c7c62ea Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Apr 2020 12:52:40 +0300 Subject: [PATCH 328/743] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a16c5f94166..c676f094f11 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,7 +4,7 @@ #### Bug Fix -* Fix 'Logical error: CROSS JOIN has expressions' error for queries with comma and names joins mix. [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix `Logical error: CROSS JOIN has expressions` error for queries with comma and names joins mix. [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)). * Fix queries with `max_bytes_before_external_group_by`. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)). * Fix move-to-prewhere optimization in presense of arrayJoin functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). From 52dd4f457688c7809cdc5ea104425419d66b0ac6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Apr 2020 13:03:36 +0300 Subject: [PATCH 329/743] Update CHANGELOG.md --- CHANGELOG.md | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index c676f094f11..b183013c384 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -330,6 +330,36 @@ ## ClickHouse release v20.1 +### ClickHouse release v20.1.10.70, 2020-04-17 + +#### Bug Fix + +* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug where ClickHouse would throw `'Unknown function lambda.'` error message when user tries to run `ALTER UPDATE/DELETE` on tables with `ENGINE = Replicated*`. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). +* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix column names of constants inside `JOIN` that may clash with names of constants outside of `JOIN`. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible inifinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix move-to-prewhere optimization in presense of `arrayJoin` functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). +* Convert blocks if structure does not match on `INSERT` into table with `Distributed` engine. [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)). +* Fix `SIGSEGV` on `INSERT` into `Distributed` table when its structure differs from the underlying tables. [#10105](https://github.com/ClickHouse/ClickHouse/pull/10105) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add arguments check and support identifier arguments for MySQL Database Engine. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)). +* Fix bug in clickhouse dictionary source from localhost clickhouse server. The bug may lead to memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)). +* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with different level from different shards (mixed single and two level aggregation). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a segmentation fault that could occur in `GROUP BY` over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). +* Fix `'Not found column in block'` error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). +* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). +* Fix `'scalar doesn't exist'` error in ALTER queries ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). +* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). +* Fixed `DeleteOnDestroy` logic in `ATTACH PART` which could lead to automatic removal of attached part and added few tests. [#9410](https://github.com/ClickHouse/ClickHouse/pull/9410) ([Vladimir Chebotarev](https://github.com/excitoon)). + +#### Build/Testing/Packaging Improvement + +* Fix unit test `collapsing_sorted_stream`. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)). + ### ClickHouse release v20.1.8.41, 2020-03-20 #### Bug Fix From e94a52178091cecf0c21a7f001970119000c68b7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Apr 2020 13:07:17 +0300 Subject: [PATCH 330/743] Update CHANGELOG.md --- CHANGELOG.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index b183013c384..e5d1c90bf22 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -360,6 +360,25 @@ * Fix unit test `collapsing_sorted_stream`. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)). +### ClickHouse release v20.1.9.54, 2020-03-28 + +#### Bug Fix + +* Fix `'Different expressions with the same alias'` error when query has `PREWHERE` and `WHERE` on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)). +* For INSERT queries shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send `INSERT` queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `SIGSEGV` with `optimize_skip_unused_shards` when type cannot be converted. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)). +* Fixed a few cases when timezone of the function argument wasn't used properly. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)). + +#### Improvement + +* Remove `ORDER BY` stage from mutations because we read from a single ordered part in a single thread. Also add check that the order of rows in mutation is ordered in sorting key order and this order is not violated. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)). + +#### Build/Testing/Packaging Improvement + +* Clean up duplicated linker flags. Make sure the linker won't look up an unexpected symbol. [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos Bird](https://github.com/amosbird)). + ### ClickHouse release v20.1.8.41, 2020-03-20 #### Bug Fix From 42a0ad771f07b670ead56bc8748d90bf8f34acc6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 13:07:44 +0300 Subject: [PATCH 331/743] Remove not relevant code. --- src/AggregateFunctions/IAggregateFunction.h | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 4387e05306a..48d31793a5a 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -97,10 +97,6 @@ public: /// Deserializes state. This function is called only for empty (just created) states. virtual void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const = 0; - /// Finalize state. This function is called once after all 'add' and 'merge' calls. Only is isFinalizationNeeded(). - virtual void finalize(AggregateDataPtr * /*places*/, size_t /*size*/) {} - virtual bool isFinalizationNeeded() const { return false; } - /// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()). virtual bool allocatesMemoryInArena() const { From b83ecbb916e3d61a72ecf65ac527f17b5e5d6fc0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 17 Apr 2020 14:54:29 +0300 Subject: [PATCH 332/743] fix word --- docs/en/sql_reference/statements/create.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql_reference/statements/create.md b/docs/en/sql_reference/statements/create.md index 36dd3aced8d..430bcacbc34 100644 --- a/docs/en/sql_reference/statements/create.md +++ b/docs/en/sql_reference/statements/create.md @@ -179,7 +179,7 @@ CREATE TABLE codec_example ENGINE = MergeTree() ``` -#### Common Purpose Codecs {#create-query-common-purpose-codecs} +#### General Purpose Codecs {#create-query-general-purpose-codecs} Codecs: From f7b7b5eb743c5f3f4f88f80b7f88a37807bf64b2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 17 Apr 2020 14:59:10 +0300 Subject: [PATCH 333/743] remove unused flags --- .../MergeTree/IMergeTreeDataPartWriter.h | 2 +- .../MergeTree/IMergedBlockOutputStream.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 ---- .../MergeTree/MergeTreeDataPartCompact.h | 1 - .../MergeTreeDataPartWriterCompact.cpp | 6 +---- .../MergeTreeDataPartWriterCompact.h | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 13 +---------- .../MergeTree/MergeTreeDataPartWriterWide.h | 2 +- src/Storages/MergeTree/MergeTreeIOSettings.h | 5 +--- .../MergedColumnOnlyOutputStream.cpp | 23 ++++++++++--------- .../MergeTree/MergedColumnOnlyOutputStream.h | 10 ++++---- 11 files changed, 23 insertions(+), 47 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index d18b31edc72..3e3496c88da 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -107,7 +107,7 @@ public: void initSkipIndices(); void initPrimaryIndex(); - virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync = false) = 0; + virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) = 0; void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums); void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index ba04d7fa71b..7b808ef6784 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -25,7 +25,7 @@ public: protected: using SerializationState = IDataType::SerializeBinaryBulkStatePtr; - IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns, bool skip_offsets); + IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); /// Remove all columns marked expired in data_part. Also, clears checksums /// and columns array. Return set of removed files names. diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index c10a6c6dd59..8bc871476ed 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -876,9 +876,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor MergedColumnOnlyOutputStream column_to( new_data_part, column_gathered_stream.getHeader(), - false, compression_codec, - false, /// we don't need to recalc indices here /// because all of them were already recalculated and written /// as key part of vertical merge @@ -1588,9 +1586,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( MergedColumnOnlyOutputStream out( new_data_part, mutation_header, - /* sync = */ false, compression_codec, - /* skip_offsets = */ false, std::vector(indices_to_recalc.begin(), indices_to_recalc.end()), nullptr, source_part->index_granularity, diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 12b2e106284..fa98a2c863f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -20,7 +20,6 @@ class MergeTreeDataPartCompact : public IMergeTreeDataPart public: static constexpr auto DATA_FILE_NAME = "data"; static constexpr auto DATA_FILE_EXTENSION = ".bin"; - static constexpr auto TEMP_FILE_SUFFIX = "_temp"; static constexpr auto DATA_FILE_NAME_WITH_EXTENSION = "data.bin"; MergeTreeDataPartCompact( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index bd507866e9b..e33d4a97cac 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -22,8 +22,6 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( { using DataPart = MergeTreeDataPartCompact; String data_file_name = DataPart::DATA_FILE_NAME; - if (settings.is_writing_temp_files) - data_file_name += DataPart::TEMP_FILE_SUFFIX; stream = std::make_unique( data_file_name, @@ -145,7 +143,7 @@ void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTy column.type->serializeBinaryBulkStateSuffix(serialize_settings, state); } -void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) +void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) { if (columns_buffer.size() != 0) writeBlock(header.cloneWithColumns(columns_buffer.releaseColumns())); @@ -161,8 +159,6 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart: } stream->finalize(); - if (sync) - stream->sync(); stream->addToChecksums(checksums); stream.reset(); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 598a4dd47fb..0aff55588aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -21,7 +21,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; private: /// Write single granule of one column (rows between 2 marks) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 5c1c6e35a20..1e5640b4e23 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -39,9 +39,6 @@ void MergeTreeDataPartWriterWide::addStreams( { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path) { - if (settings.skip_offsets && !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes) - return; - String stream_name = IDataType::getFileNameForStream(name, substream_path); /// Shared offsets for Nested type. if (column_streams.count(stream_name)) @@ -69,8 +66,6 @@ IDataType::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGetter( return [&, this] (const IDataType::SubstreamPath & substream_path) -> WriteBuffer * { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; - if (is_offsets && settings.skip_offsets) - return nullptr; String stream_name = IDataType::getFileNameForStream(name, substream_path); @@ -135,8 +130,6 @@ void MergeTreeDataPartWriterWide::writeSingleMark( type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; - if (is_offsets && settings.skip_offsets) - return; String stream_name = IDataType::getFileNameForStream(name, substream_path); @@ -177,8 +170,6 @@ size_t MergeTreeDataPartWriterWide::writeSingleGranule( type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; - if (is_offsets && settings.skip_offsets) - return; String stream_name = IDataType::getFileNameForStream(name, substream_path); @@ -270,7 +261,7 @@ void MergeTreeDataPartWriterWide::writeColumn( next_index_offset = current_row - total_rows; } -void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) +void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) { const auto & global_settings = storage.global_context.getSettingsRef(); IDataType::SerializeBinaryBulkSettings serialize_settings; @@ -300,8 +291,6 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch for (auto & stream : column_streams) { stream.second->finalize(); - if (sync) - stream.second->sync(); stream.second->addToChecksums(checksums); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 95e43cd31af..4e4f4806d53 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -24,7 +24,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 5d3b2945d47..f5c57659052 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -30,10 +30,7 @@ struct MergeTreeWriterSettings size_t aio_threshold; bool can_use_adaptive_granularity; bool blocks_are_granules_size; - /// true if we write temporary files during alter. - bool is_writing_temp_files = false; + size_t estimated_size = 0; - /// used when ALTERing columns if we know that array offsets are not altered. - bool skip_offsets = false; }; } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 77bff8e4f02..892b4eccfbc 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -8,15 +8,14 @@ namespace ErrorCodes } MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( - const MergeTreeDataPartPtr & data_part, const Block & header_, bool sync_, - CompressionCodecPtr default_codec, bool skip_offsets_, + const MergeTreeDataPartPtr & data_part, + const Block & header_, + CompressionCodecPtr default_codec, const std::vector & indices_to_recalc, WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, - const MergeTreeIndexGranularityInfo * index_granularity_info, - bool is_writing_temp_files) - : IMergedBlockOutputStream(data_part), - header(header_), sync(sync_) + const MergeTreeIndexGranularityInfo * index_granularity_info) + : IMergedBlockOutputStream(data_part), header(header_) { const auto & global_settings = data_part->storage.global_context.getSettings(); MergeTreeWriterSettings writer_settings( @@ -24,11 +23,13 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( index_granularity_info ? index_granularity_info->is_adaptive : data_part->storage.canUseAdaptiveGranularity(), global_settings.min_bytes_to_use_direct_io); - writer_settings.is_writing_temp_files = is_writing_temp_files; - writer_settings.skip_offsets = skip_offsets_; + writer = data_part->getWriter( + header.getNamesAndTypesList(), + indices_to_recalc, + default_codec, + std::move(writer_settings), + index_granularity); - writer = data_part->getWriter(header.getNamesAndTypesList(), indices_to_recalc, - default_codec,std::move(writer_settings), index_granularity); writer->setWrittenOffsetColumns(offset_columns_); writer->initSkipIndices(); } @@ -62,7 +63,7 @@ MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums(MergeTreeData::MutableD { /// Finish columns serialization. MergeTreeData::DataPart::Checksums checksums; - writer->finishDataSerialization(checksums, sync); + writer->finishDataSerialization(checksums); writer->finishSkipIndicesSerialization(checksums); auto columns = new_part->getColumns(); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index c785bbaf6d0..2c5024bbcfe 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -11,17 +11,16 @@ class MergeTreeDataPartWriterWide; class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream { public: - /// skip_offsets: used when ALTERing columns if we know that array offsets are not altered. /// Pass empty 'already_written_offset_columns' first time then and pass the same object to subsequent instances of MergedColumnOnlyOutputStream /// if you want to serialize elements of Nested data structure in different instances of MergedColumnOnlyOutputStream. MergedColumnOnlyOutputStream( - const MergeTreeDataPartPtr & data_part, const Block & header_, bool sync_, - CompressionCodecPtr default_codec_, bool skip_offsets_, + const MergeTreeDataPartPtr & data_part, + const Block & header_, + CompressionCodecPtr default_codec_, const std::vector & indices_to_recalc_, WrittenOffsetColumns * offset_columns_ = nullptr, const MergeTreeIndexGranularity & index_granularity = {}, - const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr, - bool is_writing_temp_files = false); + const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr); Block getHeader() const override { return header; } void write(const Block & block) override; @@ -31,7 +30,6 @@ public: private: Block header; - bool sync; }; From a06e693901b1f0143a5e240a23bcb4b95c665145 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 17 Apr 2020 15:08:08 +0300 Subject: [PATCH 334/743] Update index.md --- docs/en/development/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/index.md b/docs/en/development/index.md index 34329853509..bb4158554d3 100644 --- a/docs/en/development/index.md +++ b/docs/en/development/index.md @@ -1,5 +1,5 @@ --- -toc_folder_title: Разработка +toc_folder_title: Development toc_hidden: true toc_priority: 58 toc_title: hidden From 5ff94f5205ffac19efc1e3dab0eccc928923d048 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 17 Apr 2020 15:48:21 +0300 Subject: [PATCH 335/743] Add util to test results upload in CI ClickHouse --- utils/upload_test_results/README.md | 34 +++++ utils/upload_test_results/upload_test_results | 127 ++++++++++++++++++ 2 files changed, 161 insertions(+) create mode 100644 utils/upload_test_results/README.md create mode 100755 utils/upload_test_results/upload_test_results diff --git a/utils/upload_test_results/README.md b/utils/upload_test_results/README.md new file mode 100644 index 00000000000..e6b361081a2 --- /dev/null +++ b/utils/upload_test_results/README.md @@ -0,0 +1,34 @@ +## Tool to upload results to CI ClickHouse + +Currently allows to upload results from `junit_to_html` tool to ClickHouse CI + +``` +usage: upload_test_results [-h] --sha SHA --pr PR --file FILE --type + {suites,cases} [--user USER] --password PASSWORD + [--ca-cert CA_CERT] [--host HOST] [--db DB] + +Upload test result to CI ClickHouse. + +optional arguments: + -h, --help show this help message and exit + --sha SHA sha of current commit + --pr PR pr of current commit. 0 for master + --file FILE file to upload + --type {suites,cases} + Export type + --user USER user name + --password PASSWORD password + --ca-cert CA_CERT CA certificate path + --host HOST CI ClickHouse host + --db DB CI ClickHouse database name +``` + +$ ./upload_test_results --sha "cf7eaee3301d4634acdacbfa308ddbe0cc6a061d" --pr "0" --file xyz/cases.jer --type cases --password $PASSWD + +CI checks has single commit sha and pr identifier. +While uploading your local results for testing purposes try to use correct sha and pr. + +CA Certificate for ClickHouse CI can be obtained from Yandex.Cloud where CI database is hosted +``` bash +wget "https://storage.yandexcloud.net/cloud-certs/CA.pem" -O YandexInternalRootCA.crt +``` \ No newline at end of file diff --git a/utils/upload_test_results/upload_test_results b/utils/upload_test_results/upload_test_results new file mode 100755 index 00000000000..058a73d8081 --- /dev/null +++ b/utils/upload_test_results/upload_test_results @@ -0,0 +1,127 @@ +#!/usr/bin/env python +import requests +import argparse + +# CREATE TABLE test_suites +# ( +# sha String, +# pr UInt16, +# suite String, +# errors UInt16, +# failures UInt16, +# hostname String, +# skipped UInt16, +# duration Double, +# timestamp DateTime +# ) ENGINE = MergeTree ORDER BY tuple(timestamp, suite); + +QUERY_SUITES="INSERT INTO test_suites "\ + "SELECT '{sha}' AS sha, "\ + "{pr} AS pr, "\ + "suite, "\ + "errors, "\ + "failures, "\ + "hostname, "\ + "skipped, "\ + "duration, "\ + "timestamp "\ + "FROM input('"\ + "suite String, "\ + "errors UInt16, "\ + "failures UInt16, "\ + "hostname String, "\ + "skipped UInt16, "\ + "duration Double, "\ + "timestamp DateTime"\ + "') FORMAT JSONEachRow" + +# CREATE TABLE test_cases +# ( +# sha String, +# pr UInt16, +# hostname String, +# suite String, +# timestamp DateTime, +# testname String, +# classname String, +# file String, +# line UInt16, +# duration Double, +# suite_duration Double, +# stderr String, +# stdout String +# ) ENGINE = MergeTree ORDER BY tuple(timestamp, testname); + +QUERY_CASES="INSERT INTO test_cases "\ + "SELECT '{sha}' AS sha, "\ + "{pr} AS pr, "\ + "hostname, "\ + "suite, "\ + "timestamp, "\ + "testname, "\ + "classname, "\ + "file, "\ + "line, "\ + "duration, "\ + "suite_duration, "\ + "stderr,"\ + "stdout "\ + "FROM input('"\ + "hostname String, "\ + "suite String, "\ + "timestamp DateTime, "\ + "testname String, "\ + "classname String, "\ + "file String, "\ + "line UInt16, "\ + "duration Double, "\ + "suite_duration Double, "\ + "stderr String, "\ + "stdout String"\ + "') FORMAT JSONEachRow" + + +def upload_request(sha, pr, file, q_type, user, password, ca_cert, host, db): + with open(file) as upload_f: + query = QUERY_SUITES if q_type=="suites" else QUERY_CASES + query = query.format(sha=sha, pr=pr) + url = 'https://{host}:8443/?database={db}&query={query}&date_time_input_format=best_effort'.format( + host=host, + db=db, + query=query + ) + data=upload_f + auth = { + 'X-ClickHouse-User': user, + 'X-ClickHouse-Key': password, + } + + print query; + + res = requests.post( + url, + data=data, + headers=auth, + verify=ca_cert) + res.raise_for_status() + return res.text + +if __name__ == "__main__": + + parser = argparse.ArgumentParser(description='Upload test result to CI ClickHouse.') + parser.add_argument('--sha', help='sha of current commit', type=str, required=True) + parser.add_argument('--pr', help='pr of current commit. 0 for master', type=int, required=True) + parser.add_argument('--file', help='file to upload', required=True) + parser.add_argument('--type', help='Export type', choices=['suites', 'cases'] , required=True) + parser.add_argument('--user', help='user name', type=str, default="clickhouse-ci") + parser.add_argument('--password', help='password', type=str, required=True) + parser.add_argument('--ca-cert', help='CA certificate path', type=str, default="/usr/local/share/ca-certificates/YandexInternalRootCA.crt") + parser.add_argument('--host', help='CI ClickHouse host', type=str, default="c1a-ity5agjmuhyu6nu9.mdb.yandexcloud.net") + parser.add_argument('--db', help='CI ClickHouse database name', type=str, default="clickhouse-ci") + + args = parser.parse_args() + + print(upload_request(args.sha, args.pr, args.file, args.type, args.user, args.password, args.ca_cert, args.host, args.db)) + + + From ce11662ab113ec728a500701ce664047bf56be56 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 15:58:52 +0300 Subject: [PATCH 336/743] Fix build. --- programs/server/Server.cpp | 10 ++++------ programs/server/Server.h | 4 ++-- src/Interpreters/Context.cpp | 2 ++ src/Interpreters/Context.h | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1578036ba46..63becff28e3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -228,12 +228,8 @@ int Server::main(const std::vector & /*args*/) * settings, available functions, data types, aggregate functions, databases... */ auto shared_context = Context::createShared(); - auto global_context_holder = Context::createGlobal(shared_context.get()); - - /// Global context is owned by Server only. Is is assumed that noboby can access context after server was stopped. - /// To check it, set global_context to nullptr at the and, to get explicit segfault if it is not true. - global_context = &global_context_holder; - SCOPE_EXIT(global_context = nullptr); + auto global_context = std::make_unique(Context::createGlobal(shared_context.get())); + global_context_ptr = global_context.get(); global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::SERVER); @@ -331,7 +327,9 @@ int Server::main(const std::vector & /*args*/) /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. */ + global_context_ptr = nullptr; global_context.reset(); + shared_context.reset(); LOG_DEBUG(log, "Destroyed global context."); }); diff --git a/programs/server/Server.h b/programs/server/Server.h index a0233d7e123..ffd89df6af4 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -35,7 +35,7 @@ public: Context & context() const override { - return *global_context; + return *global_context_ptr; } bool isCancelled() const override @@ -56,7 +56,7 @@ protected: std::string getDefaultCorePath() const override; private: - Context * global_context = nullptr; + Context * global_context_ptr = nullptr; }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 2c123435850..876229bbaec 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -446,6 +446,8 @@ SharedContextHolder::~SharedContextHolder() = default; SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) : shared(std::move(shared_context)) {} +void SharedContextHolder::reset() { shared.reset(); } + Context Context::createGlobal(ContextShared * shared) { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 62d76fd19c9..638cd4640f4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -137,7 +137,7 @@ struct SharedContextHolder SharedContextHolder(SharedContextHolder &&) noexcept; ContextShared * get() const { return shared.get(); } - + void reset(); private: std::unique_ptr shared; }; From e59aaa88f8536e1606cc15cda7c15c93834c202f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Fri, 17 Apr 2020 16:32:59 +0300 Subject: [PATCH 337/743] Update compare.sh --- docker/test/performance-comparison/compare.sh | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index bf48fe467ca..7e9da2b798b 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -230,12 +230,14 @@ create table queries engine File(TSVWithNamesAndTypes, 'queries.rep') -- but the right way to do this is not yet clear. left + right < 0.05 as short, - not short and abs(diff) < 0.10 and rd[3] > 0.10 as unstable, - - -- Do not consider changed the queries with 5% RD below 5% -- e.g., we're - -- likely to observe a difference > 5% in less than 5% cases. - -- Not sure it is correct, but empirically it filters out a lot of noise. - not short and abs(diff) > 0.15 and abs(diff) > rd[3] and rd[1] > 0.05 as changed, + -- Difference > 15% and > rd(99%) -- changed. We can't filter out flaky + -- queries by rd(5%), because it can be zero when the difference is smaller + -- than a typical distribution width. The difference is still real though. + not short and abs(diff) > 0.15 and abs(diff) > rd[4] as changed, + + -- Not changed but rd(99%) > 10% -- unstable. + not short and not changed and rd[4] > 0.10 as unstable, + left, right, diff, rd, replaceAll(_file, '-report.tsv', '') test, query From c2a18bf418dd3e2d10405e15cbb2b0d42cc00f44 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 17:40:39 +0300 Subject: [PATCH 338/743] Fix build. --- programs/odbc-bridge/ColumnInfoHandler.h | 4 ++-- programs/odbc-bridge/IdentifierQuoteHandler.h | 4 ++-- programs/odbc-bridge/MainHandler.h | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/programs/odbc-bridge/ColumnInfoHandler.h b/programs/odbc-bridge/ColumnInfoHandler.h index 426cea15b34..e1a5ad33b32 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.h +++ b/programs/odbc-bridge/ColumnInfoHandler.h @@ -14,7 +14,7 @@ namespace DB class ODBCColumnsInfoHandler : public Poco::Net::HTTPRequestHandler { public: - ODBCColumnsInfoHandler(size_t keep_alive_timeout_, std::shared_ptr context_) + ODBCColumnsInfoHandler(size_t keep_alive_timeout_, Context & context_) : log(&Poco::Logger::get("ODBCColumnsInfoHandler")), keep_alive_timeout(keep_alive_timeout_), context(context_) { } @@ -24,7 +24,7 @@ public: private: Poco::Logger * log; size_t keep_alive_timeout; - std::shared_ptr context; + Context & context; }; } #endif diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.h b/programs/odbc-bridge/IdentifierQuoteHandler.h index 2d4cf0277be..9704703fe93 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.h +++ b/programs/odbc-bridge/IdentifierQuoteHandler.h @@ -12,7 +12,7 @@ namespace DB class IdentifierQuoteHandler : public Poco::Net::HTTPRequestHandler { public: - IdentifierQuoteHandler(size_t keep_alive_timeout_, std::shared_ptr context_) + IdentifierQuoteHandler(size_t keep_alive_timeout_, Context & context_) : log(&Poco::Logger::get("IdentifierQuoteHandler")), keep_alive_timeout(keep_alive_timeout_), context(context_) { } @@ -22,7 +22,7 @@ public: private: Poco::Logger * log; size_t keep_alive_timeout; - std::shared_ptr context; + Context & context; }; } #endif diff --git a/programs/odbc-bridge/MainHandler.h b/programs/odbc-bridge/MainHandler.h index ae139f393f8..806313fc9b2 100644 --- a/programs/odbc-bridge/MainHandler.h +++ b/programs/odbc-bridge/MainHandler.h @@ -24,7 +24,7 @@ public: ODBCHandler(std::shared_ptr pool_map_, size_t keep_alive_timeout_, - std::shared_ptr context_) + Context & context_) : log(&Poco::Logger::get("ODBCHandler")) , pool_map(pool_map_) , keep_alive_timeout(keep_alive_timeout_) @@ -39,7 +39,7 @@ private: std::shared_ptr pool_map; size_t keep_alive_timeout; - std::shared_ptr context; + Context & context; static inline std::mutex mutex; From 4d4f2539194c954edf468115cd2fc623c260c1a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 17:51:57 +0300 Subject: [PATCH 339/743] Fix build. --- programs/odbc-bridge/MainHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 3ae5f49f24b..15954c63e47 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -129,7 +129,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout); try { - BlockOutputStreamPtr writer = FormatFactory::instance().getOutput(format, out, *sample_block, *context); + BlockOutputStreamPtr writer = FormatFactory::instance().getOutput(format, out, *sample_block, context); auto pool = getPool(connection_string); ODBCBlockInputStream inp(pool->get(), query, *sample_block, max_block_size); copyData(inp, *writer); From 41473d090d8f7f2f31b3ca134ead10abdf78e5f5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 17 Apr 2020 18:47:01 +0300 Subject: [PATCH 340/743] performance comparison --- docker/test/performance-comparison/compare.sh | 97 +++++++++++-------- .../test/performance-comparison/download.sh | 2 +- .../test/performance-comparison/entrypoint.sh | 9 +- docker/test/performance-comparison/perf.py | 13 ++- 4 files changed, 70 insertions(+), 51 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 7e9da2b798b..a7997bdc1f6 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -2,7 +2,7 @@ set -ex set -o pipefail trap "exit" INT TERM -trap "kill $(jobs -pr) ||:" EXIT +trap 'kill $(jobs -pr) ||:' EXIT stage=${stage:-} script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" @@ -18,22 +18,22 @@ function configure sed -i 's/9000/9002/g' right/config/config.xml # Start a temporary server to rename the tables - while killall clickhouse; do echo . ; sleep 1 ; done + while killall clickhouse-server; do echo . ; sleep 1 ; done echo all killed set -m # Spawn temporary in its own process groups - left/clickhouse server --config-file=left/config/config.xml -- --path db0 &> setup-server-log.log & + left/clickhouse-server --config-file=left/config/config.xml -- --path db0 &> setup-server-log.log & left_pid=$! kill -0 $left_pid disown $left_pid set +m - while ! left/clickhouse client --port 9001 --query "select 1" ; do kill -0 $left_pid ; echo . ; sleep 1 ; done + while ! clickhouse-client --port 9001 --query "select 1" ; do kill -0 $left_pid ; echo . ; sleep 1 ; done echo server for setup started - left/clickhouse client --port 9001 --query "create database test" ||: - left/clickhouse client --port 9001 --query "rename table datasets.hits_v1 to test.hits" ||: + clickhouse-client --port 9001 --query "create database test" ||: + clickhouse-client --port 9001 --query "rename table datasets.hits_v1 to test.hits" ||: - while killall clickhouse; do echo . ; sleep 1 ; done + while killall clickhouse-server; do echo . ; sleep 1 ; done echo all killed # Remove logs etc, because they will be updated, and sharing them between @@ -42,43 +42,50 @@ function configure rm db0/metadata/system/* -rf ||: # Make copies of the original db for both servers. Use hardlinks instead - # of copying. Be careful to remove preprocessed configs or it can lead to - # weird effects. + # of copying. Be careful to remove preprocessed configs and system tables,or + # it can lead to weird effects. rm -r left/db ||: rm -r right/db ||: rm -r db0/preprocessed_configs ||: + rm -r db/{data,metadata}/system ||: cp -al db0/ left/db/ cp -al db0/ right/db/ } function restart { - while killall clickhouse; do echo . ; sleep 1 ; done + while killall clickhouse-server; do echo . ; sleep 1 ; done echo all killed set -m # Spawn servers in their own process groups - left/clickhouse server --config-file=left/config/config.xml -- --path left/db &>> left-server-log.log & + left/clickhouse-server --config-file=left/config/config.xml -- --path left/db &>> left-server-log.log & left_pid=$! kill -0 $left_pid disown $left_pid - right/clickhouse server --config-file=right/config/config.xml -- --path right/db &>> right-server-log.log & + right/clickhouse-server --config-file=right/config/config.xml -- --path right/db &>> right-server-log.log & right_pid=$! kill -0 $right_pid disown $right_pid set +m - while ! left/clickhouse client --port 9001 --query "select 1" ; do kill -0 $left_pid ; echo . ; sleep 1 ; done + while ! clickhouse-client --port 9001 --query "select 1" ; do kill -0 $left_pid ; echo . ; sleep 1 ; done echo left ok - while ! right/clickhouse client --port 9002 --query "select 1" ; do kill -0 $right_pid ; echo . ; sleep 1 ; done + while ! clickhouse-client --port 9002 --query "select 1" ; do kill -0 $right_pid ; echo . ; sleep 1 ; done echo right ok - left/clickhouse client --port 9001 --query "select * from system.tables where database != 'system'" - left/clickhouse client --port 9001 --query "select * from system.build_options" - right/clickhouse client --port 9002 --query "select * from system.tables where database != 'system'" - right/clickhouse client --port 9002 --query "select * from system.build_options" + clickhouse-client --port 9001 --query "select * from system.tables where database != 'system'" + clickhouse-client --port 9001 --query "select * from system.build_options" + clickhouse-client --port 9002 --query "select * from system.tables where database != 'system'" + clickhouse-client --port 9002 --query "select * from system.build_options" + + # Check again that both servers we started are running -- this is important + # for running locally, when there might be some other servers started and we + # will connect to them instead. + kill -0 $left_pid + kill -0 $right_pid } function run_tests @@ -129,7 +136,7 @@ function run_tests # FIXME remove some broken long tests for test_name in {IPv4,IPv6,modulo,parse_engine_file,number_formatting_formats,select_format,arithmetic,cryptographic_hashes,logical_functions_{medium,small}} do - printf "$test_name\tMarked as broken (see compare.sh)\n" >> skipped-tests.tsv + printf "%s\tMarked as broken (see compare.sh)\n" "$test_name">> skipped-tests.tsv rm "$test_prefix/$test_name.xml" ||: done test_files=$(ls "$test_prefix"/*.xml) @@ -140,9 +147,9 @@ function run_tests for test in $test_files do # Check that both servers are alive, to fail faster if they die. - left/clickhouse client --port 9001 --query "select 1 format Null" \ + clickhouse-client --port 9001 --query "select 1 format Null" \ || { echo $test_name >> left-server-died.log ; restart ; continue ; } - right/clickhouse client --port 9002 --query "select 1 format Null" \ + clickhouse-client --port 9002 --query "select 1 format Null" \ || { echo $test_name >> right-server-died.log ; restart ; continue ; } test_name=$(basename "$test" ".xml") @@ -160,7 +167,7 @@ function run_tests skipped=$(grep ^skipped "$test_name-raw.tsv" | cut -f2-) if [ "$skipped" != "" ] then - printf "$test_name""\t""$skipped""\n" >> skipped-tests.tsv + printf "%s\t%s\n" "$test_name" "$skipped">> skipped-tests.tsv fi done @@ -172,24 +179,24 @@ function run_tests function get_profiles { # Collect the profiles - left/clickhouse client --port 9001 --query "set query_profiler_cpu_time_period_ns = 0" - left/clickhouse client --port 9001 --query "set query_profiler_real_time_period_ns = 0" - right/clickhouse client --port 9001 --query "set query_profiler_cpu_time_period_ns = 0" - right/clickhouse client --port 9001 --query "set query_profiler_real_time_period_ns = 0" - left/clickhouse client --port 9001 --query "system flush logs" - right/clickhouse client --port 9002 --query "system flush logs" + clickhouse-client --port 9001 --query "set query_profiler_cpu_time_period_ns = 0" + clickhouse-client --port 9001 --query "set query_profiler_real_time_period_ns = 0" + clickhouse-client --port 9001 --query "set query_profiler_cpu_time_period_ns = 0" + clickhouse-client --port 9001 --query "set query_profiler_real_time_period_ns = 0" + clickhouse-client --port 9001 --query "system flush logs" + clickhouse-client --port 9002 --query "system flush logs" - left/clickhouse client --port 9001 --query "select * from system.query_log where type = 2 format TSVWithNamesAndTypes" > left-query-log.tsv ||: & - left/clickhouse client --port 9001 --query "select * from system.query_thread_log format TSVWithNamesAndTypes" > left-query-thread-log.tsv ||: & - left/clickhouse client --port 9001 --query "select * from system.trace_log format TSVWithNamesAndTypes" > left-trace-log.tsv ||: & - left/clickhouse client --port 9001 --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > left-addresses.tsv ||: & - left/clickhouse client --port 9001 --query "select * from system.metric_log format TSVWithNamesAndTypes" > left-metric-log.tsv ||: & + clickhouse-client --port 9001 --query "select * from system.query_log where type = 2 format TSVWithNamesAndTypes" > left-query-log.tsv ||: & + clickhouse-client --port 9001 --query "select * from system.query_thread_log format TSVWithNamesAndTypes" > left-query-thread-log.tsv ||: & + clickhouse-client --port 9001 --query "select * from system.trace_log format TSVWithNamesAndTypes" > left-trace-log.tsv ||: & + clickhouse-client --port 9001 --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > left-addresses.tsv ||: & + clickhouse-client --port 9001 --query "select * from system.metric_log format TSVWithNamesAndTypes" > left-metric-log.tsv ||: & - right/clickhouse client --port 9002 --query "select * from system.query_log where type = 2 format TSVWithNamesAndTypes" > right-query-log.tsv ||: & - right/clickhouse client --port 9002 --query "select * from system.query_thread_log format TSVWithNamesAndTypes" > right-query-thread-log.tsv ||: & - right/clickhouse client --port 9002 --query "select * from system.trace_log format TSVWithNamesAndTypes" > right-trace-log.tsv ||: & - right/clickhouse client --port 9002 --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > right-addresses.tsv ||: & - right/clickhouse client --port 9002 --query "select * from system.metric_log format TSVWithNamesAndTypes" > right-metric-log.tsv ||: & + clickhouse-client --port 9002 --query "select * from system.query_log where type = 2 format TSVWithNamesAndTypes" > right-query-log.tsv ||: & + clickhouse-client --port 9002 --query "select * from system.query_thread_log format TSVWithNamesAndTypes" > right-query-thread-log.tsv ||: & + clickhouse-client --port 9002 --query "select * from system.trace_log format TSVWithNamesAndTypes" > right-trace-log.tsv ||: & + clickhouse-client --port 9002 --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > right-addresses.tsv ||: & + clickhouse-client --port 9002 --query "select * from system.metric_log format TSVWithNamesAndTypes" > right-metric-log.tsv ||: & wait } @@ -197,9 +204,9 @@ function get_profiles # Build and analyze randomization distribution for all queries. function analyze_queries { - find . -maxdepth 1 -name "*-queries.tsv" -print | \ - xargs -n1 -I% basename % -queries.tsv | \ - parallel --verbose right/clickhouse local --file "{}-queries.tsv" \ + find . -maxdepth 1 -name "*-queries.tsv" -print0 | \ + xargs -0 -n1 -I% basename % -queries.tsv | \ + parallel --verbose clickhouse-local --file "{}-queries.tsv" \ --structure "\"query text, run int, version UInt32, time float\"" \ --query "\"$(cat "$script_dir/eqmed.sql")\"" \ ">" {}-report.tsv @@ -221,7 +228,7 @@ done rm ./*.{rep,svg} test-times.tsv test-dump.tsv unstable.tsv unstable-query-ids.tsv unstable-query-metrics.tsv changed-perf.tsv unstable-tests.tsv unstable-queries.tsv bad-tests.tsv slow-on-client.tsv all-queries.tsv ||: -right/clickhouse local --query " +clickhouse-local --query " create table queries engine File(TSVWithNamesAndTypes, 'queries.rep') as select -- FIXME Comparison mode doesn't make sense for queries that complete @@ -295,7 +302,7 @@ create table all_tests_tsv engine File(TSV, 'all-queries.tsv') as for version in {right,left} do -right/clickhouse local --query " +clickhouse-local --query " create view queries as select * from file('queries.rep', TSVWithNamesAndTypes, 'short int, unstable int, changed int, left float, right float, @@ -413,6 +420,10 @@ unset IFS grep -H -m2 -i '\(Exception\|Error\):[^:]' ./*-err.log | sed 's/:/\t/' > run-errors.tsv ||: } +# Check that local and client are in PATH +clickhouse-local --version > /dev/null +clickhouse-client --version > /dev/null + case "$stage" in "") ;& diff --git a/docker/test/performance-comparison/download.sh b/docker/test/performance-comparison/download.sh index fc4622fdf39..ded72d8585f 100755 --- a/docker/test/performance-comparison/download.sh +++ b/docker/test/performance-comparison/download.sh @@ -2,7 +2,7 @@ set -ex set -o pipefail trap "exit" INT TERM -trap "kill $(jobs -pr) ||:" EXIT +trap 'kill $(jobs -pr) ||:' EXIT mkdir db0 ||: diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index b125d624bfe..f316a659f3c 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -98,14 +98,15 @@ fi # Even if we have some errors, try our best to save the logs. set +e -# Older version use 'kill 0', so put the script into a separate process group -# FIXME remove set +m in April 2020 -set +m +# Use clickhouse-client and clickhouse-local from the right server. +PATH="$(readlink -f right/)":"$PATH" +export PATH + +# Start the main comparison script. { \ time ../download.sh "$REF_PR" "$REF_SHA" "$PR_TO_TEST" "$SHA_TO_TEST" && \ time stage=configure "$script_path"/compare.sh ; \ } 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee compare.log -set -m # Stop the servers to free memory. Normally they are restarted before getting # the profile info, so they shouldn't use much, but if the comparison script diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 9be84fdc4b6..c65e4019dc5 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -140,9 +140,16 @@ report_stage_end('substitute2') for q in test_queries: # Prewarm: run once on both servers. Helps to bring the data into memory, # precompile the queries, etc. - for conn_index, c in enumerate(connections): - res = c.execute(q, query_id = 'prewarm {} {}'.format(0, q)) - print('prewarm\t' + tsv_escape(q) + '\t' + str(conn_index) + '\t' + str(c.last_query.elapsed)) + try: + for conn_index, c in enumerate(connections): + res = c.execute(q, query_id = 'prewarm {} {}'.format(0, q)) + print('prewarm\t' + tsv_escape(q) + '\t' + str(conn_index) + '\t' + str(c.last_query.elapsed)) + except: + # If prewarm fails for some query -- skip it, and try to test the others. + # This might happen if the new test introduces some function that the + # old server doesn't support. Still, report it as an error. + print(traceback.format_exc(), file=sys.stderr) + continue # Now, perform measured runs. # Track the time spent by the client to process this query, so that we can notice From a0ce15befdb3f76ca58c28522559209990298f43 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 17 Apr 2020 19:01:16 +0300 Subject: [PATCH 341/743] Move integration tests docker files to docker/ directory. Rename image -> runner. --- docker/README.md | 2 +- docker/images.json | 2 +- docker/test/integration/README.md | 6 +++ docker/test/integration/{ => base}/Dockerfile | 0 .../compose}/docker_compose_hdfs.yml | 0 .../compose}/docker_compose_kafka.yml | 0 .../compose}/docker_compose_minio.yml | 0 .../compose}/docker_compose_mongo.yml | 0 .../compose}/docker_compose_mysql.yml | 0 .../compose}/docker_compose_net.yml | 0 .../compose}/docker_compose_postgres.yml | 0 .../compose}/docker_compose_redis.yml | 0 .../compose}/docker_compose_zookeeper.yml | 0 .../integration}/helper_container/Dockerfile | 0 .../test/integration/runner}/Dockerfile | 38 ++++++++-------- .../integration/runner}/dockerd-entrypoint.sh | 0 .../test/integration/runner}/modprobe.sh | 8 ++-- tests/integration/README.md | 2 +- tests/integration/helpers/cluster.py | 44 ++++++++++--------- tests/integration/helpers/network.py | 4 +- 20 files changed, 57 insertions(+), 49 deletions(-) create mode 100644 docker/test/integration/README.md rename docker/test/integration/{ => base}/Dockerfile (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_hdfs.yml (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_kafka.yml (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_minio.yml (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_mongo.yml (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_mysql.yml (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_net.yml (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_postgres.yml (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_redis.yml (100%) rename {tests/integration/helpers => docker/test/integration/compose}/docker_compose_zookeeper.yml (100%) rename {tests/integration/helpers => docker/test/integration}/helper_container/Dockerfile (100%) rename {tests/integration/image => docker/test/integration/runner}/Dockerfile (70%) rename {tests/integration/image => docker/test/integration/runner}/dockerd-entrypoint.sh (100%) rename {tests/integration/image => docker/test/integration/runner}/modprobe.sh (80%) diff --git a/docker/README.md b/docker/README.md index 083c6a1dc28..ec52ddd143e 100644 --- a/docker/README.md +++ b/docker/README.md @@ -1,5 +1,5 @@ ## ClickHouse Dockerfiles -This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They updated each release. +This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They are updated in each release. Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. diff --git a/docker/images.json b/docker/images.json index 434a3c7af10..0ff38981242 100644 --- a/docker/images.json +++ b/docker/images.json @@ -15,5 +15,5 @@ "docker/test/stress": "yandex/clickhouse-stress-test", "docker/test/split_build_smoke_test": "yandex/clickhouse-split-build-smoke-test", "docker/test/codebrowser": "yandex/clickhouse-codebrowser", - "tests/integration/image": "yandex/clickhouse-integration-tests-runner" + "docker/test/integration/runner": "yandex/clickhouse-integration-tests-runner" } diff --git a/docker/test/integration/README.md b/docker/test/integration/README.md new file mode 100644 index 00000000000..4aa10d6db80 --- /dev/null +++ b/docker/test/integration/README.md @@ -0,0 +1,6 @@ +## Docker containers for integration tests +- `base` container with required packages +- `runner` container with that runs integration tests in docker +- `compose` contains docker_compose YaML files that are used in tests + +How to run integration tests is described in tests/integration/README.md \ No newline at end of file diff --git a/docker/test/integration/Dockerfile b/docker/test/integration/base/Dockerfile similarity index 100% rename from docker/test/integration/Dockerfile rename to docker/test/integration/base/Dockerfile diff --git a/tests/integration/helpers/docker_compose_hdfs.yml b/docker/test/integration/compose/docker_compose_hdfs.yml similarity index 100% rename from tests/integration/helpers/docker_compose_hdfs.yml rename to docker/test/integration/compose/docker_compose_hdfs.yml diff --git a/tests/integration/helpers/docker_compose_kafka.yml b/docker/test/integration/compose/docker_compose_kafka.yml similarity index 100% rename from tests/integration/helpers/docker_compose_kafka.yml rename to docker/test/integration/compose/docker_compose_kafka.yml diff --git a/tests/integration/helpers/docker_compose_minio.yml b/docker/test/integration/compose/docker_compose_minio.yml similarity index 100% rename from tests/integration/helpers/docker_compose_minio.yml rename to docker/test/integration/compose/docker_compose_minio.yml diff --git a/tests/integration/helpers/docker_compose_mongo.yml b/docker/test/integration/compose/docker_compose_mongo.yml similarity index 100% rename from tests/integration/helpers/docker_compose_mongo.yml rename to docker/test/integration/compose/docker_compose_mongo.yml diff --git a/tests/integration/helpers/docker_compose_mysql.yml b/docker/test/integration/compose/docker_compose_mysql.yml similarity index 100% rename from tests/integration/helpers/docker_compose_mysql.yml rename to docker/test/integration/compose/docker_compose_mysql.yml diff --git a/tests/integration/helpers/docker_compose_net.yml b/docker/test/integration/compose/docker_compose_net.yml similarity index 100% rename from tests/integration/helpers/docker_compose_net.yml rename to docker/test/integration/compose/docker_compose_net.yml diff --git a/tests/integration/helpers/docker_compose_postgres.yml b/docker/test/integration/compose/docker_compose_postgres.yml similarity index 100% rename from tests/integration/helpers/docker_compose_postgres.yml rename to docker/test/integration/compose/docker_compose_postgres.yml diff --git a/tests/integration/helpers/docker_compose_redis.yml b/docker/test/integration/compose/docker_compose_redis.yml similarity index 100% rename from tests/integration/helpers/docker_compose_redis.yml rename to docker/test/integration/compose/docker_compose_redis.yml diff --git a/tests/integration/helpers/docker_compose_zookeeper.yml b/docker/test/integration/compose/docker_compose_zookeeper.yml similarity index 100% rename from tests/integration/helpers/docker_compose_zookeeper.yml rename to docker/test/integration/compose/docker_compose_zookeeper.yml diff --git a/tests/integration/helpers/helper_container/Dockerfile b/docker/test/integration/helper_container/Dockerfile similarity index 100% rename from tests/integration/helpers/helper_container/Dockerfile rename to docker/test/integration/helper_container/Dockerfile diff --git a/tests/integration/image/Dockerfile b/docker/test/integration/runner/Dockerfile similarity index 70% rename from tests/integration/image/Dockerfile rename to docker/test/integration/runner/Dockerfile index 173551eb14c..648b98fa60d 100644 --- a/tests/integration/image/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -41,32 +41,32 @@ ENV DOCKER_CHANNEL stable ENV DOCKER_VERSION 17.09.1-ce RUN set -eux; \ - \ + \ # this "case" statement is generated via "update.sh" - \ - if ! wget -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ - echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${x86_64}'"; \ - exit 1; \ - fi; \ - \ - tar --extract \ - --file docker.tgz \ - --strip-components 1 \ - --directory /usr/local/bin/ \ - ; \ - rm docker.tgz; \ - \ - dockerd --version; \ - docker --version + \ + if ! wget -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ + echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${x86_64}'"; \ + exit 1; \ + fi; \ + \ + tar --extract \ + --file docker.tgz \ + --strip-components 1 \ + --directory /usr/local/bin/ \ + ; \ + rm docker.tgz; \ + \ + dockerd --version; \ + docker --version COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ RUN set -x \ - && addgroup --system dockremap \ + && addgroup --system dockremap \ && adduser --system dockremap \ - && adduser dockremap dockremap \ - && echo 'dockremap:165536:65536' >> /etc/subuid \ + && adduser dockremap dockremap \ + && echo 'dockremap:165536:65536' >> /etc/subuid \ && echo 'dockremap:165536:65536' >> /etc/subgid VOLUME /var/lib/docker diff --git a/tests/integration/image/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh similarity index 100% rename from tests/integration/image/dockerd-entrypoint.sh rename to docker/test/integration/runner/dockerd-entrypoint.sh diff --git a/tests/integration/image/modprobe.sh b/docker/test/integration/runner/modprobe.sh similarity index 80% rename from tests/integration/image/modprobe.sh rename to docker/test/integration/runner/modprobe.sh index b357d893fda..141e45d0ec8 100755 --- a/tests/integration/image/modprobe.sh +++ b/docker/test/integration/runner/modprobe.sh @@ -9,10 +9,10 @@ set -eu # Docker often uses "modprobe -va foo bar baz" # so we ignore modules that start with "-" for module; do - if [ "${module#-}" = "$module" ]; then - ip link show "$module" || true - lsmod | grep "$module" || true - fi + if [ "${module#-}" = "$module" ]; then + ip link show "$module" || true + lsmod | grep "$module" || true + fi done # remove /usr/local/... from PATH so we can exec the real modprobe as a last resort diff --git a/tests/integration/README.md b/tests/integration/README.md index e067b385577..c72c009a0d6 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -94,7 +94,7 @@ cd docker/test/integration docker build -t yandex/clickhouse-integration-test . ``` -The helper container used by the `runner` script is in `tests/integration/image/Dockerfile`. +The helper container used by the `runner` script is in `docker/test/integration/runner/Dockerfile`. ### Adding new tests diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a9fd572a8b0..39c6322acb1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -28,6 +28,8 @@ from .client import Client from .hdfs_api import HDFSApi HELPERS_DIR = p.dirname(__file__) +CLICKHOUSE_ROOT_DIR = p.join(p.dirname(__file__), "../../..") +DOCKER_COMPOSE_DIR = p.join(CLICKHOUSE_ROOT_DIR, "docker/test/integration/compose/") DEFAULT_ENV_NAME = 'env_file' SANITIZER_SIGN = "==================" @@ -174,14 +176,14 @@ class ClickHouseCluster: self.instances[name] = instance if ipv4_address is not None or ipv6_address is not None: self.with_net_trics = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_net.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_net.yml')]) self.base_cmd.extend(['--file', instance.docker_compose_path]) cmds = [] if with_zookeeper and not self.with_zookeeper: if not zookeeper_docker_compose_path: - zookeeper_docker_compose_path = p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml') + zookeeper_docker_compose_path = p.join(DOCKER_COMPOSE_DIR, 'docker_compose_zookeeper.yml') self.with_zookeeper = True self.base_cmd.extend(['--file', zookeeper_docker_compose_path]) @@ -191,72 +193,72 @@ class ClickHouseCluster: if with_mysql and not self.with_mysql: self.with_mysql = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mysql.yml')]) self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')] + self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) if with_postgres and not self.with_postgres: self.with_postgres = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_postgres.yml')]) self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] + self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True if not self.with_mysql: self.with_mysql = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mysql.yml')]) self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')] + self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) if not self.with_postgres: self.with_postgres = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_postgres.yml')]) self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', self.project_name, '--file', - p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] + p.join(DOCKER_COMPOSE_DIR, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) if with_kafka and not self.with_kafka: self.with_kafka = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_kafka.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_kafka.yml')]) self.base_kafka_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_kafka.yml')] + self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_kafka.yml')] cmds.append(self.base_kafka_cmd) if with_hdfs and not self.with_hdfs: self.with_hdfs = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_hdfs.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_hdfs.yml')]) self.base_hdfs_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_hdfs.yml')] + self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_hdfs.yml')] cmds.append(self.base_hdfs_cmd) if with_mongo and not self.with_mongo: self.with_mongo = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mongo.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mongo.yml')]) self.base_mongo_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mongo.yml')] + self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mongo.yml')] cmds.append(self.base_mongo_cmd) if self.with_net_trics: for cmd in cmds: - cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_net.yml')]) + cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_net.yml')]) if with_redis and not self.with_redis: self.with_redis = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_redis.yml')]) self.base_redis_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')] + self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_redis.yml')] if with_minio and not self.with_minio: self.with_minio = True - self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_minio.yml')]) + self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_minio.yml')]) self.base_minio_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_minio.yml')] + self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_minio.yml')] cmds.append(self.base_minio_cmd) return instance diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 82384c6e183..3ba8ae3f9fd 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -5,7 +5,7 @@ import os import docker -from .cluster import HELPERS_DIR +from .cluster import CLICKHOUSE_ROOT_DIR class PartitionManager: @@ -156,7 +156,7 @@ class _NetworkManager: def __init__( self, image_name='clickhouse_tests_helper', - image_path=p.join(HELPERS_DIR, 'helper_container'), + image_path=p.join(CLICKHOUSE_ROOT_DIR, 'docker', 'test', 'integration', 'helper_container'), container_expire_timeout=50, container_exit_timeout=60): self.container_expire_timeout = container_expire_timeout From 44f470827d3c17f7a0d02ff1d150076f03949b57 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 17 Apr 2020 19:17:40 +0300 Subject: [PATCH 342/743] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 955f9d1a5d1..eee9eeb71f8 100644 --- a/README.md +++ b/README.md @@ -15,5 +15,6 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events +* [ClickHouse Online Meetup (in English)](https://www.eventbrite.com/e/clickhouse-online-meetup-registration-102886791162) on April 24, 2020. * [ClickHouse Workshop in Novosibirsk](https://2020.codefest.ru/lecture/1628) on TBD date. * [Yandex C++ Open-Source Sprints in Moscow](https://events.yandex.ru/events/otkrytyj-kod-v-yandek-28-03-2020) on TBD date. From ca5ab8f10cc695eccfe4d2ed7549143d31adfd23 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 19:19:14 +0300 Subject: [PATCH 343/743] Fix build. --- programs/odbc-bridge/ColumnInfoHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index e3c00f48fb5..c5fb149284b 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -120,7 +120,7 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques SCOPE_EXIT(SQLFreeStmt(hstmt, SQL_DROP)); - const auto & context_settings = context->getSettingsRef(); + const auto & context_settings = context.getSettingsRef(); /// TODO Why not do SQLColumns instead? std::string name = schema_name.empty() ? table_name : schema_name + "." + table_name; From a0d42eb6ffebd539a0aeec246a2cdf1296d2e5a5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 17 Apr 2020 19:25:08 +0300 Subject: [PATCH 344/743] Update README.md --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index eee9eeb71f8..a7a4cb97b2c 100644 --- a/README.md +++ b/README.md @@ -15,6 +15,7 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse Online Meetup (in English)](https://www.eventbrite.com/e/clickhouse-online-meetup-registration-102886791162) on April 24, 2020. +* [ClickHouse Online Meetup West (in English)](https://www.eventbrite.com/e/clickhouse-online-meetup-registration-102886791162) on April 24, 2020. +* [ClickHouse Online Meetup East (in English)](https://www.eventbrite.com/e/clickhouse-online-meetup-east-registration-102989325846) on April 28, 2020. * [ClickHouse Workshop in Novosibirsk](https://2020.codefest.ru/lecture/1628) on TBD date. * [Yandex C++ Open-Source Sprints in Moscow](https://events.yandex.ru/events/otkrytyj-kod-v-yandek-28-03-2020) on TBD date. From 4a3d80fd712086846f02ff62763a9dffd2b1730b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 19:34:45 +0300 Subject: [PATCH 345/743] Fix build. --- src/Interpreters/tests/create_query.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/tests/create_query.cpp b/src/Interpreters/tests/create_query.cpp index c9fc4bc4bea..82bb8db5d78 100644 --- a/src/Interpreters/tests/create_query.cpp +++ b/src/Interpreters/tests/create_query.cpp @@ -78,8 +78,8 @@ try ParserCreateQuery parser; ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); - SharedContextHolder shared_context = Context::createShared(shared_context.get()); - Context context = Context::createGlobal(); + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); context.makeGlobalContext(); context.setPath("./"); From c603acd51554a6a06f47d5dda82cfd186fea650d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 17 Apr 2020 20:01:18 +0300 Subject: [PATCH 346/743] mvp --- src/Dictionaries/CacheDictionary.cpp | 70 ++++++++++++++++++++----- src/Dictionaries/CacheDictionary.h | 72 +++++++++++++++++++++----- src/Dictionaries/CacheDictionary.inc.h | 14 +++++ 3 files changed, 129 insertions(+), 27 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 36a8c704f4f..30bd521c0bb 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -46,6 +46,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int UNSUPPORTED_METHOD; extern const int TOO_SMALL_BUFFER_SIZE; + extern const int TIMEOUT_EXCEEDED; } @@ -63,10 +64,12 @@ CacheDictionary::CacheDictionary( const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, + size_t strict_max_lifetime_seconds_, size_t size_, bool allow_read_expired_keys_, size_t max_update_queue_size_, size_t update_queue_push_timeout_milliseconds_, + size_t query_wait_timeout_milliseconds_, size_t max_threads_for_updates_) : database(database_) , name(name_) @@ -74,9 +77,11 @@ CacheDictionary::CacheDictionary( , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) + , strict_max_lifetime_seconds(strict_max_lifetime_seconds_) , allow_read_expired_keys(allow_read_expired_keys_) , max_update_queue_size(max_update_queue_size_) , update_queue_push_timeout_milliseconds(update_queue_push_timeout_milliseconds_) + , query_wait_timeout_milliseconds(query_wait_timeout_milliseconds_) , max_threads_for_updates(max_threads_for_updates_) , log(&Logger::get("ExternalDictionaries")) , size{roundUpToPowerOfTwoOrZero(std::max(size_, size_t(max_collision_length)))} @@ -332,6 +337,13 @@ void CacheDictionary::has(const PaddedPODArray & ids, PaddedPODArray { if (find_result.outdated) { + /// Protection of reading very expired keys. + if (now > cells[find_result.cell_idx].strict_max) + { + cache_not_found_ids[id].push_back(row); + continue; + } + cache_expired_ids[id].push_back(row); if (allow_read_expired_keys) @@ -693,6 +705,9 @@ void registerDictionaryCache(DictionaryFactory & factory) const String name = config.getString(config_prefix + ".name"); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; + const size_t strict_max_lifetime_seconds = + config.getUInt64(layout_prefix + ".cache.strict_max_lifetime_seconds", static_cast(dict_lifetime.max_sec)); + const size_t max_update_queue_size = config.getUInt64(layout_prefix + ".cache.max_update_queue_size", 100000); if (max_update_queue_size == 0) @@ -708,6 +723,9 @@ void registerDictionaryCache(DictionaryFactory & factory) throw Exception{name + ": dictionary of layout 'cache' have too little update_queue_push_timeout", ErrorCodes::BAD_ARGUMENTS}; + const size_t query_wait_timeout_milliseconds = + config.getUInt64(layout_prefix + ".cache.query_wait_timeout_milliseconds", 60000); + const size_t max_threads_for_updates = config.getUInt64(layout_prefix + ".max_threads_for_updates", 4); if (max_threads_for_updates == 0) @@ -715,8 +733,17 @@ void registerDictionaryCache(DictionaryFactory & factory) ErrorCodes::BAD_ARGUMENTS}; return std::make_unique( - database, name, dict_struct, std::move(source_ptr), dict_lifetime, size, - allow_read_expired_keys, max_update_queue_size, update_queue_push_timeout_milliseconds, + database, + name, + dict_struct, + std::move(source_ptr), + dict_lifetime, + strict_max_lifetime_seconds, + size, + allow_read_expired_keys, + max_update_queue_size, + update_queue_push_timeout_milliseconds, + query_wait_timeout_milliseconds, max_threads_for_updates); }; factory.registerLayout("cache", create_layout, false); @@ -782,20 +809,32 @@ void CacheDictionary::updateThreadFunction() void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr) const { - std::unique_lock lock(update_mutex); + std::unique_lock update_lock(update_mutex); - /* - * We wait here without any timeout to avoid SEGFAULT's. - * Consider timeout for wait had expired and main query's thread ended with exception - * or some other error. But the UpdateUnit with callbacks is left in the queue. - * It has these callback that capture god knows what from the current thread - * (most of the variables lies on the stack of finished thread) that - * intended to do a synchronous update. AsyncUpdate thread can touch deallocated memory and explode. - * */ - is_update_finished.wait( - lock, + size_t timeout_for_wait = 100000; + bool result = is_update_finished.wait_for( + update_lock, + std::chrono::milliseconds(timeout_for_wait), [&] {return update_unit_ptr->is_done || update_unit_ptr->current_exception; }); + if (!result) + { + std::lock_guard callback_lock(update_unit_ptr->callback_mutex); + /* + * We acquire a lock here and store false to special variable to avoid SEGFAULT's. + * Consider timeout for wait had expired and main query's thread ended with exception + * or some other error. But the UpdateUnit with callbacks is left in the queue. + * It has these callback that capture god knows what from the current thread + * (most of the variables lies on the stack of finished thread) that + * intended to do a synchronous update. AsyncUpdate thread can touch deallocated memory and explode. + * */ + update_unit_ptr->can_use_callback = false; + throw DB::Exception( + "Dictionary " + getName() + " source seems unavailable, because " + + toString(timeout_for_wait) + " timeout exceeded.", ErrorCodes::TIMEOUT_EXCEEDED); + } + + if (update_unit_ptr->current_exception) std::rethrow_exception(update_unit_ptr->current_exception); } @@ -968,9 +1007,14 @@ void CacheDictionary::update(BunchUpdateUnit & bunch_update_unit) const { std::uniform_int_distribution distribution{dict_lifetime.min_sec, dict_lifetime.max_sec}; cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)}); + cell.strict_max = now + std::chrono::seconds{strict_max_lifetime_seconds}; } else + { cell.setExpiresAt(std::chrono::time_point::max()); + cell.strict_max = now + std::chrono::seconds{strict_max_lifetime_seconds}; + } + /// Set null_value for each attribute cell.setDefault(); diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index e425b9c391a..bb103c61107 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -55,10 +55,12 @@ public: const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, + size_t strict_max_lifetime_seconds, size_t size_, bool allow_read_expired_keys_, size_t max_update_queue_size_, size_t update_queue_push_timeout_milliseconds_, + size_t query_wait_timeout_milliseconds, size_t max_threads_for_updates); ~CacheDictionary() override; @@ -87,9 +89,18 @@ public: std::shared_ptr clone() const override { return std::make_shared( - database, name, dict_struct, source_ptr->clone(), dict_lifetime, size, - allow_read_expired_keys, max_update_queue_size, - update_queue_push_timeout_milliseconds, max_threads_for_updates); + database, + name, + dict_struct, + source_ptr->clone(), + dict_lifetime, + strict_max_lifetime_seconds, + size, + allow_read_expired_keys, + max_update_queue_size, + update_queue_push_timeout_milliseconds, + query_wait_timeout_milliseconds, + max_threads_for_updates); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -206,6 +217,8 @@ private: /// Stores both expiration time and `is_default` flag in the most significant bit time_point_urep_t data; + time_point_t strict_max; + /// Sets expiration time, resets `is_default` flag to false time_point_t expiresAt() const { return ext::safe_bit_cast(data & EXPIRES_AT_MASK); } void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast(t); } @@ -294,9 +307,11 @@ private: const DictionaryStructure dict_struct; mutable DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; + const size_t strict_max_lifetime_seconds; const bool allow_read_expired_keys; const size_t max_update_queue_size; const size_t update_queue_push_timeout_milliseconds; + const size_t query_wait_timeout_milliseconds; const size_t max_threads_for_updates; Logger * const log; @@ -366,6 +381,12 @@ private: alive_keys(CurrentMetrics::CacheDictionaryUpdateQueueKeys, requested_ids.size()){} std::vector requested_ids; + + /// It might seem that it is a leak of performance. + /// But aquiring a mutex without contention is rather cheap. + std::mutex callback_mutex; + bool can_use_callback{true}; + PresentIdHandler present_id_handler; AbsentIdHandler absent_id_handler; @@ -412,6 +433,7 @@ private: helper.push_back(unit_ptr->requested_ids.size() + helper.back()); present_id_handlers.emplace_back(unit_ptr->present_id_handler); absent_id_handlers.emplace_back(unit_ptr->absent_id_handler); + update_units.emplace_back(unit_ptr); } concatenated_requested_ids.reserve(total_requested_keys_count); @@ -428,31 +450,51 @@ private: void informCallersAboutPresentId(Key id, size_t cell_idx) { - for (size_t i = 0; i < concatenated_requested_ids.size(); ++i) + for (size_t position = 0; position < concatenated_requested_ids.size(); ++position) { - auto & curr = concatenated_requested_ids[i]; - if (curr == id) - getPresentIdHandlerForPosition(i)(id, cell_idx); + if (concatenated_requested_ids[position] == id) + { + auto unit_number = getUpdateUnitNumberForRequestedIdPosition(position); + auto lock = getLockToCurrentUnit(unit_number); + if (canUseCallback(unit_number)) + getPresentIdHandlerForPosition(unit_number)(id, cell_idx); + } } } void informCallersAboutAbsentId(Key id, size_t cell_idx) { - for (size_t i = 0; i < concatenated_requested_ids.size(); ++i) - if (concatenated_requested_ids[i] == id) - getAbsentIdHandlerForPosition(i)(id, cell_idx); + for (size_t position = 0; position < concatenated_requested_ids.size(); ++position) + if (concatenated_requested_ids[position] == id) + { + auto unit_number = getUpdateUnitNumberForRequestedIdPosition(position); + auto lock = getLockToCurrentUnit(unit_number); + if (canUseCallback(unit_number)) + getAbsentIdHandlerForPosition(unit_number)(id, cell_idx); + } } private: - PresentIdHandler & getPresentIdHandlerForPosition(size_t position) + /// Needed for control the usage of callback to avoid SEGFAULTs. + bool canUseCallback(size_t unit_number) { - return present_id_handlers[getUpdateUnitNumberForRequestedIdPosition(position)]; + return update_units[unit_number].get()->can_use_callback; } - AbsentIdHandler & getAbsentIdHandlerForPosition(size_t position) + std::unique_lock getLockToCurrentUnit(size_t unit_number) { - return absent_id_handlers[getUpdateUnitNumberForRequestedIdPosition((position))]; + return std::unique_lock(update_units[unit_number].get()->callback_mutex); + } + + PresentIdHandler & getPresentIdHandlerForPosition(size_t unit_number) + { + return update_units[unit_number].get()->present_id_handler; + } + + AbsentIdHandler & getAbsentIdHandlerForPosition(size_t unit_number) + { + return update_units[unit_number].get()->absent_id_handler; } size_t getUpdateUnitNumberForRequestedIdPosition(size_t position) @@ -464,6 +506,8 @@ private: std::vector present_id_handlers; std::vector absent_id_handlers; + std::vector> update_units; + std::vector helper; }; diff --git a/src/Dictionaries/CacheDictionary.inc.h b/src/Dictionaries/CacheDictionary.inc.h index 7b108438f76..746b2609a36 100644 --- a/src/Dictionaries/CacheDictionary.inc.h +++ b/src/Dictionaries/CacheDictionary.inc.h @@ -75,6 +75,13 @@ void CacheDictionary::getItemsNumberImpl( if (find_result.outdated) { + /// Protection of reading very expired keys. + if (now > cells[find_result.cell_idx].strict_max) + { + cache_not_found_ids[id].push_back(row); + continue; + } + cache_expired_ids[id].push_back(row); if (allow_read_expired_keys) update_routine(); @@ -249,6 +256,13 @@ void CacheDictionary::getItemsString( { if (find_result.outdated) { + /// Protection of reading very expired keys. + if (now > cells[find_result.cell_idx].strict_max) + { + cache_not_found_ids[id].push_back(row); + continue; + } + cache_expired_ids[id].push_back(row); if (allow_read_expired_keys) From d5a3a15e8edfe7a99065546e05d7bb35cecff4f2 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 17 Apr 2020 20:33:32 +0300 Subject: [PATCH 347/743] Update common_meta.html (#10338) --- website/templates/common_meta.html | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/website/templates/common_meta.html b/website/templates/common_meta.html index 2aca17f93a2..b71bbdc9539 100644 --- a/website/templates/common_meta.html +++ b/website/templates/common_meta.html @@ -1,24 +1,24 @@ +{% set description = description or _('ClickHouse is a fast open-source column-oriented database management system that allows generating analytical data reports in real-time using SQL queries') %} -{% if title %}{{ title }}{% else %}{{ _('ClickHouse DBMS') }}{% endif %} +{% if title %}{{ title }}{% else %}{{ _('ClickHouse - fast open-source OLAP DBMS') }}{% endif %} - - - + + + + - + + content="ClickHouse, DBMS, OLAP, SQL, open-source, relational, analytics, analytical, Big Data, web-analytics" /> {% for prefetch_item in prefetch_items %} From a2b77faee30da2f9e507789bf930abd8fead0e7c Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 17 Apr 2020 20:53:23 +0300 Subject: [PATCH 348/743] Update success.html --- website/templates/index/success.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/index/success.html b/website/templates/index/success.html index 961dc859535..83b5c1427c9 100644 --- a/website/templates/index/success.html +++ b/website/templates/index/success.html @@ -13,7 +13,7 @@
    - +

    Yandex Metrica

    From 0f11526f24d47f0a1f1c26b159ef7200be70c84d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 17 Apr 2020 22:19:03 +0300 Subject: [PATCH 349/743] Fix style error --- src/Processors/Formats/Impl/MsgPackRowInputFormat.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index 0a6f4ffaaf5..d75197e8a23 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -11,7 +11,8 @@ namespace DB class ReadBuffer; -class MsgPackVisitor : public msgpack::null_visitor { +class MsgPackVisitor : public msgpack::null_visitor +{ public: struct Info { From 741775a12f4fee11146b81a0c4aba61be3ee4b1a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Fri, 17 Apr 2020 22:54:53 +0300 Subject: [PATCH 350/743] fixed some remarks from review --- src/Common/SettingsChanges.h | 5 +++++ .../ClickHouseDictionarySource.cpp | 11 +++++----- src/Dictionaries/ClickHouseDictionarySource.h | 1 + src/Dictionaries/DictionarySourceFactory.cpp | 2 +- src/Dictionaries/DictionarySourceHelpers.cpp | 20 ++++++++++++++----- src/Dictionaries/DictionarySourceHelpers.h | 4 ++++ src/Dictionaries/FileDictionarySource.h | 1 - 7 files changed, 31 insertions(+), 13 deletions(-) diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 2e037a50a17..004a08c3b4b 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -10,6 +10,11 @@ struct SettingChange { String name; Field value; + SettingChange() {} + + SettingChange(const String & name_, const Field value_) + : name(name_) + , value(value_) {} friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 5ad4bcd3af0..06b6da4a7fd 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -50,6 +50,7 @@ static ConnectionPoolWithFailoverPtr createPool( ClickHouseDictionarySource::ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, + const std::string & path_to_settings, const std::string & config_prefix, const Block & sample_block_, const Context & context_) @@ -75,8 +76,10 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {}); /// Processors are not supported here yet. - context.setSettings(context_.getSettings()); context.setSetting("experimental_use_processors", false); + + context = copyContextAndApplySettings(path_to_settings, context, config); + /// Query context is needed because some code in executeQuery function may assume it exists. /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. context.makeQueryContext(); @@ -228,11 +231,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const Context & context, bool /* check_config */) -> DictionarySourcePtr { - Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); - - /// Note that processors are not supported yet (see constructor), - /// hence it is not possible to override experimental_use_processors setting - return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context_local_copy); + return std::make_unique(dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context); }; factory.registerSource("clickhouse", create_table_source); } diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 84bbd78b93a..13dc0323039 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -21,6 +21,7 @@ public: ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, + const std::string & path_to_settings, const std::string & config_prefix, const Block & sample_block_, const Context & context); diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index 25931e6a724..a9e90024745 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -89,7 +89,7 @@ DictionarySourcePtr DictionarySourceFactory::create( throw Exception{name + ": element dictionary.source should have one or two child elements", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; - const auto & source_type = keys.front(); + const std::string & source_type = keys.front() == "settings" ? keys.back() : keys.front(); const auto found = registered_sources.find(source_type); if (found != registered_sources.end()) diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index bbf6fb8fb1b..d176e931a0d 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { @@ -59,14 +60,23 @@ Context copyContextAndApplySettings( { Context local_context(context); if (config.has(config_prefix + ".settings")) - { - const auto prefix = config_prefix + ".settings"; - Settings settings; + { + const auto prefix = config_prefix + ".settings"; - settings.loadSettingsFromConfig(prefix, config); - local_context.setSettings(settings); + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(prefix, config_keys); + + SettingsChanges changes; + + for (const std::string & key : config_keys) + { + const auto value = config.getString(prefix + "." + key); + changes.emplace_back(key, value); } + // local_context.checkSettingsConstraints(changes); + local_context.applySettingsChanges(changes); + } return local_context; } diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index ac173b0178d..3f42700d336 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -32,4 +32,8 @@ Context copyContextAndApplySettings( const Context & context, const Poco::Util::AbstractConfiguration & config); +void applySettingsToContext( + const std::string & config_prefix, + Context & context, + const Poco::Util::AbstractConfiguration & config); } diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index 5155e3d81e1..fa47b280911 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -11,7 +11,6 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } -// class Context; /// Allows loading dictionaries from a file with given format, does not support "random access" class FileDictionarySource final : public IDictionarySource From ef80a3bac6b83947812893d98423cd27d18f1260 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 17 Apr 2020 23:05:35 +0300 Subject: [PATCH 351/743] do less things in SquashingTransform --- docker/test/performance-comparison/compare.sh | 10 +++- src/DataStreams/SquashingBlockInputStream.cpp | 14 ++--- .../SquashingBlockOutputStream.cpp | 12 ++-- src/DataStreams/SquashingTransform.cpp | 59 ++++++++++++------- src/DataStreams/SquashingTransform.h | 18 ++---- 5 files changed, 60 insertions(+), 53 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index a7997bdc1f6..b7cd9b35101 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -370,8 +370,8 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes, create table metric_devation engine File(TSVWithNamesAndTypes, 'metric-deviation.$version.rep') as - select floor((q[3] - q[1])/q[2], 3) d, - quantilesExact(0, 0.5, 1)(value) q, metric, query + select query, floor((q[3] - q[1])/q[2], 3) d, + quantilesExact(0, 0.5, 1)(value) q, metric from (select * from unstable_run_metrics union all select * from unstable_run_traces union all select * from unstable_run_metrics_2) mm @@ -405,11 +405,17 @@ do for query in $(cut -d' ' -f1 "stacks.$version.rep" | sort | uniq) do query_file=$(echo "$query" | cut -c-120 | sed 's/[/]/_/g') + + # Build separate .svg flamegraph for each query. grep -F "$query " "stacks.$version.rep" \ | cut -d' ' -f 2- \ | sed 's/\t/ /g' \ | tee "$query_file.stacks.$version.rep" \ | ~/fg/flamegraph.pl > "$query_file.$version.svg" & + + # Copy metric stats into separate files as well. + grep -F "$query " "metric-deviation.$version.rep" \ + | cut -f2- > "$query_file.$version.metrics.rep" & done done wait diff --git a/src/DataStreams/SquashingBlockInputStream.cpp b/src/DataStreams/SquashingBlockInputStream.cpp index 12f6f23583f..c6cf54c9947 100644 --- a/src/DataStreams/SquashingBlockInputStream.cpp +++ b/src/DataStreams/SquashingBlockInputStream.cpp @@ -14,23 +14,19 @@ SquashingBlockInputStream::SquashingBlockInputStream( Block SquashingBlockInputStream::readImpl() { - if (all_read) - return {}; - - while (true) + while (!all_read) { Block block = children[0]->read(); if (!block) all_read = true; - SquashingTransform::Result result = transform.add(block.mutateColumns()); - if (result.ready) + auto columns = transform.add(block); + if (!columns.empty()) { - if (result.columns.empty()) - return {}; - return header.cloneWithColumns(std::move(result.columns)); + return header.cloneWithColumns(std::move(columns)); } } + return {}; } } diff --git a/src/DataStreams/SquashingBlockOutputStream.cpp b/src/DataStreams/SquashingBlockOutputStream.cpp index 5d0638c1c57..fe8a5dc264b 100644 --- a/src/DataStreams/SquashingBlockOutputStream.cpp +++ b/src/DataStreams/SquashingBlockOutputStream.cpp @@ -12,9 +12,9 @@ SquashingBlockOutputStream::SquashingBlockOutputStream(BlockOutputStreamPtr dst, void SquashingBlockOutputStream::write(const Block & block) { - SquashingTransform::Result result = transform.add(Block(block).mutateColumns()); - if (result.ready) - output->write(header.cloneWithColumns(std::move(result.columns))); + auto squashed_columns = transform.add(block); + if (!squashed_columns.empty()) + output->write(header.cloneWithColumns(std::move(squashed_columns))); } @@ -25,9 +25,9 @@ void SquashingBlockOutputStream::finalize() all_written = true; - SquashingTransform::Result result = transform.add({}); - if (result.ready && !result.columns.empty()) - output->write(header.cloneWithColumns(std::move(result.columns))); + auto squashed_columns = transform.add({}); + if (!squashed_columns.empty()) + output->write(header.cloneWithColumns(std::move(squashed_columns))); } diff --git a/src/DataStreams/SquashingTransform.cpp b/src/DataStreams/SquashingTransform.cpp index 8ade6f5a926..ad6d0af2cf2 100644 --- a/src/DataStreams/SquashingTransform.cpp +++ b/src/DataStreams/SquashingTransform.cpp @@ -16,65 +16,80 @@ SquashingTransform::SquashingTransform(size_t min_block_size_rows_, size_t min_b } -SquashingTransform::Result SquashingTransform::add(MutableColumns && columns) +Columns SquashingTransform::add(const Block & block) { /// End of input stream. - if (columns.empty()) - return Result(std::move(accumulated_columns)); + if (!block) + { + Columns to_return; + std::swap(to_return, accumulated_columns); + return to_return; + } + auto block_columns = block.getColumns(); /// Just read block is already enough. - if (isEnoughSize(columns)) + if (isEnoughSize(block_columns)) { /// If no accumulated data, return just read block. if (accumulated_columns.empty()) - return Result(std::move(columns)); + { + return block_columns; + } /// Return accumulated data (maybe it has small size) and place new block to accumulated data. - columns.swap(accumulated_columns); - return Result(std::move(columns)); + block_columns.swap(accumulated_columns); + return block_columns; } /// Accumulated block is already enough. - if (!accumulated_columns.empty() && isEnoughSize(accumulated_columns)) + if (isEnoughSize(accumulated_columns)) { /// Return accumulated data and place new block to accumulated data. - columns.swap(accumulated_columns); - return Result(std::move(columns)); + std::swap(block_columns, accumulated_columns); + return block_columns; } - append(std::move(columns)); + append(std::move(block_columns)); if (isEnoughSize(accumulated_columns)) { - MutableColumns res; - res.swap(accumulated_columns); - return Result(std::move(res)); + Columns to_return; + std::swap(to_return, accumulated_columns); + return to_return; } /// Squashed block is not ready. - return false; + return Columns(); } -void SquashingTransform::append(MutableColumns && columns) +void SquashingTransform::append(Columns && block_columns) { if (accumulated_columns.empty()) { - accumulated_columns = std::move(columns); + std::swap(accumulated_columns, block_columns); return; } - for (size_t i = 0, size = columns.size(); i < size; ++i) + assert(block_columns.size() == accumulated_columns.size()); + + for (size_t i = 0, size = block_columns.size(); i < size; ++i) { - auto & column = accumulated_columns[i]; + auto mutable_column = std::move(*accumulated_columns[i]).mutate(); + if (reserve_memory) - column->reserve(min_block_size_bytes); - column->insertRangeFrom(*columns[i], 0, columns[i]->size()); + { + mutable_column->reserve(min_block_size_bytes); + } + mutable_column->insertRangeFrom(*block_columns[i], 0, + block_columns[i]->size()); + + accumulated_columns[i] = std::move(mutable_column); } } -bool SquashingTransform::isEnoughSize(const MutableColumns & columns) +bool SquashingTransform::isEnoughSize(const Columns & columns) { size_t rows = 0; size_t bytes = 0; diff --git a/src/DataStreams/SquashingTransform.h b/src/DataStreams/SquashingTransform.h index 836423dc90a..3dd6e336bda 100644 --- a/src/DataStreams/SquashingTransform.h +++ b/src/DataStreams/SquashingTransform.h @@ -25,31 +25,21 @@ public: /// Conditions on rows and bytes are OR-ed. If one of them is zero, then corresponding condition is ignored. SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_, bool reserve_memory_ = false); - /// When not ready, you need to pass more blocks to add function. - struct Result - { - bool ready = false; - MutableColumns columns; - - Result(bool ready_) : ready(ready_) {} - Result(MutableColumns && columns_) : ready(true), columns(std::move(columns_)) {} - }; - /** Add next block and possibly returns squashed block. * At end, you need to pass empty block. As the result for last (empty) block, you will get last Result with ready = true. */ - Result add(MutableColumns && columns); + Columns add(const Block & block); private: size_t min_block_size_rows; size_t min_block_size_bytes; bool reserve_memory; - MutableColumns accumulated_columns; + Columns accumulated_columns; - void append(MutableColumns && columns); + void append(Columns && block_columns); - bool isEnoughSize(const MutableColumns & columns); + bool isEnoughSize(const Columns & columns); bool isEnoughSize(size_t rows, size_t bytes) const; }; From 1927212f671c0eed603e25bc64888652cbdaaccd Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 17 Apr 2020 23:36:08 +0300 Subject: [PATCH 352/743] Filter out some tests that require Zookeeper. --- tests/clickhouse-test | 7 ++++--- ...erence => 01062_alter_on_mutataion_zookeeper.reference} | 0 ...utataion.sql => 01062_alter_on_mutataion_zookeeper.sql} | 0 3 files changed, 4 insertions(+), 3 deletions(-) rename tests/queries/0_stateless/{01062_alter_on_mutataion.reference => 01062_alter_on_mutataion_zookeeper.reference} (100%) rename tests/queries/0_stateless/{01062_alter_on_mutataion.sql => 01062_alter_on_mutataion_zookeeper.sql} (100%) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 27a71f9949c..81a3275c218 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -182,7 +182,8 @@ def run_tests_array(all_tests_with_params): if args.skip and any(s in name for s in args.skip): print(MSG_SKIPPED + " - skip") skipped_total += 1 - elif not args.zookeeper and 'zookeeper' in name: + elif not args.zookeeper and ('zookeeper' in name + or 'replica' in name): print(MSG_SKIPPED + " - no zookeeper") skipped_total += 1 elif not args.shard and ('shard' in name @@ -568,7 +569,7 @@ if __name__ == '__main__': if args.tmp is None: args.tmp = '/tmp/clickhouse-test' if args.queries is None: - print_err("Failed to detect path to the queries directory. Please specify it with '--queries' option.") + print("Failed to detect path to the queries directory. Please specify it with '--queries' option.", file=sys.stderr) exit(1) if args.tmp is None: args.tmp = args.queries @@ -578,7 +579,7 @@ if __name__ == '__main__': elif find_binary(args.binary): args.client = args.binary + ' client' else: - print("No 'clickhouse' binary found in PATH") + print("No 'clickhouse' binary found in PATH", file=sys.stderr) parser.print_help() exit(1) diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion.reference b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.reference similarity index 100% rename from tests/queries/0_stateless/01062_alter_on_mutataion.reference rename to tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.reference diff --git a/tests/queries/0_stateless/01062_alter_on_mutataion.sql b/tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql similarity index 100% rename from tests/queries/0_stateless/01062_alter_on_mutataion.sql rename to tests/queries/0_stateless/01062_alter_on_mutataion_zookeeper.sql From 96f4d5b7c6f41d9e4797b4c8d86a9ebe69a7dfe5 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 17 Apr 2020 23:51:53 +0300 Subject: [PATCH 353/743] Fix build error --- src/Processors/Formats/Impl/MsgPackRowInputFormat.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index d75197e8a23..92e4f5d0bd7 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -30,7 +30,7 @@ public: bool end_array(); /// This function will be called if error occurs in parsing - void parse_error(size_t parsed_offset, size_t error_offset); + [[noreturn]] void parse_error(size_t parsed_offset, size_t error_offset); /// Update info_stack void set_info(IColumn & column, DataTypePtr type); From 5d5e5ed0404cb9a28c9d5e30db411770571499fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 23:53:25 +0300 Subject: [PATCH 354/743] Fix build. --- programs/odbc-bridge/IdentifierQuoteHandler.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.h b/programs/odbc-bridge/IdentifierQuoteHandler.h index 9704703fe93..86230d8ad17 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.h +++ b/programs/odbc-bridge/IdentifierQuoteHandler.h @@ -12,8 +12,8 @@ namespace DB class IdentifierQuoteHandler : public Poco::Net::HTTPRequestHandler { public: - IdentifierQuoteHandler(size_t keep_alive_timeout_, Context & context_) - : log(&Poco::Logger::get("IdentifierQuoteHandler")), keep_alive_timeout(keep_alive_timeout_), context(context_) + IdentifierQuoteHandler(size_t keep_alive_timeout_, Context &) + : log(&Poco::Logger::get("IdentifierQuoteHandler")), keep_alive_timeout(keep_alive_timeout_) { } @@ -22,7 +22,6 @@ public: private: Poco::Logger * log; size_t keep_alive_timeout; - Context & context; }; } #endif From e970203ab25a5feec526a432b08f7443bb75ca10 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Apr 2020 23:56:38 +0300 Subject: [PATCH 355/743] Try fix test. --- programs/local/LocalServer.cpp | 2 +- programs/local/LocalServer.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 096a150368b..d2d19a395bb 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -148,7 +148,7 @@ try return Application::EXIT_OK; } - SharedContextHolder shared_context = Context::createShared(); + shared_context = Context::createShared(); context = std::make_unique(Context::createGlobal(shared_context.get())); context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index 874319c1ea5..77d0b9ff5dc 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -41,6 +41,7 @@ private: void setupUsers(); protected: + SharedContextHolder shared_context; std::unique_ptr context; /// Settings specified via command line args From 6793cd7ccac939786d31f6ffa0184df494152592 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Sat, 18 Apr 2020 00:36:07 +0300 Subject: [PATCH 356/743] More granular assertion in Arena.h --- src/Common/Arena.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/Arena.h b/src/Common/Arena.h index 32c0f4c12d1..5febed8ddf6 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -220,7 +220,8 @@ public: // This method only works for extending the last allocation. For lack of // original size, check a weaker condition: that 'begin' is at least in // the current Chunk. - assert(range_start >= head->begin && range_start < head->end); + assert(range_start >= head->begin); + assert(range_start < head->end); if (head->pos + additional_bytes <= head->end) { From f29e5c60a5739cb809fc962041f2f911d72bf763 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 17 Apr 2020 01:23:31 +0300 Subject: [PATCH 357/743] Add const qualifier for IFunction::isInjective() --- src/Functions/FunctionStringToString.h | 2 +- src/Functions/FunctionUnaryArithmetic.h | 2 +- src/Functions/FunctionsCoding.h | 20 +++++++++---------- src/Functions/FunctionsConversion.h | 4 ++-- src/Functions/FunctionsEmbeddedDictionaries.h | 2 +- src/Functions/FunctionsExternalDictionaries.h | 10 +++++----- src/Functions/FunctionsFormatting.h | 2 +- src/Functions/IFunction.h | 2 +- src/Functions/IFunctionAdaptors.h | 4 ++-- src/Functions/IFunctionImpl.h | 4 ++-- src/Functions/concat.cpp | 2 +- src/Functions/reverse.cpp | 2 +- src/Functions/tuple.cpp | 2 +- src/Interpreters/ExpressionJIT.cpp | 2 +- src/Interpreters/ExpressionJIT.h | 2 +- 15 files changed, 31 insertions(+), 31 deletions(-) diff --git a/src/Functions/FunctionStringToString.h b/src/Functions/FunctionStringToString.h index 1f712a7a785..8bc8ad8575d 100644 --- a/src/Functions/FunctionStringToString.h +++ b/src/Functions/FunctionStringToString.h @@ -35,7 +35,7 @@ public: return 1; } - bool isInjective(const Block &) override + bool isInjective(const Block &) const override { return is_injective; } diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 89687b5b23e..cafbfcf6641 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -112,7 +112,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return is_injective; } + bool isInjective(const Block &) const override { return is_injective; } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/src/Functions/FunctionsCoding.h b/src/Functions/FunctionsCoding.h index 597caec91d2..6a3d6db546c 100644 --- a/src/Functions/FunctionsCoding.h +++ b/src/Functions/FunctionsCoding.h @@ -72,7 +72,7 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -326,7 +326,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return mask_tail_octets == 0; } + bool isInjective(const Block &) const override { return mask_tail_octets == 0; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -447,7 +447,7 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -546,7 +546,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -739,7 +739,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -837,7 +837,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -941,7 +941,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -1224,7 +1224,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -1313,7 +1313,7 @@ public: } bool isVariadic() const override { return true; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override @@ -1408,7 +1408,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index b493aef4cac..64708f45598 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -913,7 +913,7 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - bool isInjective(const Block &) override { return std::is_same_v; } + bool isInjective(const Block &) const override { return std::is_same_v; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -1268,7 +1268,7 @@ public: } size_t getNumberOfArguments() const override { return 2; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { diff --git a/src/Functions/FunctionsEmbeddedDictionaries.h b/src/Functions/FunctionsEmbeddedDictionaries.h index 12b478a26b6..86e8a8016f4 100644 --- a/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/src/Functions/FunctionsEmbeddedDictionaries.h @@ -589,7 +589,7 @@ public: /// For the purpose of query optimization, we assume this function to be injective /// even in face of fact that there are many different cities named Moscow. - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index fc3c2c583a9..dcb3572b874 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -241,7 +241,7 @@ private: bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } - bool isInjective(const Block & sample_block) override + bool isInjective(const Block & sample_block) const override { return isDictGetFunctionInjective(dictionaries_loader, sample_block); } @@ -763,7 +763,7 @@ private: bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } - bool isInjective(const Block & sample_block) override + bool isInjective(const Block & sample_block) const override { return isDictGetFunctionInjective(dictionaries_loader, sample_block); } @@ -1328,7 +1328,7 @@ private: bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } - bool isInjective(const Block & sample_block) override + bool isInjective(const Block & sample_block) const override { return isDictGetFunctionInjective(dictionaries_loader, sample_block); } @@ -1476,7 +1476,7 @@ private: bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } - bool isInjective(const Block & sample_block) override + bool isInjective(const Block & sample_block) const override { return isDictGetFunctionInjective(dictionaries_loader, sample_block); } @@ -1617,7 +1617,7 @@ public: private: size_t getNumberOfArguments() const override { return 2; } - bool isInjective(const Block & /*sample_block*/) override { return true; } + bool isInjective(const Block & /*sample_block*/) const override { return true; } bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } diff --git a/src/Functions/FunctionsFormatting.h b/src/Functions/FunctionsFormatting.h index 0a789b1223a..a5c5635e0ad 100644 --- a/src/Functions/FunctionsFormatting.h +++ b/src/Functions/FunctionsFormatting.h @@ -42,7 +42,7 @@ public: } size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } + bool isInjective(const Block &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 3887f3f1669..b2dcf0ad98c 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -131,7 +131,7 @@ public: * * sample_block should contain data types of arguments and values of constants, if relevant. */ - virtual bool isInjective(const Block & /*sample_block*/) { return false; } + virtual bool isInjective(const Block & /*sample_block*/) const { return false; } /** Function is called "deterministic", if it returns same result for same values of arguments. * Most of functions are deterministic. Notable counterexample is rand(). diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 123faa859e9..9291e721a91 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -68,7 +68,7 @@ public: return impl->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments); } - bool isInjective(const Block & sample_block) final { return impl->isInjective(sample_block); } + bool isInjective(const Block & sample_block) const final { return impl->isInjective(sample_block); } bool isDeterministic() const final { return impl->isDeterministic(); } bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); } bool hasInformationAboutMonotonicity() const final { return impl->hasInformationAboutMonotonicity(); } @@ -195,7 +195,7 @@ public: bool isStateful() const override { return function->isStateful(); } - bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); } + bool isInjective(const Block & sample_block) const override { return function->isInjective(sample_block); } bool isDeterministic() const override { return function->isDeterministic(); } diff --git a/src/Functions/IFunctionImpl.h b/src/Functions/IFunctionImpl.h index 66196070afe..12461bbf402 100644 --- a/src/Functions/IFunctionImpl.h +++ b/src/Functions/IFunctionImpl.h @@ -107,7 +107,7 @@ public: virtual bool isSuitableForConstantFolding() const { return true; } virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & /*block*/, const ColumnNumbers & /*arguments*/) const { return nullptr; } - virtual bool isInjective(const Block & /*sample_block*/) { return false; } + virtual bool isInjective(const Block & /*sample_block*/) const { return false; } virtual bool isDeterministic() const { return true; } virtual bool isDeterministicInScopeOfQuery() const { return true; } virtual bool hasInformationAboutMonotonicity() const { return false; } @@ -256,7 +256,7 @@ public: /// Properties from IFunctionBase (see IFunction.h) virtual bool isSuitableForConstantFolding() const { return true; } virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & /*block*/, const ColumnNumbers & /*arguments*/) const { return nullptr; } - virtual bool isInjective(const Block & /*sample_block*/) { return false; } + virtual bool isInjective(const Block & /*sample_block*/) const { return false; } virtual bool isDeterministic() const { return true; } virtual bool isDeterministicInScopeOfQuery() const { return true; } virtual bool isStateful() const { return false; } diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 7cf0b2ce891..3c204f098c1 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -41,7 +41,7 @@ public: size_t getNumberOfArguments() const override { return 0; } - bool isInjective(const Block &) override { return is_injective; } + bool isInjective(const Block &) const override { return is_injective; } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/src/Functions/reverse.cpp b/src/Functions/reverse.cpp index 2c135cf3d7d..bc1237fc457 100644 --- a/src/Functions/reverse.cpp +++ b/src/Functions/reverse.cpp @@ -71,7 +71,7 @@ public: return 1; } - bool isInjective(const Block &) override + bool isInjective(const Block &) const override { return true; } diff --git a/src/Functions/tuple.cpp b/src/Functions/tuple.cpp index 451f732c869..772cb4e3c07 100644 --- a/src/Functions/tuple.cpp +++ b/src/Functions/tuple.cpp @@ -43,7 +43,7 @@ public: return 0; } - bool isInjective(const Block &) override + bool isInjective(const Block &) const override { return true; } diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 1f6ac0a9926..d47335eb1ee 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -510,7 +510,7 @@ bool LLVMFunction::isSuitableForConstantFolding() const return true; } -bool LLVMFunction::isInjective(const Block & sample_block) +bool LLVMFunction::isInjective(const Block & sample_block) const { for (const auto & f : originals) if (!f->isInjective(sample_block)) diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index 995fb35e52c..dd486c04c5d 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -51,7 +51,7 @@ public: bool isSuitableForConstantFolding() const override; - bool isInjective(const Block & sample_block) override; + bool isInjective(const Block & sample_block) const override; bool hasInformationAboutMonotonicity() const override; From 322681eb37f84364ae3410052fc5ac48b8675465 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 17 Apr 2020 01:24:51 +0300 Subject: [PATCH 358/743] Passthrough isInjective via IFunctionOverloadResolver --- src/Functions/IFunction.h | 1 + src/Functions/IFunctionAdaptors.h | 3 +++ src/Functions/IFunctionImpl.h | 1 + 3 files changed, 5 insertions(+) diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index b2dcf0ad98c..0bd52e835cc 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -186,6 +186,7 @@ public: /// See the comment for the same method in IFunctionBase virtual bool isDeterministic() const = 0; virtual bool isDeterministicInScopeOfQuery() const = 0; + virtual bool isInjective(const Block &) const = 0; /// Override and return true if function needs to depend on the state of the data. virtual bool isStateful() const = 0; diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 9291e721a91..82afaad4c27 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -96,6 +96,8 @@ public: bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); } + bool isInjective(const Block & block) const final { return impl->isInjective(block); } + bool isStateful() const final { return impl->isStateful(); } bool isVariadic() const final { return impl->isVariadic(); } @@ -226,6 +228,7 @@ public: bool isDeterministic() const override { return function->isDeterministic(); } bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); } + bool isInjective(const Block &block) const override { return function->isInjective(block); } String getName() const override { return function->getName(); } bool isStateful() const override { return function->isStateful(); } diff --git a/src/Functions/IFunctionImpl.h b/src/Functions/IFunctionImpl.h index 12461bbf402..116363705de 100644 --- a/src/Functions/IFunctionImpl.h +++ b/src/Functions/IFunctionImpl.h @@ -152,6 +152,7 @@ public: /// Properties from IFunctionOverloadResolver. See comments in IFunction.h virtual bool isDeterministic() const { return true; } virtual bool isDeterministicInScopeOfQuery() const { return true; } + virtual bool isInjective(const Block &) const { return false; } virtual bool isStateful() const { return false; } virtual bool isVariadic() const { return false; } From 2dbf8ba2193b2d32fce875a4f883eb496a18f014 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 18 Apr 2020 02:31:54 +0300 Subject: [PATCH 359/743] Make logger name colored --- base/loggers/OwnPatternFormatter.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/base/loggers/OwnPatternFormatter.cpp b/base/loggers/OwnPatternFormatter.cpp index 1f918f01697..029d06ff949 100644 --- a/base/loggers/OwnPatternFormatter.cpp +++ b/base/loggers/OwnPatternFormatter.cpp @@ -75,7 +75,11 @@ void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext, if (color) writeCString(resetColor(), wb); writeCString("> ", wb); + if (color) + writeString(setColor(std::hash()(msg.getSource())), wb); DB::writeString(msg.getSource(), wb); + if (color) + writeCString(resetColor(), wb); writeCString(": ", wb); DB::writeString(msg.getText(), wb); } From fb790ef82ca51a2d0be99bdb4d97442d936251df Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Sat, 18 Apr 2020 02:59:03 +0300 Subject: [PATCH 360/743] Update report.py --- docker/test/performance-comparison/report.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 84b0239ccda..a783cc46c46 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -256,17 +256,18 @@ if args.report == 'main': print(tableStart('Test times')) print(tableHeader(columns)) - + + runs = 11 # FIXME pass this as an argument attrs = ['' for c in columns] for r in rows: - if float(r[6]) > 22: + if float(r[6]) > 3 * runs: # FIXME should be 15s max -- investigate parallel_insert slow_average_tests += 1 attrs[6] = 'style="background: #ffb0a0"' else: attrs[6] = '' - if float(r[5]) > 30: + if float(r[5]) > 4 * runs: slow_average_tests += 1 attrs[5] = 'style="background: #ffb0a0"' else: From 2c1a67454c1a33e8153c2887295392a5e97137c9 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sat, 18 Apr 2020 09:28:49 +0300 Subject: [PATCH 361/743] Experiment with Metrika counter --- website/js/base.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/js/base.js b/website/js/base.js index 4e43a44d63a..fa0e8431839 100644 --- a/website/js/base.js +++ b/website/js/base.js @@ -73,7 +73,7 @@ f = function () { n.parentNode.insertBefore(s, n); }; s.type = "text/javascript"; s.async = true; - s.src = "https://mc.yandex.ru/metrika/tag.js"; + s.src = "/js/metrika.js"; if (w.opera == "[object Opera]") { d.addEventListener("DOMContentLoaded", f, false); From aa8840447f280a0bdf8f5f53e91e6da017898c5f Mon Sep 17 00:00:00 2001 From: Vxider Date: Sat, 18 Apr 2020 15:20:10 +0800 Subject: [PATCH 362/743] add performance test for mv_parallel_ insert --- .../materialized_view_parallel_insert.xml | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 tests/performance/materialized_view_parallel_insert.xml diff --git a/tests/performance/materialized_view_parallel_insert.xml b/tests/performance/materialized_view_parallel_insert.xml new file mode 100644 index 00000000000..a430c4e480c --- /dev/null +++ b/tests/performance/materialized_view_parallel_insert.xml @@ -0,0 +1,38 @@ + + + + 2 + + + + + hits_10m_single + + + + CREATE MATERIALIZED VIEW hits_mv ENGINE MergeTree + PARTITION BY toYYYYMM(EventDate) + ORDER BY (CounterID, EventDate, intHash32(UserID)) + SAMPLE BY intHash32(UserID) + SETTINGS + parts_to_delay_insert = 5000, + parts_to_throw_insert = 5000 + AS + -- don't select all columns to keep the run time down + SELECT CounterID, EventDate, UserID, Title + FROM hits_10m_single + -- do not select anything because we only need column types + LIMIT 0 + + SET max_insert_threads=8 + SYSTEM STOP MERGES + + + INSERT INTO hits_mv + SELECT CounterID, EventDate, UserID, Title + FROM hits_10m_single + + + SYSTEM START MERGES + DROP TABLE IF EXISTS hits_mv + From 548399725fa5a014f26d8cc34d51554d91584045 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 18 Apr 2020 02:25:14 +0300 Subject: [PATCH 363/743] Use IFunction::isInjective() over manual list of such functions for GROUP BY optimization --- src/Interpreters/SyntaxAnalyzer.cpp | 28 +++++----------------------- 1 file changed, 5 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index e19961e7a7c..223b7b459c1 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -33,6 +33,8 @@ #include #include +#include + #include #include @@ -216,28 +218,6 @@ void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t sub ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query); } -/** Calls to these functions in the GROUP BY statement would be - * replaced by their immediate argument. - */ -const std::unordered_set injective_function_names -{ - "negate", - "bitNot", - "reverse", - "reverseUTF8", - "toString", - "toFixedString", - "IPv4NumToString", - "IPv4StringToNum", - "hex", - "unhex", - "bitmaskToList", - "bitmaskToArray", - "tuple", - "regionToName", - "concatAssumeInjective", -}; - const std::unordered_set possibly_injective_function_names { "dictGetString", @@ -278,6 +258,8 @@ void appendUnusedGroupByColumn(ASTSelectQuery * select_query, const NameSet & so /// Eliminates injective function calls and constant expressions from group by statement. void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, const Context & context) { + const FunctionFactory & function_factory = FunctionFactory::instance(); + if (!select_query->groupBy()) { // If there is a HAVING clause without GROUP BY, make sure we have some aggregation happen. @@ -327,7 +309,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum continue; } } - else if (!injective_function_names.count(function->name)) + else if (!function_factory.get(function->name, context)->isInjective(Block{})) { ++i; continue; From cbcb4d49b5d4241deab79832db66ee1059f193e0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sat, 18 Apr 2020 11:07:52 +0300 Subject: [PATCH 364/743] Update common_meta.html --- website/templates/common_meta.html | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/website/templates/common_meta.html b/website/templates/common_meta.html index b71bbdc9539..0fd1f474b17 100644 --- a/website/templates/common_meta.html +++ b/website/templates/common_meta.html @@ -11,8 +11,11 @@ - - +{% if page %} + +{% else %} + +{% endif %} From b9ea0a33a7d174e8cdc7adfaf1181a71e68d10e4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 18 Apr 2020 11:23:11 +0300 Subject: [PATCH 365/743] Fix flaky test test_settings_constraints_distributed. --- tests/integration/test_settings_constraints_distributed/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index 854f101fb18..58a780ef16d 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -103,4 +103,4 @@ def test_insert_clamps_settings(): distributed.query("INSERT INTO proxy VALUES (toDate('2020-02-20'), 2, 2)") distributed.query("INSERT INTO proxy VALUES (toDate('2020-02-21'), 2, 2)", settings={"max_memory_usage": 5000000}) - assert distributed.query("SELECT COUNT() FROM proxy") == "4\n" + assert_eq_with_retry(distributed, "SELECT COUNT() FROM proxy", "4") From 4b4c7b9230e0af6407aec296b5003aaa488c2587 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sat, 18 Apr 2020 11:39:37 +0300 Subject: [PATCH 366/743] Add files via upload --- website/images/clickhouse-3069x1531.png | Bin 0 -> 49099 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 website/images/clickhouse-3069x1531.png diff --git a/website/images/clickhouse-3069x1531.png b/website/images/clickhouse-3069x1531.png new file mode 100644 index 0000000000000000000000000000000000000000..1bb110a1fb22db6f44720c1480e1c7d03a96d0cb GIT binary patch literal 49099 zcmeFaWmuHk8$LRSiiilJqJWeVN(<68VgS<8ASvC_H4NB@grIbafOK~Z9n#$(-OUig zFq}1md;j+ToX_W6=Yw9?-W!Rvp7q52+|T{4m+h-4FG+ln@*)HRA(obU{2ct}83b}Z z>B1TCnX%covfw|2c2b%S5D3v#>_2#r$mpBklN63mG#r&}O&nbe?2RE0jcg5#>7=a< zOpTu#8yLC0YB3gpKoA zWVZR}-;e1eCkX!ii{Q4{zaKY$+5dMxXu#hs{yGW*`I}J?$ltnv`~?tzl7EpF0{Po7 zAb%Sx1oD?2Kp=nV!C!iS<<5WU!C!iS&4a)6;4eJ@Y4DdG{J%#Jh9kY>8h3&_ac zE)jwJJQ+K>jVo5~3+L-zj>N70&JeWhh|E8atwsoB)70~?8uGU}{r|o>RjrQe*r_5b zH(x2xZUm5Sbl~}P$je;YoMs@cA%X@$B_}>HL5ZraycW@tJS#8SK`P}Pp+y6Y(V{(P znEB}$-aT7+`UjsXxgI>L5Rj(R;AX}va4{w5oa2ENt<$`}^QnTt<796luG1)-T`pe@ zW4TtDH#CVpfk3AID+>NsEqwc58PV=P`XcFnRl(Z-%7p^|(Gf5FS77|R$tfB^{<`ar zBmQRBA4mMHlRu953oCydLH8GhPdx$s+fPm%;r+M${;7?>MBO0372tv9lm7tdNA+$&Q0N zuaA}Pp7iyOj}N&nMk4Hu`zw6#36g0^XhaU;Fm8&dt?+JRz2o+HgGV23!g0>#mc`u} zs1KIibB-2pn>Jdwq35}^l-jYg(23~DDby=2v0MmXJuGZP$|+))pK}*YJBU^+kK&}RDMWg@m=7-o2T#3h zZMt@y*s%onb0vJIdtMzG5!MnD!bq=d!93u!&R$h3VW@#JN_oBTsq8$9iN*VK#>Pp{ z#UU5c4>ek2X4ac$$;GWOJ0-buPsP+LHm6$y#fu(x^JNrR?SgoixMDA%c-J!E zG!4{KZI31l{stelxs{cO(F2ae^p#zd6LP6ZflXV?tG51L$VSiM^^Zn*E}av{wpCjw z7cn^uhER`vH(Fl8(t@0XN~r!Oc17q^5boB;U{d))Ta@jZB2p)6A#OApRq}8_9Rkib zSd1uin|W`k@lz<~ZL*kJf7X>ltaG7L2eI_nb7ei`9xnIjtV%20L5N!|_9|rd?dha| zHD0)FtVY>#Xzk5@s*{YoBo%9Jx$!eC^622DOm@Or`4F(zMK1ba*>aTl9(JK?XYz5T zsHcD+)Vv|!WaE^XeAc-Tru*3mgLVU{#KlxSyies+;O2qxBzy{UU_q6E%B*Zd&~*gP5F(jtbHF36(7=I)aBvw zu2c+_{H4>M#s%oNu;YHTn@L@APfyP;4^!>8{B_Q2V_TbVh-|A$e>ywqXnTCXYhf?n zU-dwxemi!tVaw9|zKOkf<1NC3+rCOw+e30~xom@k3bm`uT3#kk6~tD?6*0Lkdi43J zynyGARPlM>fvgx|KARD%pICp=>9t_*=y=<9d1fTJZv~G^`WIZby$kZ`XN8ONt1S{; z*Dd4Tq9+664kk;%k(aFQA}_&^g=^<;VNKr-#@*LBpzSf31o`WlWRTwdF%=}mOdz{l z_R#3KL!ospQhu6UhnccX6xjn_f(z~CTm5CPXXb>zH zrx-rULnHKC!t?ZgAbYW?oSC^HzM1t1yE^3X@##-6R#I+&j94EjtuvdA%bk09NrIf( z++EyH+g7M&krex_P_fe^;0XdF(=4+3U3dDgwuaugPW<4q<}QE|Hk(RdyGtj9Bo1u0 z-{V4XQ3901YRmOrmUSC!X>Ru)5MCvgaE=jpb-PvAF0WS!ySR4+JvLLk(Jlv4%I4-v zJDt{i*AA_J&6Z!1a5jduuXJ=M2VmrzRQ^{3}B8(QPQPXknKQAME%2yLka9 zTe!~F59ywcyB5;i7at(;gj=m8~`n+ZYE|_$kIEjfa8x)jguA#9PTXR&v;j`AK zB$N*pLu=kN`m=8YWhMX; zR;b>rOh`(^Jy+Wo2n2vc!t_O1~S9 z%-*}sV>%WRE671#12-P5J%NC(H1@q@9+Fo@xrcGj-LV-xMr$Td#qcGHny*` zCbl!XP%M?A<~!rUSXC$a%Ysl&`&L%&8;fyb0+9`m^o=J3$W;f8_FDu@93v93HbC4S z;SyVrEn-#YMLt9AS$lOuC8u`LUehstQ#CQ_*Pxk;>{h?-Yl`GekD3s6->k-5qP@(z z^FA0^_)2ojk5?1j#cisWn+;@VNMBIe(> zuu=E+yp1clTURUAuI|C)+_@2b-mC7Wwoe)gs*a~aQdDkEnORtnn;Ms=TMS<7xD|oj zSCQ^C=VZVEz^5-9RVi6O*Q0n=HmAgQ*FIIM%@dmqbVuU%IlLVh2C@d=cuc+|?JF~sf>L*N-vtDGchNYKi*;vy z01vi4EQdH;qneDc(k<;P&2?DVD8OB$4cf;AoU|`J+gcA)a|cF6t=-bvTJ@T7Bfytf z59i(w?Ql$Ep?hXqlmm^se$A0&U{;!rLyVrOl9SwW2A~tko!A=kmY~Dg^bx?72>N%e zz)^mhb9={RW#-*qIj>%}lF(x6umv%QnfO&vip?UxV~!cwgxB_z)zC>!Q!O0GT2*CF7z}h z7y*<3&{iK@qC3i8iK*OgkC-bl8vOa?4uAs+af9iRvi^LN6OE;Bv`xF;ld>lH=&(Ot zopkEJ)<(G~#4DEUHqbrxX$9<*d>)620t9LaVTu*rk4V6NFXZl7++G)q0IdTAL~`h= zwVo_6SsHc|ht%E@o1%}f-}n%ik6!u2d}r0S==Ab)o;(iml=}MnLD%m<*_fT)ai%+i zYnX30+uq9o_SF2pFI>QV!AK9xWhrr55y%fkQnme_AoTf0% z2~bNQEdO-46o!D2?-{W=>fVU%BvRecl|2Xil{c z?rYwsLu-&ZXqZ^iL+%p)-B1fQPDy~{D3IHy_4|}Dq|$Ol&D4c4Gtvfc-}POkAOwW0fP+Cr}8i8)Q44DV+A zKRz$)H2Hlw67&-LYq%Z${6zeuF>JXilCpIhcUow`DKSKh7N6*o7gvFz27}@$TZ7Do zo^r6=bdXFtjl`N`K9P|P0cW7+K>|nr!*J$Q7Fu>E!Q_mzK&aeS5;5N%Uut?oK!Qnu z(|mPTe9rCp+Zzx3os!S-WA}OQwy}1nXn|o5jHl^BY9G9a=H!5qW8E+9L@WZe8LVwv zwv_zv&G~U|*RGGFHtZ~vW^a8CfG?wk_lMu9*CMlquK``cC%B9rVOKz1 z=SEih8PSjN>IsCopaF@>fZ7|?i!M^I@a zyey>NVqsqSC-p5Io#H2GaGL!{%uq2WG8K$vcBPyeO82c7a}1WAfn~VcX(7A5^Do(9 zUND3#Chun%6LsN_ta&YRx)JLR)hFuemV+JClZS#!<*~afM$#>t+UH3qdF|iUeSHKD z@k2fZaN$3w_3hT~N~xxN7(8`!cB<~fr1OfMDbXKJf4tJpuiH7047$WO)==TI5ZHGx zM@rJtp6sfadT48(RJJ<3;~4kZ6{bisPLz4}e$v^}cZ39z^%&$&Xkb8`kkj%^pcaZj zxP(QI^#n}c{ZhlE-!;CI&e-qzVP=^d2a(kg&E!8GA z5L$3J^9+L3=hG+imredM`*=WwDAC{^_Q;q^6o+lrr+Hee41W$v5=&~8LPS>kOx8W%NrR1H_uL1|# zs<=^Y}*6e<%fQgV8p6u&#qTBB%Q?8v&Uc;@ikN~_JPvP zm|UWj8vLV~6QMEU8%bWNRI@G0EH6}B*l|$3$O-JrJFx5moIQIAQeqF{a70oOb@eJz zC)00o%N?{%V5)qCXPD*LFfe6Gj}pwrId*UFY1_`PGl~^1qy~Z(kSYbqTI1bRpuN?; zq^DaZvBV{GJhUNzj2lZsRjr*DgsH|LX9V+2ohQ;T_XNjLHfmsATa(IJf0b88d%8{=;X@&wu~oRfqRpsBa?NWxJeZdsJ)vOuQ* z^qYHcU?ar0_((5T__xA8Tb4UG}g;v-nPBt4)b6{ z{p1H%1k%rI>mv>+owQgli@E&|#F?i;L2n`#=n<@H8Er$I>{$Kjott!yn{xvz4h59IT<)UVO4Y6w&2ypSqPZ9U2VjJUOqghO@`^Fxz8JUCg-kB0-Wr} z=lT~roYWo2h)k1jbx*}|W5Mx|bn8K*BQbh#NMlT=sLx8{(bqXD#^lvv5z;@?ku z#w5@~IN8u&+;-+WT`oX4LcLl~I$pV4o7cvub5Ij}L|=RcbR*d!7*^79h>y$KW|~uF zfEqkIUes0PIP(v5oJn4f;U;xV zoNr+1Ka5;`CaHf6x;QhvaJP(h&q@_<;lOqF-Q4w99t!Q*_2&*IonQ>iuES|hFhig{ z_4GXNJ(!CF!XnE?OiYBes#^PKk&_GL50-JJD@rjq){at#rA3 zEpJy#Vq$wk8$g^#XB_|K!ylQIFM$fzX2tsD?nHrkVw{4vShr_}G00a@%+T}W8r9)K zhYu8P^tM0NU7mNErEl$(0w(wF@Q$TBuP%bp<=tQ}kk34UfcO4&+bMgZZJqvAu`e5E zgZ6p-Ew;W^8w2C`ayM4UIPX<66#zIM)3)ZeUIwS_RdG9gFfF{NE|Fg{zI&a0@Q&!6 z>_T_?6g|T98=Se$7UF=N8{$peP-&0Z_*|w%B3@!Q9Kw2S$0+<1#GaF@&x6s42?4pN z%sm{X>p!TCIX0-NJtCG66V+T^UgjCz^QRQ7&A}gR;ri_PBW$+sU_l-&b5-I zTV&lEKq(xC=eA|dz0KnuSZNp2~4;T1SVgt#S8AE1anUA6;tBZVPLip%7Y%9ok|D z!{}WqQeXA|`nF8?Nd;muwNcd;qbTg`@$^8XB(1Q}4!#v&t!?H(dkG0snSACxEpch? z^tH>kJ>u`rBQM5$J3N0#K&pn8#8B)F3Wj#|E$^y=z0xc#8Z3^qIE%9NqU1X8&c1Xt zatzD&PKIeIcFHdk{!*PcDWcQ&1SV=mL|C~!_!b#TE@o+IvmST3e{ehDgVBAbjMju& z67IX$Q4+`?Uw2XXni|I)HzD({W;u=Z43vy!&ur`CWpNj4gtTTO1m&!0z3Ft!R`ku{ z2l~q5W0S+#FWl7BJAsIsl5H-3x;~X()N`J@;MR(LpPHWt)bcA%0fvxr3kC2Sa&vm; z)P6yPtE*=)5Dpyt`q;nst?XB^?YJzky-s!B85k4&#M?d65Lm5IcPBB5&xVfV%yZ>n z_7fU}!1i^^^rv9ii{fVfxIPVmNSN5!No9A*&YvR{6W9)`H_|04=^99=;^xkMrZcDd z&Bp#RoxW~1=e`0}JxlQY4Y&!Bmvhd0707#T5$<2Ix)j1YF`>2^S$y3ai1H7eTEZ#^ z5C3O#r6`TgG(?)fGpX>qdxM`wM^AY*>9ir#M9*cT9zQ5CNfMU`k0Ajpfjh-~nq3 z+@zrx*DKM>7PfXZRd|FxT!k-FlR>HADpm$t)LEFc=oA!;hSsY?H)2w$x zcAl5FC7Ba5BKGSc;Y);D|k@5$Pin6I1Ykz|nEqXeR-swM|!7i+^8M zX|$MUQV@Z(2i}|7sHIsROo$pvr@d0`x|-*e_(HomF+oI z+t=A2iw%Af+nk?2zX%GAoaRih6dLFn?>(9+v}{jNpqMrF0coyJ@x<((xLt6q6srud zxQl@38<+@g7Y`feIZU_<@^QGi_4FJt(wo1W-+0R9?0`mimGScNF~RxjjC6PGzm+ze zFMDNiu6l&1ay&MdjehD|{f`62S9ok&fiCM5G)k>U>~oDsygMYa(cvY>!+I(cPq_>c8Et*`u|WB&RRC&E9>6)oEX2eXj-NWxo= z_ag=?o{JrBS>-7%c6QA!!|mf1rQT}^mVC$(ilDEl(4#tev5d|tZc5aNq0KaCKWZ4xwf2)`(-OkV8IrKgZ?Kx! zF4CH2OdqCUTMtq0uT;;{U9T`I5JHHalcy{^!!6Gn*AL5Dd?TE&X|iB098Z97Ny|*# zI~lrH(5s$Vyz=T>sgs^$pV#oxSkguuYk!_9a6D&XQIUK`_TkKB71fI1@cLvz`iD5G z1A)+UJ#c*#R+4IN4(b7UGry?wOH{GJOmhEHx}*5wHb3U@pl-Sf~h{ zmeD55f`(+eRaT8Gzp-+Tov#UuNn;texI(BJ6W4Iw(9=2R>7@7t5c^bFB6dk!H-PaF!j_y3uQn@$C1q{H zF9#NQde{K4c|`4!l$5NZT6VHO+c(7-IAV1_0X4`SG5Gnt1}`=Wlr6c&tHE?HrLcb)2WlcXh{aZ5*BZ7NWfR&-$}8Gt(?uGX^QO zl$2W7FV`ggK_dMn*A9c+NeX=LvP7Gak{UYBMiW2a*6tIGt*vU(XfuI$x8Dr_-TPf# zia$T00Xz!n0x4_CE8@=7bM;cx(;x?kPegQN$FH_N7;3iS|ARWdK4}>_bD@EF z-5>6$Wr&-|FQwzxElQ^-&p&Vv?M-r;?C?xEkzb_sw&eKog=iqV-R5v_O#FjnAB@`@ z7WD1Bp;Xb9*H}uNdU1g#tKf(j1Q^e8y*zQ(x<5lLU(r$XGf2uYkH>%jt*D}9> zi;mFy8^EVGw+=gxoc0#Gy1G0(T%wW!W87mBShinsXD?L=lt-WA?o9?gjIrU#>@8^Z ze){w@+4v0|oU0z;?q8OZC=){TEL+_Q$LuKZwHB9-A0AAY6`TH~lIhJoPtGx6n}A{+HgKi_ct!XZs%P z3!Q!Qr$}G(^6(_ej1BhaF@wCR)BA4pc0}wX72$jFh-T03i=O_;`#`7E`kVb@_s6Qe)sVb|{agGfm@1+yF;bbnn>y0{iVQiCSL^su0sM0E0TT}lZ z=HZGFG!JPpNbAq6)c*Zucx;5iG3^xRHjz~{ZHQZYgX+APiXQ9ET6_@+lndZrXsMoP zJIHWnmy`a!IW|kR6cD(hrGs`vP+*JW>>m9TU*Qn9D~!FRy|@p9^ekW-wxZ5QHt zWKwFWu~*-2!*_CG-`(^e1 zetw|pGN%0Q=u!Vn>)Pk_M7b3IwuTt`iU_fJ*DV|(ON=|)c9 zJ0}n}Xf}^80XhXk3)iq}SxzeG`xP-|m1Z|GHc1QoW=VCycY-ERHvYN#;a+A`9ZNAL zd4BnQs`I13OZ*Db@_E(UTJ!dV1ghHjg)X`XzR@#bA3WdTGTL`0`9rnh9}cGYw8>`q zdgR4R5c2qqFDD{?u_$pH0G#44Dp74h-uyiIB}{S&0N8RlB$);x@)xip!{_hQWR+f| zP6w@tYB3K}C%(E{E)y@4>A>RW)-PsR)47Hy_tGeYVY5~@e0R-bnpk=5eOdX*hV%$Dn0<^kg>d#mQcWqD~f=AHYS zW$R_`pZkiiOq5+|tq&bix0HqS$vC5hMOw=qs&IIG9Da<7>K(St6Q;EC@T_V#%}0- zzI_OhwLYbzWpDRr<={$eC*Peel#9-+wNptKenT&)`v8>5 z$%5`n!xnkTh)?yUz|R-2nHwVt`MB-zhKoLf&aNQF^5yfvLZnbDXU4Ly{rsw6t{vcM zv(FlL{vplri@a*~81#i;lIn^of$eXy-+J!+T=UQXNg>xnc^U5JNCS|mT6I3uZTYU# zH+9uA-2U95{B5}XC*UrDsFKooAIRtN3P#d*12A%BW#u~*>j1_-NXu-1x>OL)4)skc za*{1>`e$2+@^`*--fnX8a)WBLUjt>VC$D0=;Uo%EWiZ3^+`vfh*5NYr+v7SJknYm2 zt`HKC{umh%n#!%n*xuPq-%S7cb2gIT3>ScWr;nh;6s+e5g9aTe9-=(hiLG7c%8ga$ zPnih|9We{;jSdNa8gn%Ot3|Rwn@+kzNL%{%@6QjK;WR0j{AV(9gdw4qn4A(aXO3O= zFPycXou{_6`hDppbhG@&BV=Li!#V@U-!FbLNCn*36`tt(8Ojn&B9fg@L#6e>)v4jx zwiShL_6C>}4X0N3K3{Zk(=gUpd($`bRyJe4H>Ho@m#Tm19~^`1Xx^fhBZGvixF}?! zH5@FC^p~?gztjn68w4E-LazicM)Fczw~xb@(8wP5TpfW$FtOcJhQx+-rW_VCF$(&M zXg8+L=Y!Lxr)Lw@W%{^B2OY{4qV9UQ!+)nWj{P%K{LY%Z*~hpJCK}Llc(&bP{RB=o zTV*6^!}~Uy7h}()KV@Q%>{XX`>Rz zX^I#NF;>r6@r}U{LLpT3XHdC-qMg%n?3y5h{&cOZdgc;g^!PN&5%3qPN!pB(+|x%J z>q@LQA_(Ta>iUH0^WL5X{`L27s@K|2otvqwEU)A1HzY=DX?Nt?7_R>#WY{6bV!9b^ z9zQOZV&-bK8x#{`D}p_zFnJ z0VCR%-1|^vumGLcW8d50X2Ye$NS;qC-)C)ZZr6$h#yx(7CM0tM4%2@W5P-*?o0k5We`YLv>gHu zQHB!2Iuo#y+!3*->i8F~PYu9;t}$9{!?c#K%zp@b_r1;!T-lHk)+gm;|4|W`xx14) zY&bEz#<1|LhQ{~2SKC6v{}LsciP_1wJV?Cz3>Vd*f%ON3$g@Gb@NQEAOQXx1KvlsC zAJ?-7_c7%j^|ixR_vhw}f513c#^!?WxCip{?jcEnoPS4vbYmb4_v10`^lM;oP~XNE zL}aw_8=WD3v7qKH_-_rmiRb{PpaY_e;m;eo!a(j`yx;gxw)E(=)gNvR1oY}JN7~`!BZT1RY3TnbESrx zhY%VSn`QtJua=gHCr(}Tq0-gBf6VrVEFqi1PPwe?Y}@coFcwyd`E422xWYj7ftkx^ z`X-LRz6V|b#O8u@(fVk_hOCSn{X(Y)3jE>#xzOkmCTx5r)%r~4QS4pe1$LjikKXW0@&0#hb4vt+ z+bn`-vL*LyTf+)+&7QoV$ZTS^$HcF+Zv^R zIC&?zObu&2!21|(LCuIkcO0vy(OPu*vZG&-KYEYHR52pFWPHHOHyU3EHMf3Anrll9 zl`*y*8G~dZgntKWTLg63=abb-=Rd`#&z<{9M(PYid2xLS=~`A_yVmB?(@iix&Ei1& zGmZ4ZE@-qCM-a2AF~QPl{K75=R-)%dv#(!^dNkA$p0wdt-$5Orq+Yw>k`|kT)Kzg$ zS8HDW7OLdiy7bT36`vb51Ipv$N6?~Vr^;Ky80)&rNapPmG9M)uWdkkbQ97Pk%?h$* ztVv_>ky&q}NUjErg1UT{pMjY8N|ui%6BIFJ*4>vSXpX$i}FG^{Dm^Pv54% z6VwuUYUAoZhzZ@_m}Dk|Y(4HssP>l&PlPA($xe(LTHC-s|aZhx6T z;tOOg00)LsZ{e$WzpnrZBls*2s5ssTD4%v)Wd9O_Cpq^c6@ZG5((ViYp*Il7+t^KM zGHRIqZJ_VcUfZ~FL1md`vdW2vkRbkxgK6i-hLN0CR|!cpx%r!UH-%O7R*!T1{{ zMp0jk9A32et}!w4Jf3q^zVkoZArMmcU}kA&kMm$%aa*57tUJVHV!}RxBP4siMpXI#;s*31 zNEvq#9m6SS#G_Wc@Hc}i4&|(8EslV&1Ug9ky%xU8UGj^i{1a{0{1}Jlu5f(;mB{Ur zMnc0dI=?-`n{a)=rc+9z=<6>x$uU3Y0F#ia7jrEwM;t*_-wh~P6@^DeT42*(E>CNvy^W15>m309dPC_w_p`FIb3+0VP{1a} z*Dq!dN_O~fG+7QBok{h7q-AlDFwPGk-Hng-3mw$Y{WbrLQP?LFS!g>sHgXW)i_G$T zt>7hq<<(g#JZN>`;OwuXfqBJ=6klvy&%fz7auVZodQv?zJ#E-vn3`=+TK+nkN)N6V zCmX*SUEExhAS)*ye75Z$MGpc=a72D!HO%F+8hc!8V0U9g$k}T2HtFK&pa}dw0G{b= zB|nYOIJ)S!D3wOt((zHWQas4+_mwS~X@)xLv}~turPAeJr1@&$fd7%%+zsX-)qnbL zpS}kt%WS(F;jmJ80^n zBP5^!yx_}tEIq{*RQ23-ucv%=I_qi?!EG0X)q!+ie5QMe{68t$EJY*L)lG1oUxYjI zTf{w5pjjP(0SuV`xu%9bS0fl{YH-(m>vry81(2P;*!_M5#K77sBr2-c);W_uHHX); zB&_XSDm)uxmq*f8*?K9y6uz%dHwhgGlX~t zEp4n^^=mNFy%{>VGCIe$=~`BB89)NucEbjg?lxusj5|!B8grTg+J*M-G}u~bI=wX) z3lo)GwpCfxo%eUQuiR|fZ2noT@CNOe%G$tlxITV|(3SFkfYUhJR{#|aiMzue*iylj zVej$|dMJ~cUXj)#SA-|EOk2W1&U)|Om$#Pe?5bJspXW94=#yJW=4a`{6Y)we*C9|F4dfD*b8rcw#%YQRJD*&@FFg?M$m5^I5m?G_wj$^eu93udFZ4Ib4}|mQyQ{A<5nL z@n86%r*osD=X-KFJaOtvHxP8ps2gxhA|l7?9%w3laT{$P8$RhtZ28=h%u+PJ@{Xwb zDnL4rWBeT}VC`@NIIKNVJXTurs(O4e=&mAYougX~yfE zMWXV!d2luG5zJ_)V5`c;HTmbBo)xn3td)}?*1vrLBYNr~O`|Xf9_x}_y{cnu!?b6y zwmx2-kpuY`j`2R}R!D`FfQNfvBEt_B3DTfMynS0oclrPsYOS}@;2bawS(M2@Mr!Z~ z9vnBdO2kC>*-JBX{r3HSgHs4d^Y~R<$U;#?)vXT&li8X(O`Vqqq~y&>Siy6-yxKGW z?Jv+(Xsu1>$ImaIwFab58W=%8EDQh>B@B!fdDay0m7}8*AGe$N6B0L+LqfQH>LXjG z^a%CHn8__2SD6S2Vi$pQ(J$YW!`V5rgqlqZo@8W8>`)*wY?r&irq4bZNDK|C8VgD;Pe$pbU+dg zv4)g|jJ>IEXb85G4u*Z%n5dMg^+^FDxl*T>iCZDb;9=>nG4Lqd^R>^?3N5MM^($8% z*2WE1dhAP`ar}2wb=a7I1w*P885T00Jn8NxK$Uk5Y=KdyAmI5jOLm$g*k-DTK$8nD zMtFm1ueVTp=F-AB6XoYGjApM?D$H&<0xn6FjrSg%8k}>06b^Y2usA)G*;ypKCe$YXnW~X1CS%=r)uYA zzNw%*#qx;TZgbl|)Pap}kcCFFhnxc`p@ELIar>>;eA0*fwjgc5%;csyaQhZQ`+?#VP-m`;E{4)|De(+LA4nuD@igo zHt=*$BcnSvis6=Pa@w=vJN%u7w#%-{GoS=9qVBKOr)D9Qhy4pzl>&3*MQ7F`GfsFy();k{HY+3!<3N08UYC-sd zT!6R*?kp7i3jxH{;S8hq8nD#^ai#NQL1U0AWA+oLL1G)zyLH*g%RnsrX7#^%hoY_w zTqmrg3e=An2F@)ykx&jgy$3cgOb!^opQHw-NoMn+zz`)KYXNVxp91Wt^bLzsF=sez(}VM zY;fZW?!^i(3K3U3wwYK#2emCcx^($ykwSf4Yi}PuYpeDd(@@Cl+wb3@D?7Rsu1bYd zPp{v&k*PV39wSP>xLKZUWf866u#jVLvW@m@cOU=KWJAL?(X>x?y6|BNZKK1&T&HR0 zT``DJXX(z3_1Fig<>jnxc0}|2u>x+-hVdW=VhZf(qnL@Kq@+5qq-4sRg1F{MC*z}q zwwV3(fEefV@$nyV@3kO4-@Gpp`U-I{P1Hbeu`B)|{&gP+hfdiGb#+>{QU2di1fi)a zFCS!0DRbf6cl&TD{}TyhSngJ*%wD9Z3L*=YRJ8@^FbkWug0>wKWg88nU+myoJ**V?*)4 zEao;Y*MM`UQhPV1nX$3>J>YV>{3iMjZYL=ALie~t`pW{iL`-lz~~XcbH7?! z_t!RZ)Pq>3-94gT(4QK^yBzxE6|r^A6-aa1^z?U?w1fgjje`VrKHjSGmnq<2&<(j5 zZp~)fYX0LK=7F(Pf7&40(;y)}`T0$knW7Gtc$TpS@VWPuA-5dcVw^WbndM{i2qBOX zdQp0W2Bi|SWSM_l*h}fE67mo9>Zf^m{xb)wpYfPDwRrLYB*^+|qdH=DyG-MKl}8T) zwIoQdH{d37ab-Rdgf>odLNVtFa4Jf(UjcsfYh#rJx_(+Nmky-eL~J$IJx}(58}Qh) z1mJtoBbzhME-vhu>FVHD2%wTJUKsa&>eK6pfiKNDohvuo0I#oTV61*w@-Q5_q+NLN z#ufX2D=?~Nzav@CZR6q+#(M;z`y4v2nkzy&UXY7Cyb;w;3-K0Xr4Taeu01^3JH9{O zmJJ-0psH`IJTo@pM&`w!h=-vmRlaXszmvN8rpIp zaCN4i$UnCp%bECGU+9Yi1(UDppM8zTKw4a*{WvJcKYq=MZHvi}-|5$^#z?dy@SSEw zy@er^?fm6iKQDj_)e*f0=UG@-o?A-=fCDX}o!gh^Yg-PVEcBm?+Qhg2dg{T(q5Icg z5m(y6KcMjMm$})MxK66;!7_i>XPG}?3GH|UvYG0+|1!y2m)7PhHI2(6jv6(1wdLFk z`^zg)JLTOb_7l+*_YF&6BqgVn*Zeu{)wfr~u+^bgNCNL5zJe@Hc4~4jtl?fD4f;S- z*zR#~xX&hNT5nh#x0dc?S5>)z1gJnqg3T5NI{ZYZ)E}5B=Hzz7AoTa&-(tk&J;l{Q z>dt`UHIOp&0Glh3djkL#(7ER*_s@W2WCZW4Zg*w)Qw=PvlvR$77F}1dUG=+xC{TYC zaGG|X4-Yeo8@O#ejwY@$E2N5(hJlx0&Qq=I^qO9ae*2SB{Sch!C53xUg;pg$q&Z_} zq2nEe*}=yzX~d;-znnaF*cOVNN+~GRT*ErcZndWb8;Ua&h$)F@wY+2i#00N%}VO?=)EHN zX504ro*sizOj4{@p02GG-PCD5y##BfrAFLr?HX7(=dJM|nKD@}?uyqK$P+c*;-BI{ z@W+RE!PzAxX0->)$=6e}vfz`Wm>LJpj87}?HJ;WFQ3)G)W172PI_ot5$#m(Nv@}DC zK~x2_;x*FhU`LZ0CKZYW5|-E^M%E}wJ%HD85a`LfwnET{UewUX$==jE?#nBK9bpcJ zeM1BzcUIQdEv#M`D4+2?jZMVvj<&78rP<2Pnfqmls1AVh$bR)Sxt_27G=(-WOSVVi zs7?Hs50^*JLLvDJeeWrxzKWCI;Pj%f%J}~9&9z(G$c%$u(RwzSqvhr=Ehi9yiQ*iy zfAZ-E;Y|8W8O$7}8&4W7?jnChYM!tfdqCsGqm3xHr5@ubNHf)Uw3}^Lyw4}dA$!b@ zO!nvzaOjJ}A=c-ejrO{t)~L^1_%WgmBOQc-&u|Y9xUB~8Ng8q406P-yqy5;hdF9TV zcBeEsZqS@KG%V9uom^bnGh~_Q5rH;n7^fdw({JF*nno5X(PuRYP~e#Hg4@Vq zgw6=)>opQq$Aa0q{bOWaW8c5uF2DGvqXik<&+0tds(G_aQg8UX+D=CUBzF9724$I@ zw8%)`|KNY#a-5f|a*m)}A>K9+=)1vdw4)QlDByHL+xg`DyD8Uhd8NhORle9J=V|HPXp{P|WG_CLkAlctN84v%q^!tgT;zl0ChNSiB41{VBRUEyIU{Hg22 zl-H~>Iu+uS?8wrsc7LvIRMO(m&3=0J9*9D46u}3-{TLLw;57lcmE7z@JOxT0BNh%0 ztpYnQ^+AMWN_%cWRaKE3mG(}Fj~)%2x7qd-e%-Jx8kpzrr@fGkD-#TA5WylK4cKm#K4gFkZY?PB*)s@O$qHlTo#O&Zg|9N-$sQtC0muDV%`A{9aTYU#V zOwV3*4e~p!p&h2r*#Rp9MxgLPsIAV>Rp|S%#)u{+l1Qtcw&2|8^K978Lq`P~@Gen^8}-R-=+ghDf?U@P*x&F>p|H!u-Arfc^9EY6zNG+58OTa>8{Oy3At z7OuOrYtTfHCRYnUikCn8(;VR(==Yt7A*-v9VJ;7O>u{d{TH&-2dYBhVHYZmiYx~k{ z5a^q<;$G;autCPzoz>xdX0Y+m4CZi^LYQjy_GutKfo#7r*ad?Bezgh`#OL`q-job* zArryiY@@}X;)wueyOZ4zq}Jh?VJ_Fr-yq>ZwG=+OINX+M@gnIp{$whE3D6ATf-Yhj zk!VUaK3?52N5P>RBzu)rfNOeC{<~Fh!Xv{9>fw>H8YajYDlt!xRT6vQK4R&V>J;ku z!RK}?6rx3puRPc-ya>seyMFWNv1rceMmZA$eZYrQN3Q?{?xeDjuLqz{yfoN&@ac zeQpXT-yYHc=)vmB#jFs0!LiJBr#+2$w-G?>NTG*IMa1tw5-*UL0ME}+wQ58EiH$b+ zwE(wf(i?3n*I+N`eg?^8Qj0sQ)U;gpPkpaJBMa;q2(nOt)4h2*&doifA%r~du+%fB zzeWas;Q977ex(ZFNACxaoou!rSMW1*ddXI;BZ7Ko7E&tN2udY!Fp_6B@%kO;pI+mv z?t2iQbFEx}f2WPie!>E38hvQ}$p|ky0ygIY3b2MSGugy2JQw4L4uAdO6?n-{|K+>? ztGzD`hw}a3)*@}FBxRRMAu1ue7F0ro>_*wLC4089q*SsNvddZtA&qrxWtZL9hwQr; zgE3>C&zOF{Y6Dn-igcG{sRRHcT`;fhX9qd=i*n)bap!C#AVML2=#e-g!WO=ZVWj zUj-ZGD0f1OtAN_4BQ1pT( zI`JFvjoI7iB_kuoeyC~}9dj)EB}lyqLv7}UmQ}2H`#Md?Ik|I;9iEiEE);m1th6(= z+*s!eXDy@MY04)+1Kz!H9~!UD{4V<-D!9NjMu`}+m*zV?SE1RhE+N+MYgH4>q50Il zLA_!6(Je7REuEd6?l0*^`*SuI4`)K8u8>J(a#E6hh0o?=3q$5j{i5Z&iF=S$N^cM1Q?Cg%FqrsJuYaI%V?NjpS6$64 z2DbKsFJFv|EhOZB1AL*xo#1t#g>cMHHHK!gjPIB*y<_6Stj3)6#@bNDQ}fs1#?0)K z6NR{>=&`NJvG?I-_Oa?KguI4G=)cEDGHe;ex+0ov`L7gChWeYyj}JU!GXI=v_tKo)1_ zW*FJ8w8)0AD+NuTtPYI2fR?P5vhGor_;GUd>4p?TcO>^eR(0jC*kT`0mVEIa@PR*G z=;zsbn;#j?1=xdtwk&v^T=b;w^}BTIor%ruTHkV@I0hr@` z{0bbW>>IwYbo#Jq%An7T8nVvbmo#x-fXZm1Y zUv(b3^CnY3{i{M)6s~Xp2w``Ep4#^L9uqN#4^+6n1|K;0OKz)-u~_C(9qXv z@mlg}GCzO)^FXKR<@So|XpiPHS~jb>=wsshc(jPgM}UT^Zupbk>#^k3D^3MDNhHK` zW}oAo!g|N7_odh>3%3^ARk^Y4to>5VNBRRitLkljsg2&m$0DE)^{PRZcSK{Q?M@P> zu3rArR^(<*D~}}-yZXAaEcS_Xb!GNNKS=42evWrx-+S!S?Vf1)wRQ0wWrC97&Av+R-(Y-0BVh_qxyQj$+{QX^X4j4DWcX(}T^~o- z>@ZPt^Bgp?OO6 z=vqc*X5i{BX+XtYsGYfUo8ED{(2jLAUd29urehne1yTi2-fmP`)fpNBx3<;~=l~=R zx}(zSAcG8Ul}J%hrUM?PgGQ;pTMaHs25Qj}tRLUHwdwg8x;KR-SQp^&rIgR4viHZ3 zR=~;+EtqTPKC4SrF4{|@qv2U&!{gt+{#>8JBMVd+k2Pk5Z_p8jvWn+{$}DU)3+#_S zr-yn1sWR#nlQiU!Fg|XkkSJF#P(3;^?9?^@cXn(skp>sFRWW}i{&0R`G4<`ZoIIed zm+w)dK7#xwBu8KWQhrXK?!)Dp2#wApJBV|uyC?U{Jx%^nt=ZLa=b$qwsr0VmjR|Ed z%=g`FPnHH;PUkTsKRIzCcig$e)_pGUye?a%%YI^7KcFcJ7LYm>`|l$GZKSR={2?8L zUnSzhu|3D+@x(t_iHwe-i@c21j}5rsNs7>zGA*Cn^BPhd;Ii}_+|herY-qg=qFzx8 zUHA1m6p$_8+!o{#t{f?peN!fbVKHuJb~9T~o_TQY^?!U%8Ul(-**{(YPHpSczH9qKUrL5;+aw17gv?dxB1ii?#wCesh|zMPoRI#42;ZMYWVK`llxMMXv3a#h(g5jU%7 zdYE!QtEHrx=10Gqfn`GjzM?8)H_W}+?(6I0fb$qt!DEiTxRUoD&vHVjH2s7s@=ia{ z$o6BbghLnFB^40xwX8!boC2(u&mzinhgMv90|NzS?}eEB`#?y)=F)HcgUu}i15Ob8 zN09OncM2!wLj(e!0JJ%G&UFZy#*4QN`cs=5W|jjx`a_~bs`5`^%CqHmqUK}B887w; z*S3U(UD!ne{923%hV>Nwpq0U3@-sBKW@#lO{2<3Achwfe8(zWx?BSHwveor0{q;B~ zy!Ss>!*BGl%U1(}Zra~hzJPI3zVmq0hftK-p4Plq?up}ryINY=oAL|cft<{jd=q^T za<^IFb|LWocA_#|d*Nx5ANR@#HXJC$0m`K1%qc9?>EX$twp>+cngY?$kI2U%CV=4a z9ol7b-Da*oqzVY1XbNk1ob;9JH&a=!iItTseKi7M4*GlP2@&%>Y*y{nT@HQeFTR^&t)HHnk~0vyyFgVpkXO20Ji5Pm!rqZoKH4wQr$GAw)cnJp)PUW zDZy4L`AWvxXh_tZ!Xib^DgkulQ6K2{U75k@w>qKQ%;@hb6$gQ+Hexix(Boc{twwG9 z9jH2-u}%rmh`O-IPyhs^q9?B9%d1k98zd{^|nB?Fd zrZZ6)+Jg)Q(B5P=>O*1EZgfVyrK_=5TbYQxSdWWd9kjBw-q#~9 z#HYKeP#yGnuUHps+kOwex4#?JBgRg*3mKLiH+rw#Ymwd*E5WOEM7c`EV8HGZDs$d zYn=u4k%oqmerhM-Mz62!LAF{`006%lCk5LnM`ED}D$h)Ja!sa&&=XHWuy>Sb(S=5I z^5~_F-JU`3z>b>v27(vIiU;witk`&>{tT(S_M5|muwPXF9w59Um{n;B0Sw8pT(2qqV{#7?dmddIo>Y&EI-=iWoN3mfCPkkw?zYc8DbkBwb3G zlr7BqFHwJ#E5AT#!1e`7T*z|T2GLk4uVF~8Ye8$)mI_(l=aq}s$77xcuKcYC-v5;E zE%Kv_TJ&hmO)=usv&3iESBI-cNNGMXBWX6aeLsjj?*|TsUEg9sM^d(m9_q|!6+rp{ z8H+GJ@QtC$j0re|>es^d%CnI*%g1rA42i{h1)0&@Owf69?)hCUw za*X+}F=*YzIJR0;ftc*$a&3PNn#3`scX|hCUd$V-fvG-Nn$ks{Th!~ZqcjdJ_xJ@I zFm#9U-^+L9K78r`Re%TsAV4B>>;C663^h(H$%E_z?cbkEt1`HxwZDYlL(#chd`I+4 zjby?ujHYPTrN#Wl`m}0jElEpSZZ0*yKMC?E#mb~R1FDG?==h|k_>x_yC`}-?!Lmm- zcY5SG-E);Ex7N^hqKj;aEXXpDH7e*eFJ0T(xUf&?Cl-QpuIOgys$*kggFW;^sBEGT z|4O@B7hsT(;pfHs!xsT)pV9O zJ2`sY&{9KTo%MVgi6$}B#J&eD84?T)agS;Kf}Acy;!7Yq{q~UwYsLqX?t4v;hZx{|HbimkR-&-+~7#Op8sm2lwCX(xjFilNoaOCw`i!^#a- zO5MU(l^;QRO0lbv#~N|x(SWW~xN!;8E7pLIfhnA-MkS11npe`vZI$dNA{g*0K;NJU zVb&6W(brH6C3+9EXdy7hS{Cnr1lId1psBEz6!lMAdJ)tK5R>4eIZ$as-I0HgrL0QNdUF2>%UbQ^?&4w71n+y2**b=0``hlXr40g6>KcW7IM5gPt7FIw4 z0d)I7dQwYCq2e7=Qs)h?7iC36JawrvlUX!MLc2qiFBn|B#buIgpdn1O3YbF>gjg|L5ST)G=c{(8&&#w%}G0Ui%rL{(c@EXdWdN^XxBf22Bx5{ z{+8?C7%0iwBD*wa4Bz(zIG8BGy~h6mLwyw2J^s!^&^xx&1NUc)`bvULZU@AMS+sr# z_x^lsY14@su~;kc0BJvd&S%ORp_ak#w!r9)*I!3)d9L|LI2fdOWK*z~;!UsnhN0j0 z3mO!i8q}19+-Rmu5SYs{LCc-2Ve6cb3jWnKkxHi4i75y&BFrI})+%7!y7_fFJs`59 zaK+;~;n=J)wA7_~VOPo=L0=h1slv{{n3Y8t5zt2R?@E6|*JSfujB zI<_3?m9qQL;&AWuOH#6E%)dgwKaw~;wa=Lw?LdQO0u38ROZYn6e^fTs7m(V$fq z17pUh3&RMQt3EwDRru_4Mh2q<=CqCJAZNg_-@&UV-O9yzI+m8~54fc~_ay6Ia)fE2 z6S>B)D>Z?-L*mdQPOm%RA$;!|8VZfB#!-v}aOaH3yX=~u$>PZ%zA&Y8+x}eXAoD|9 z-w@Hhn*TQ-Z3ga$H0mOJ&}DM*u#`E2tEJAsti<2TYDF+Ut??Y{g;9?iOgnQ9%E0`7 z2~E<((U#eAlv@XgS$(%#HWSW-IS{fRfjM8dB2YXjDbDL|1hBO-mIvEQ92ecp2KfcX zy%Og0(mY>of<=3Ax>>~h_f2cju&0MPNQ;!A^aR3MsIkF^Z9hI0HV8=_jKzj7`XOM1 z8iLgv9LRC_-R~+V#5S<8;aQsFG$WRO+7^mcxZg@myA3gk%X+ZJynW_5Ldu2+6AgT0 ze_LD07nrLG%EcDuFL!Vi{D2x(6Miiy$Ys4ZS)6dcRVx5(B$%wkcDeWKx<>Q#U(p

    5q+yaA)8n75jY&ws25`5&YY6{G3ZT7>_PB1B&a~Y`e+o)+lnFM`Kx>4 z$HHQbf-XvWg+ts4e-|Gk9GX>bKa=q9R`kOb{jlET6H5Z!#m%iOUYozhagrrknQl>s zAr%IgP9;!=grL=_RPwz>euMjri)3t0y2Z;eJJUoM(#z^m^4kqXiYo8f+V&kInON@} z>iu(@hjDG(f`FSUYs_0iPJFCtjoi=fJayQn{_*qi9w9ff=il#BxA}rXv=Js9sQNJ`Tm zL`A(t8)1DtWxh5mdr?Vy&@Ci%6EA;an@hq0bB-uSH#35;xgvrHib-D4%`S-O_46TW ztm^dpT(4XldgE`tet0PWt>AWUaB%mA+xXJ}?}=C-7C*{Xk@|8`zSG@1a906h=-I-& z4mY{DI+EMEo^#jM@z-Af&2kw83*q3Yo^Rl}*t##>OO7}7WzHloXT276LF20#}OzE<35Z-x@aC-CCubm}$y*A6M)X5V6(gp>6IL{M0RXRMEmq@r=!xBgS z;TTf~o`|I-1uU+Oy_IElVsH%#RV-7RIkkGb+n*ec{E_?lueqjN9Hc1BA(c7Bmw{3J zhTlUjKL2B}I{5B0=N=3`Vo-N9lB%#vF!JpgRq%fsKfe=W_-0jVu9rktZ2CJ2s+4{@ z9l*hEu z7*BaXc%t5^qD(I6>Z*(Ut- zDpc*YiLj4Q=*ODqCnxo0`>;909u2p{gXLDn3a8J<{&;niGNF|SyZpk9Pm9Jl*tvhb`z zUd4C!h1d9~5(~+}Vmscbk|@*@XT_NgSs6LCl)Af4)Wn&Q(SLnY`wh{SG8v=Kh&IV(g)v`Uy? zV?`Li7)%m%r|e=_;DB&%a@PKPz(YUDn^i(zHRZo>bWUVS;(U6thSnRjLW#}oNFjqT zJ>~$j@ovVih3;_AG{S$89*c>2R|(@>;UMqP*kfRHD>p^N0o|z%zc7;*-)92pkWqut z(0Cu5!5-7ivf3P1_H?Z_2jp$%PkADPFHdOc_H=0LNuf+856QuqNL_a`*Xs%HzfCm> z!#cP~a0ldZK3kkX4MzLD&H(VRi;F|rZS-k1OV;$-VzLrvv}r=4M62Dy0YVof>q{;- z5G#gx6~{f-c1@7ppD&8nK=gi}5`h}xTIL{o zWd_SA2S~YI<3z5((T9JEQiXo-`46 zA6NM@<+=h3?1h}Mfe}M<;qOun;)!&^>XDehA%W-Z!|PPJMGO6hoa!M>eb) z46oPQ{~AkS$)5Ol`5WZT9e+FvEr2_Lh_DL8qLPsW3&hMkChWh z#W2~L0HU#%9VW$3P|AIt_#%i=&dq8$rmk?$>)+0ld6S=lM)E?zcchLWU5l+UlkF?3{!(^J8j?d{of?Q>ry0Ei&N_o|2ewnI^(B}?HWfIDPsRk{;$+;#*s|*kPz)P* zmjeBye``1aIKHzB{~PLX4))+&dOW77-K_aDhxD^WInrVpkXH~kK9uUU0eUZZ_P2qu zD1@2pU@j@Q&Hh6Dy4&+a62o8FrTgGd`W3*!)RvXD%A^I*ztz-Kj#c*LNZtczfe3x# zlJ#igl#!`Uaa!ud&b!~#3>{#Ax3=6P@-V&yVOmkrYf$Vm@o0pJdvrxP$#*YQPBoW7 zZO)=}bx(Kah1+q?kQ+yD?@tENx`itxFUK_Vue}S>foAvNcfwXJgH5C0{eGb}F6fBj>^P zOPS&iE{l%M>$7lZk5~@HeGZrel@%+!mS(MgBLzDYDD;VU*u2Cxz}fw;EJ1#Y6E8u3 zyw@kC2Tzt#4%;vo_++c3eOuXo;2tSKT);p0xWv<;@=0M{#e>37BO=L_4G2%C@x~q; z7M72dWX^DU9j%TVE?rSZTUr=8tuKxB-#(A%3Azl!-#;LW!jL)@!&*F*7W0JDgH#hS z_5H6Gait-EMYvi@Wp3e0NXP?4+|?~2^?gevH@<3m)m!$aF5m#NzWh@dibc%3h0mS6 z16x2RD|sC4epgr_^_7_V)uxHpmP0O`o35soUN9U~P*8x81={nT^y8NDY*34()P#A> zn`47k*`)}hSPXZj_(Ywn(M+H1uVyc1mf?=FoRCUOiX{Jt-#|`HJt=n?(0v=X;yT5)B|6q}sR-PcJC|Ks=z|kY3qzF#x5SHv37(BsjP^7n=wre>^2fyjyx%Yt*E<`*M?b zw$6aH01YOj+z-*Lpwv7ap{4i`hV;4Qt0*YR;GD1)Jg{@{ljDzUn&KQv$%-q(hA!q6 zkb7P2KVnuir8Hq2cn!IOJ9@_-(lCNKHCoJ5(ed}OM7!RJ^k(8lLNn*|?45C8fiG1s ziC!|I3&Nl-e>>@ti#6 zPwp|BEh714Sw}_a1y&sp>x-17%|!fYq;(Rbh$~k5MM2J-osI>1C^q;E*F!MaY z;FkMAeAdd2J|DLzz|Y_NK3#a8Pao(O#84snj;0T=x+U;bxe@9Ji(6(byUdL}qg}t0 zE18g5948KZj9G&g5gnt&yu@|`jGy3{ogs?k0j%82Z{ELz&cLQBN208e?She)XSz`1 zJ&uvctICzabtQ0GmneK1dZIT`9o)QTPRkkDwnFDQ(pcrQ)E={mfAunReSZJvpK>KS zW*!W9ubx2Y<{b1Sdx}7ZG-9H`M5wM6+TVIL8ItqSsdRpbp1!%4Pvop*R03c;AC`w+ zH9oPm7g^o|z>AGk>43o?G@bCd811PQSD1!U=fI7MXeW3a^n9(gbh>dq^a=kt)$1?3{ zJ_D#({S8rFS8^6yvMlFgvyUEqmBxIonXl5y7Mro5%Xt>eku!6dIR{HiZTh~9AA{*b zQ+NaTP>vukx&fE1rqA^xmqA2=QuJ98=T(-w$MlqBa**nqj*9pb&yG0I#%b3fzQp`f zs)?w_!^CFQSllPU1nHb-Q-4QlGO9KCX=zPij_KYf_wBB(5eFACWv?r@q)ExvNXf9L zV{L1?{nUV(urnC8fXU-L&sS^$Kkvb`l6d+sBbg}EjoGNb`d7W$);|kO?w$s5f|Eo1 z##Rm8?<K)RJe|4Kp|VoXPv>6K8mazkP%4-kl;ugSI)a& z&Zio2kR$0RE`BRjkA?vKB#-s4!={CwR7#D(s~ksU9)%~e{_)h^*qvu#JOhE+$sY$pkswiXu1)d|yv&^jG@f6Yr@5Lq1I|KAumfjYUs#d2DhjQp zM6i|%fAiIuN(dZ*?hXq_=5JbHTYysqIoCa$*$Q3Ik@7Ql?+#?Hk2XiKJUSI3bh45%L9cj zviIA*_rp5DY$M<{4wR$!e{NHhdcS81N+A%&Z;_R^FAl|w7ueV=Xn(?lM#fUpi#8LO z7Z_m6Vx<Es5vP6yXZ|zfJA|&lOU)_ z*e$(Rp}`qS1}uUo`#vcF`u&^@*{EsSEb_SLJJmG{A99?f<(c{3(&B`DUn;h5uei?K(vaZ;WuD@~6yv3!#3H|S^&xS4cP zi2Asq%0}(ptrk60rU4MU;30`GE@66L9qTDWgf)p$gbZQTFLnu+6qgmw|yK^Z6wt%|J`hpu1fw z10IaBQ8G@A3m@*`l=~qCTC?xh3cI@vKCpL5T`e|+Fwc~ZKR0!g6y4c`m%pUERjbw* zraT(qqydJFHv{&VGF68w5C$d@C|8^4DAiUv$Mwv1$LtAx}>+)q(L_0;U6^h5x=<#8yF}!Dc`pJKDYlZ>{a;phh>a=)ncd_N~@HdpUh) zm(C?9q&gDbbY72}+GjINKdXKn}jsIDv4)G!`6JW4^-QXO@ta91a(QCYYNg0KZ zHiF*+#wI&@?VCv$rX`W*rqfB)eYZ(R!-YH<@788FV>*02O@g;7=u-}eI1tjGxSP*F zaRqU%LI%AFwe5qUzYE+G%#WFO?_3<6-FUY5YVbImxM0VE?C@`Dzgs>bRS&;^o&L~l zz%9u6)?fA->yff7L7_S`TV! zLK{yRZsR8H+6o*kQp7wqFdz$cvqXMcYOT{mn%`+}h*!j3&PYDUe&yQ*^KVct9|RZ51p?!@vOLg~Bv&r9HPr3~!A3qp!Sa(zx@z=5Z9 z4T>#T>W>>F_w4w?9H*a(I8#?aExu_Hs6N38s zXP=v>A|M^f&Hm;aQnIqoo|>u~BfYTyv;{EzEa3vI0)nQzB$3v0(Z}7UWWb>W)&C(2 z?ImO)8BAw6cdU9IfF!m_#~EwiJn-kiEhdhJUNEW82W@z*dKp^SVq<0|LX>ABRnQw* zAzJxxcViM7!>N(1n7@(S6rtTLyhf9UxxZ|+|T9T z*OuXW!z*dm45+cpl?*55+$p1#q4y#ld|L&V+t_U$2LZj(?>V@>xv(rXEe;0~VCraF zHupfrpBpogg&QI3H=j*+Um>NGR5Ixh@DAWW3F4Gs-{do(e9TksGfWIF6 z5kEW~>}(qcP#$yc6iX3?kFK4?{(>hrT|3HEpP~A@nXU+2C{3*>NO!U;>r=c+!C+2` zDu^H#-veFFKjJ-{u7YDsYaly3E86oSjj9yg(p8iob&$b^@d~#?J^@A%!?5ol>C1rd zHj6pa>QpnCo15@nYp?DYsc_6En4!A3Ayx0%gDsJ7CNSTlHy>TFW;(i-lRVKJ*YFx8 z(7qfW5bYuJBVNWqGH0QoXSMeHA5Z-R_B)8$3Y-u79qhQB#PRlMizc|{UEHgbPzwk^ zz?oIN+$LcR#F&v%y14m9@y9KRT=ZOa@J_%1>9sSfYu*AYChyzTAAB=sp6an!9I7DW zs$#&ta~ysOi(Ombm`2X_BF*%Oq|42&arcq@0sYKL6wDkj{JjelH(R1+j z(>zi`@aA%$C`%!&XgavmGW_~&Pb^}E)u4Va`O6VM8st2sSV#W+5_T6#+Qn{IMkz8> zxdBd}J;w5~yk52Qgbm=KYo~+%#ng-$QKAIyi{4v09KFgy8iR8*tzJmd3I-GjRdNEs zLCy_aLh)shT5oTWO=my6nUpg%^_HE1L^MYeKvF_@u4rPN^-H~q!mFDyu)8M!Xn50(2&MZWS|U;BZxIyE(KHFcj{N!rewG6Kntu^;Zi0d4rl=)?0|D{19?n%eI@MVRXnR! z`;gqM!c*#Byx#tX zW_t~2w$}jCVLJ}C*ML$L{}Tu0u9z^$|;*-^jQ{s{2!Y%kPyM8UIl{x^9+tU#LYN~S(02{J1E zyV#C0H!o=q*@5ndwd-sUEhVAy6@+fAJk2ir}3yRkwTY&X{b zC84I|(AL-g_lYj%?Q zHfG9q+rO~AO535aoe%#ND7UM|cB$O%58GXo?jKm7q51zNA5?dKx2oAmx$o!1%=Db( zD{Yi#yHmf6l>WzPhJ*SdimjhG<0y}or+#&}ZvVo6r&-?HtMvb1XzV Date: Sat, 18 Apr 2020 09:07:56 +0000 Subject: [PATCH 367/743] [ImgBot] Optimize images *Total -- 48.34kb -> 24.21kb (49.92%) /website/images/clickhouse-3069x1531.png -- 47.95kb -> 23.87kb (50.21%) /website/images/flags/tr.svg -- 0.39kb -> 0.33kb (13.85%) Signed-off-by: ImgBotApp --- website/images/clickhouse-3069x1531.png | Bin 49099 -> 24444 bytes website/images/flags/tr.svg | 9 +-------- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/website/images/clickhouse-3069x1531.png b/website/images/clickhouse-3069x1531.png index 1bb110a1fb22db6f44720c1480e1c7d03a96d0cb..f143c1d15184aef0161abab41e901b91b7435af5 100644 GIT binary patch literal 24444 zcmeIa2~<B!GZ`6CgoA<}t)}fEE=51IQQ$ z4ly90j4~w(0tx~m0c4DffiQ$QKtl3W0@~jD-gm$Md+)yY?pptLtzPuWIaRf5?_In0 z{_S0L>!hLXs^uG&i;0P?($_l*gFld%*pJB4@4+t@Pj{ifpC9ZF85|N5dl9;VcWyCw z{`G<$Y#=7)yF*Ou>J2fmS@6@v%&-Wu9(CZuW3X1&kg!30{jvdp*8k=@>c8!dTk{%^GUvYhw_9E!k zoh@6o3MS|RRw}>iKEK*%%uavk*X_K9QtrDf0e2+lLq-|s>$~%4-J@cEESA;-56iCT znYoFHtym}g?>n)WxL?4JOWgGhjxFi_VeQg`+Z{HSxQdBc_URuzg!OJ7Y8$xrxFYA_ zsr=B#4j%`;K8h_K> zLhriJccSqEw;^I;_uw3luZ{og?o00r<82HfE)x3ZZyKX)4vLALwTOZP!N!y0 z0UrOJ*sqNdBADq_M@vX*wV2qa2Jraz#D1OWH{C7t{%?r=()gS17J3&FE8^q7A@)Br z)Bh>4lv}I(D>3jNNL#39{{dC`pJL$uUE2RxRsO+D|2Osezk3D$m&Nkmyn_EZV)@Ik z{_Pg>UpB1&zF7X14Xe;Dh!*mHa0~hOXZk<2-~CJNrvHEy|9{&ITGZod?qfS;8+<=} zx8Ql{p@6To;HRN%Qlj85p4dGY_(H&M6D~R!g1(#gee_VMou_9 z?}Qc$xMsgW`Y53zCv%xSm$89e6L3O!O!nK3vrIGC|s!prb_# zjN}zxh!24(K>BNS?^sK$RNx>~617@UvWDNU3CJxcWjuGQT*=0&|6e z@%2egqn11OlIhbygxveEkmU=TRE+H}&djP-Ql-*FWh~6r5xg%s6#DbG?Y#@h<2mc& zV=7|cF4H-$50S#zEeL5;IcoUKRqkmlWJycY0j&~$7Kt*1sQw&7P(F>UPQaeyFjZtNE3?4H#{#tHIj{Ij?egqw6D`Lh3;En-q2136>%8vL(9+dBQ z&SBupG3S<4HLVu@^_El} zgb0}XryghS5Abzi%;Dq5Wg)U%?z1*qp4{IJQn%{_Nu`8se--vxg6ZNv7 z=*j0v{wG*dRR640DmyxjJzgZ7xjss0L_y4RukHikiZ*L(Mj-H>QUffAg!gSutLs?3 z<4<*@yFj?nRdTa?>T5>4+wWU5q7)HZ|IpDZFQ4dxo)%Vw$IrKEQMrEG=hQZ{QKNRD zt_WSFQm#O>GUsU-Oiqk!%Xh}@i@PF4D|B~i=(&*xCoj{vSkonK^v&{+&34sOsfr>e zIZ-4#Y%H!!D@liTNuT_iFMRb>4C+>PDb^R2W+L=$zRSA53*GrA$;9wk%U`>xQN`@^ z@mQ~dSC>%|T8bX8{$>@*PR15YS(;40HpE{W;OzjXpsv)5fV(k8L-t z%jkZ+v@qut0r_~6Q2*zR5Tt*z$b^@WX#Iws{eXzGvNmOQ-P4T_elQ<=CVQ4RPN66R z`8pl_L8L1XZK-nmH5$*VtKao~iBJp+%ro}Wuj1Rb2b`V@;_;mV+?@Z2+J{hmc_HTC z>S^<&8ya^%cIV2Wtm#>5A*y~QA-|06Ne>t5%z||=PK)((VVF3McqV zlyk}ZFzI!V#1;Q!G`4i}KIiNF*4ZW03Ff%Z)U6kF@)jcqeh$ARm_o1)x-Mh!r`#yM zRYg9G)30iHaaL;UJMKRb+534Y;n>OC(2a)v>^+2YqP6*_hE~5U53k5tiWV8z1u$Vx zRR6EC!156Rs}~@ME%Dq_kdUQ9;^w!xBYW8;MByhw5`e^nn~g_QiTvMxwzb635E?2- z6m|D?^=;ofFh+G@(w_ZI2W0!0q?sDQ=~6-ix7AG@))x(!`168mkK0tezV}Rfqx6VK&ELIZ9SJS5Y?u3!)h$BU1^-N=wdY%#Cxm$D_)VUthb{zN7?yo3$6V2 zEwm69`vvgDYWmofWCuqz92DQ&c&CU-l73v2)F)J5YO{Sg2VWNEAyPKSiJfU!%}xZ&)GW+l^MASDbe{Qtu4ID z-}i24VEw*i9RjPtQMx$P#pL*$2Kk4t?P(KBv``T$2=qRyeB47@?h{_ySJ$(IxM5rQ zwCpGmp_?!1o5Q1qT?WzKYb(cAe|o;_w}5rUHK~WEM|q!9kroRB{e}6&xXDeVW!NQ3o+-@WaCDmx3w?P~CZS-VjxQ)cT){Y4i?o8*zrVO{+`T7qWnS^vHDfW2!=N ziV@|6150;LwgKc}28@-2|BYUTV`wDYHB=bmFY}wPyN%|W^INinWMey}0vYFQ<+_cx^tLD$^&!*i-F=YqPdoqr5EMm@>W?gs76=O$jM?on~#9W=QQmv9R+c9D@X`_Q?dn;(GH z2*f;}j)IK6WxY#L*~KxlbJZL8HV5=54ryRNBJqAs@gMFDXr6F+CW{#6W~1fHS5$8k zw4lA@zz!>t_*s+$mC82m*yosy%5Eqxi7E^#3=ocv^X-nwAU1oevIAGiltXZ`rBM3y|0>+{&X6Dtk`gd5~AJ0fFvujqz;Q9`n_V4tfS!f_zt#@W;B_`6f&1rhc+4Ku3vH~#OQ?dRwT#A!S-EL;avX%3+|g35C;e?P~4M5`lEDq%5Egb3}e?u zyx2wzCerr}tl5rGRh=?pm@l)N*TAf-s;rLz2fVIdNOKUmoO|N7CDK)9i%<4U;wKTh zi32N6^_-~zWzYD>ZHQ~ukX2f~`freBj_+|P6P@;KBABYk+E+!?;Qn zk7@)nfMA~{2Da+nBR2Cl=_b3#C?Zy|tK^$(ezHhd;<#~MqsU{#4NuaSk=04|4iLql zk*>Cag+oc0J=V5NMd)$l*eW#Zy7NyO=OHcCjhKweU)M)s1vnC!aA&B%dg6ElcY2HG4?*~>q^a~At#~Sa9f$-lX{y$Z5T$X0LpIU8*!wWLk=|dQ5a4yI<%+N8YlDjo#;ytow)@V2&E0K!5dO*^?N&7m>4#CHMI#q> zcLj%bg6d+>99nsYPnKD#?w=f0`gNr6~XcX+v7&x#Ah7Yr)K8M;X`W^F-&4JkGb ztbrL1Xl3K@Kq)iAS8!tFm1d(E1;J8}xX-eFK2*2A@tB&2CEKOd8CtbZS!$TnQNpWq z=vk|U%M7_|I$chCpI4CI@`bxqD2ch=VvE$8tcXP5yRyEv;1)lA*kQe(N+65{PhJoW8eUFpd64+u6yxPN3+C8f&N%k@2aRYfDTUS5qBnBrC zSvuNG*O?hQHdv>6I=+>$Lj2~4*JD$zN#cCZo3xR~&MgM@+I1%P;C(uq+5<8U@pmNI z^6vux<8FFy1Z>Y7Wn|36dnahgSR(?+p>nTUbmD z#9*sOC)taabB!oIxm-McHby&(B(u2*!^*Y-tAGEvUUrPNoxg3eMUP9X19UUN)au>v z1l|`v()&LBXiC1IPp~{;DgM?5k%wPG*@rw%Kj4^MlLb0dlT?xQSuw+`eA;V84c9;x ziNA9^J-Srg7pj`}eOyDb50v6;@J<^&IJ_TGQYh`qHQI$KF5fkKpo3c?kn^_DwKo#Q zV8Se9#PNx>e>i}C3qZfBy8_7hNkc)Z&t&=uy_oi(i}}xdevsBg3T0AX197JBjgBvjOJ(xIpP{B(SgS6g#5b&xj(W$}W&S!z9(oSsZRB zZZK6up(cAWPrSSgTPhy7p}aFLkQdrzh`|+Y%=K5zr)lq>9Az+nDyVjYM!nS1=$Ah= z0I&AK!Ggg*=&HveVSUv>soXd{4ka*YhQCqXj&?In1u;|zx=WlrJZ<>XP&3WyTFrB8 zEF4;8m~|)jhup)VrS6Y&v4<6nhjLrzoyY1&7h?0nrdcxkLnW+=PJwZt?k7TF0*MyRur8z`mN5+#47mT&xO_T(l zos+_e_GF+WtT5!2>0Jct;~q6Gt%~@wf!SyXnlqc&h$OwSw)8S%3Q`XsRs%G(`y&*4 z^O<|M3z42I;Z`AZMjpUUpL5y7<%A_YM5aQdKz)yj06zzyqr81QzsYTVZ&CHm9yg9# zuoAc~t#%N8rmro~l~m7gka2zF>1Gbi+s=?!Sc37^qY>r7$VrUptyRvK)29-bZ>nvZ z)&1~0hLzX0W`iT;z78}GSO~zUu*6(T7iPGHNwH~Z-}&+Qf+V7WBv#;QsQO`VPc@kJ;uK&YqcNcG z3B)Nh)hZo!oq0QbaCi?AI2VAGUt8l7-Wxf3u;lP1ur5oQhHe}7t^hM=P|5yqSMG?& zvCOYu1$`M0}maY*H~254?rfcG1E(WYo$et|}W7F-BSMI$t%ikz1^)7N;( zR*&+V{@qyVL**wZr-$sUl`hCc<#+pSN;yF}uQR#M=e)n&ERCl9*|I}!!9fIm^Qf~< zHFKyPueck@uL2tje!}|M6?Cj=HL2?<%9!PFxckYg1br#mXJaSD{>FI1g zfsBga)=g-jDX&)7IePyfy%!05OW2({(z;ed!?R*aIP2yju;5}{Gvk*Qy$t%|8SX-^ z$r_KVP5{ewUJeN^irMY~-Q5^TB+hmdPRmY_qyQF5d3-cOt*dGYEx>I|)@k10>Vl|k zNHFwCpMs`{`*sqS`#+*P)NSkg10}(IO!o+A(q`O9qz|tixFZ)x)s{POYvVk{YG)zi zxj|YbF*u$m&)Ga>kC7j_39C83l%9J$pjFX^BcJELjY`NR!pHI)W7$!2MdYo3MZk{? zgB1p^;;Vq+JVAN8WD!*4XFW#ry=(!)tj4791hd0-E;7-pfe#b&wrxXZfVn-SwRG1F zp#!=JM5*C|mKQa(lDzMjM3ltzVM-zAz&#ip)jT;h`oX?csqWD(#AMye@O^BR_30OA zG4oOl0g-os9);OXj@*rGmGwY%*7ThQ%@fj9bgs#$g%Y#P*x717&K&xb zuGYc|*n*l&n~C-eLV2{|1xaYiDxn{TKM>_1Ui&r@Uny#syOw0SsEAj=+t$6zbAFC- zd$q1}vGp>%F5__d__9M`ODBsH)$ zENq^UL>xljjFHd!dCE^`?6COz)5U(TU|j9l2L7V{w>}xBkw@~XiSi@e(AhUlv`<|{ z<78VPP7z`f4%d_EnJ>L~xe(=*B;;{y)1e?V<*k973AN;6U!BE~5K&w9a@t$U2e0pp z9LE+`D}Z#G#~2~4?wvW@ZOS9Bh^K%g?ZVcPQwXD8Bw4n+0p+*IGiX1PTXD;ae%`&D z6F(mo2&H*|mwSYr7FwV)Mh!T-DEAP1h1#`{NLbjI@qS0mv7m?~?B}UvrA3zG@*Q^6 z!RLpBak~&af*@%W(k@Hwq8)-3LoDr|U4M{PM7B0^QVA{BP~}F(4$$J`r<=)0MIYxn zE#Xf+K*;peH(oqdJxba=5ccTYI>Yd?7CgOOxNP+Om7VOp$Re_LQbro~b}P;KX6U_0 zm-;>P5)~>DZGAqK{X(#fow8)dNPS`G`H+X#5&AyeO#k8VCnGom%G=u|5Zm~Nh${l> zjtNfjVzMG&28Ds7`4_w5_$^ll#pKM`{>~aPM%*NDW!q994|hdk|-BpA*I7^ zTp~z4wBXPK=CT+{BM0g>CSpIYUlesMc>eC}8P?dP(UF|B#TZQPcBR(XQ%8%kWFmF7o;ZT0=59^N1jykt- zGEu3nRtSO46N{&QZ!$MD$b~3xi=SUt7}e3Eq&w5IrzC?QN>iSz^7N}%AKg2?7~BKf zO*?Z_Ngl(p?}f1QJP6l3BC7`>X1`30q{>zTUZ;V@EnYGA6nojEksdT=h>>@{AL1NP z>3AXe!4t!;&BEcHB&=+mc^qu5^SBP7>pZ!hh}}aHSkEq@CutKoF>q+~5;Q?^*4`g^ zlBPAiB6%U6K%tDUK=IWcg*|!u5i0*MWfC}7c;&HW?<;L}t;Pmk$s1DoOwo9^Xog#g z_-n1(YAmp*MZVRhoycYwX9vPrKrJG>0;oj)U38f4hcGy~YP@F0+Nal1lj#L5MBn#1 zXOo;_nIx}0y76G82<>&;_XzgB%cF2ujX(6Zc=zDvL)O3%P$q7u)Vvl#gk>2XTc>Q_ z+?zhx?g+v@r7kW8!(P>1vy=H0fY8F4KL9@`Vqc&nPXFTM=lvlH!CGI`UyNa8(bnwZ zR*L)IayOkIaSKlw$HOI#X1}LfV=#NZ15Fl$q)$1I!6kRw_@09Bc*lkU`nEnjl<`#g zOyH|f&I#VZVO2!?U{}7YL4+H<=$i@8ey;39I_;XZz&e|XiBUzGVIxmHit{nxWp>23 zhcosU3=?Ar%v;V5$pw)UsxLn_F697&v=D(Su`gb^L6m2P&<4o#(>SnWG=-K?2APH0 zqg^kVM04L;42D(L6C&+(owoSy1*^XSW|@!f8%hA!w5%VKOm7tL!SCcORX+(u;J>GR zm^9_Qp|KNyVnmMK4jRWRc3)Du`Gw0kSbla9c04zejY1%6%s4`C1b~tm7}!$!357sy z2F+?pUcI9{y86USq)kE(k0?Zvi$a0YD5F^ii2grF??fO09nUIwWJ-c%@Ml;px*gG@ zFwF`&H;+|bJ~u(9r4W;Q_wu5F11kv`L_H3D2xQz}eu-sNPba}uTE>T4GwX@O;v6TT zFNX47MucHp3sF8FsJK@J-yDU?es#sjbWJpll>@q(kr!WZCr(4hbYM+=%2xgmUDdT+ zAM&vA3MR(*+;Pg+PvW1ae4mK@7+HbA*r677=FD4tZZu*u8XD>{_BDa~V!M7|}{!}t`#mz6&G?o4AKb$^&8<9s-m`a-`)KDtxt$?Al2z{lKKjZ}< zvqk3eqgr#D;pOwtsNF{DX`B(tBYB6fT_QbuIy6Yc(%*OXebf$-NW|_d{3vPpl^YbrO2DiYC}`zmVxxm!Y&OBZ}1Jp|V>wjLc7v%CH6L#E+`;D-UA?skH01oPx4}7qG?Gpf+=G zp1D5;6r+Z8Ou=iiMHs-Kp9|5t@+&!Oi!MHC_gk$`xrUJk04#O_z=EC;Ff&VV<`^m> z-8{+ecOC3zRs6Pjf+P^^#QI!kp%0)(`EcwmNPlQ*LE6`VUM6oxVsuF{MLLGVMMfRX(eG;kOH}H$@&P=EOgpMEfGx8I;OV$sDgd!eQ{eG5EeNE+6IN&>FOBiQATB zyy*fi2;xD5iNy~a#6b`+hDjc@Fwz5*w*tz0V|a&YT-oy1XO>zkEMycSfNVM`QNX@f zAlQA!8KXr%`~09!aSf^UI(t6~9$(p|;OO+8r;)~tUNgUa+YySL`}8TPf+0PlGdQ)r z0PnMpo&6bsYeOG)CDJpe?ZUMkxq6gcDk|xq#$mxju0@STImFjw0i~>4tvZE8)gN^( z;(`~#z%dsXk%ZE z32B6m#B5Yl(x*(DKGX`I3Y?hrB+VujKw=^hHP-iV5+rb!CI6uMw>Y2$! z7Zy4e5pj>XP57NdOVte_l~QIvr_NT?@@!k*Z>W07 z1TH0g@qO63fxg6a_84QG+OYJc7xOC_`Uy`-8Cy=PfN?VW%wymx4PW~YC+Whm-p#$u zQnHsRzd#1)Mx&EGbW9M!&Mtg+V?H&LiSCfh-#|mWj8K!+8N6W>0Mv!X$`MXhz#XUz zV80QxcVLq316?|DIX~)5eL|x|+}IPR$k=~^m%jtqJJ%AXaz&DHG*1%=v~Vd;CRRkt z!^wd{Ma>wbV+Ywu={*KJ7B)@%a_jfnbk2zaGBJPnij7emJnF{@g>kou8&(iD=;Qf^ zp_{9Qt{E9jz6-H67N&4S;b+qghO$3MKY?cI^?M(S!}3`vq#EY9JBlgVaQPBCzptpW zg5{$|BbixPi^|v&r0q$i`Y;~|Rm3jYrmlC^T?cE4Y?M*g8Q*&q(Rc$kD>cUv{6-;zhD>t6E5Os3M~XJv@6 zPEq@Wo4LH}BPTOPsIa6#m zabe6mDZH(cXYx0y&(E+rr>rC)ve_7_9}Bx`gpto!BgkGa4x&h1h?3#lAvulhLqn;b z*9xlJ-~P7MXlaxX_7p{)6+3u4ItntZ{Y?R47h)qWW5+!RG$+AZ!bLdGtuhu(gCTQA z#yDFO=^szMp=EgKxg87@?F+(*Mi@+&t;5(@*MZ$DpJ_($ih`?ToB58*gifmb+%3|L zkA!z1>#?g9Zrp!8b-2@gLFvNZIBTA2(z4-1z+s(juRV5=$XH%D9rGOkGGX~Esr9y; zcyBpDJc8v%EBJ2Km)d?nNoa?vYvt%%1F7ViYosH? z?ENcpOny)50Q_v>iV zO5xD=9Ugb>cF#_4c_f6kMMF2F9QW`__+$ZD;y(Qnj$lVF44641cz(f8 zHER=gwE#G^&seU0UDqdJ*^J1XNR=9)>)q;S+JPPMAyW4wqA6$B#?{o>l!!*olb-#N zl>6S5Q;GQwP>p!!p(YI{+D~@>Fo`z*LO;FN`6m<4TZ~~?KEsC?(8#!fl9;{o2Gp1^ z0*68#K0&Y-ccd^%l-F*;g!FS{-)FRf5{KD=Wm;?I5kTOAF&zYxIwqsm#@a(?y2uDF z?~St;%A={PGXdbFEoHjTh_m`(xq~(1Mmq^rFF1@*TM4nYpd+5+ME)&Yb%_-e(vu^&nu_?M++t#?HvzbU7?h z1s}RZ)XU)hT?hD)1}KSqW8)^tjO_-LXM1!QZ>-05~qi(-wAMX{UbYgE15 z>_DC~XVbjhA)as0h>O~&hjh)N7kH?RlK@i^ zRvl*Vsdr3@e{o2U9S(Kd9e*aCG}YS{X?|;(zUPEmF-I z9efhR{|=U)3V}u$)*-@z1>UJu|GehpE)Q#o{$gy0Ut?f$ktZ<(%wY+PgG#t&9@ep(gLT#M*s1PKWj^fy{AHD2amb z9*;y~FahJ7`;Js8WOIL0_Cvd@1B~yZ2RIvQmgrOZxTqJ*s}R+4vxwq~N$r#9p}okp zs7wj22T=`{=f4PZ0ofUVH0P^TKnz8X8ZTIW;x}7R(2=|q$#*Y5H|XP%+_`bh3*qU$ zM`tsYMq7N)L;H};Tq~)JL@ZDGYGt$Thb;g>u`MC=^7IX3=ORnuVpG=xgv5HQ+Rd)L zQ)dec%YDASIMFSh{n^L#nZ+@H*1on6l8??fzd)U z8-Hems>)V@Z;0I6(KEV`7MyTwxmAzGjH@t!110cBZ08Cio7q~mULZk}n^;-6%{K`o zJQ?>7>_O=48x;q*T4IOajlRx)&T-_gRAxdj?!;`#5{7|`q%cz;>Y-%+z|H)t?5GWq zmRTik|VJT;Bk6zopvW=SpAu z%G}n_i>wtDwr#GD2Z{Me>QKw9bJ9Yg{`6U6<*o~Ap#l9l&s#v+SasJ6WDLQs<0rLW zTv14z9eMQ5_u8+5#vkJj?2K-bHFkn5*U1EStg9mnodDsiOxoT{4g_Sz8m(mep_j>+i928}+0uXA;lNZZ++lH@*a#2hi2FA8zz}{@PLop3^35wrmH}SYbEbpU z^@;RtGc{m!WH*nW*T{=GDFkwKA?dmc%_Req-rT;b9$@vSH(Qk!g)>i3rXwh~sArBt zcpjtM9Cw{#J3V5rFj4;jWrSATNm)$0z%}nW14Z$jnN}uf%VFv|QA!T<*I?s`P7N@f z7@g`UsMIi|d|I7bEKRmk)dwkWeCWx1<)|>2#Lo7KZ2mmnNM?NUx8yh`b$D3GB2EsP zvATwkff@f-Qyhh0=|8-XH$r)w?AoB*^zJaOcF89WC;~`N)FKu5r}u;gcoz2>P|mf1 zK#!u+``#1nI1&HQNO#{Ge(X`z(O5$HEt{IkMxOKQ9~e`8uDmA-ZRO5!t;!2?x2Hrq zr6}I*QhaT^-_MYu=JkxGv)7(v4(6Y`QRk_7V zyC^fLNz>4Y&}X_GD(K<&d3&`6?%EmQ$6{MW1@R^Anv#V$+GYpd#V*Bnv3CkU&3x3f z&Fm{WVx6(`q9J8kLk!jK5e4ge3h0UwijGJpQ z{JWaUb1^PA=hI(A^=D4C77z2R4+eTd6V+@g2G`(6@7Fo5T^OafJP#wE@L<`L7izL} ziuSJ5q${g^%@wmDNnVw6eSA2XPGcM?u*}rD9h6WM8K<5wp7o%=EHl)pUSZk1;mL@M z$ta^E=-gk1G>}d|T*x|%V7Mu|#bk;Y%-G2v@4`Z|GwB}_#LPH?l>xS+UF)1_e(ipV zMigfPqFwizThWl8nXtZwJ)vw8Dx*7@p)d08^-zk&(22YFirn2^3Glx3SA*-Uj!h0p zMDCNB;(?&h2=eDITaSuJcDNs}dY93iW;7-^W^^r}0gmFbNfR)cF22Nr^J(xVQseE- z^bPGFy=`3uB1+L=X$V~vN?Dms3*|^^jJ29RD8=73j(>%f_2Gf-1 zMpNYyu^DMvjPOK0%HOW?C}V13sesV9X`PUMQo?uvJ$JFxHD!er3Ur*9Q`GA32Mqa>4@7n^o zzi!*zWwnbk`>TJuWrQP_`T}o%E>xq%*!&`WyQAW zYf{!5Ui57(!w5hplMQaPi z#2b}nn>EHKuLwZ;(=}2?drOx04w~8eUVQhZ2;y>58h6C=6D&U(yN48aplD(r|H%yo zYJb$qC zB#$(%WJB4CL=qkhwFkPgCj1SSF*>bLNGEf-cHZ=Hs5C*xRw-w$aNzUin&=t{&bjO5 z<1M+7RL!kQVVR?ip9wllvppYvopWoNdHgp1dbp*pUvwl5zR9rhy~Fn7%Sp}O4uD-1 z$W^;Zbuh9nT&ZJNlVg3xY1~QklrE$)w`5&j5r2PJ-Sq0YF1MV7Y*2eio;GpZNZtnHx^7y8 zsVwgz-b@j7&p}z!`KmHKl(r^k^tQ@lHUm_6v+FL~=X*9t=^7euac@uQ3gea>og%{`?u@flUa`^qRu%R+!hTie@Ya+1 z5Tb7PvNeu##uQDNp5=$bc^F*BV$CDylHG8Yf|SDq3rwAt)*S2ovHr^-(oc zM=mpD;;7XpVHqofocA>Gx8JDqdZyCCA!#Xq4DZ9k4ukA})XQ^o$pGc)M;s zQNBCorCsui!b+cWf75}ha$%xv+p3}@Vsj2++p6YL4+^>~saHxPVXp5~U(lJ*{+E{e zcpqpNKk;-~onLO_iuuN~2WLFS94pGSi0w zLNB5wpDL>d+oQg!Ooh&#$(Wf0H%nCg8P&q8@&WqQ`KWr$DF5lBP>(*v_R~ab|32pR z{PR2bpEkS$nO94FfEW{Rf0UVr!r|b@Mkfq1(~qa`LUNw9YWLJ;Rt3FP@=4l-EcY3E zus8HvcIY(^H<`LCkildG7R8>k$#dC0ADz`dQGT9iHLmmHDGSJPCW$az3)0cWb}N#$ zR1y)al-;u7rkdqva&ULCD3ylOuuZGa)NtHTr=(#oe`gOqBmDo#9#XeE?HeW`6L>50UQH6ZTCCS8k%R z>w9*MrjSjHDAOD=)eRJ?OFVBf9Jq7#W~8su!R6O$LKe5WpMSsqL0yvboPo$q1ioEq zjdEd+zo5gfLovkR1jVua+7Py}OcNYIdwulI6qlG(Xzd)%n(d#B+Q<>5ll4$g zb2H&gXN$P?Q zNl8hN3W(CNO8P|h_d9$20m~$|F4!wUVm@aJ2TLiR>}rlOEQI}ug~P-O+B!w^^#>qP zOdgemiG&IMVqDhY?Hawglf(VXM+Yhy z&D}txp`LdKcvdYu$BRY5TkAFO^G9eY-}`F1S@(I#tfMyh128mW9ezwLy9z|IYa;C` zP-sfMcx~DrOEhceI=8aTxY1FnC!~AUg08f zjAgoWeh;MYS#=@Ai)VFV%#>1+h%GManCxGabq<_!wJ=)Q56nVwP#=@*;>6C-P8@4m z^17J(c^m-%ew1C{irmUeAzS*>J-yC-(SrEO@rd2zVfL<;zTaVa{SHIRYpUU@Cn&vZ zg%E+d^T>5AZF5I9dy&qd;Iwa8t7BQcb5tDQnSmP)W99Hc(tUVGJvX3e{M5WB8oWtu za;AO@hLP}Ns7-R%F6&#f4MX^Mv}UUUS)&b@!21o9Gfd?Hj-8LOw~yyC?YVkd`eME< zsfcab$-W_qx$7?cK^^Xo&b9*wPcY7E*@n=PkSxxQ2| zOw%;|;8mbzr*)=9nLwiY4R_kJ_p@hs{sWAH>C(;QoPq0dPlxf`i4CF@**4@fH-ZJ= zN~)kIDCve}8I=OOPagHTlI*0M8?d z%5Is;r&-$_j4z`ld>}N*(K`-A?$ zjR&G=?mu}w!@Sb|>t5?5x)es3-}|4s=Hj1nC&$0~?IR0F(cj&+^{;*_(f`W5V*jOE ziTg`V){6!0we=L;d``{fI8E*;!{2P#v9 zSLcB{RD{o>rt>d=f&TghxKjY!fVXd!$S zHU0YH>t^=9Nr>=3{@+sc@9~UK|1I(V-`e~yOiv_d;q8MWnJk*;J2+x5x~LEw%@_U; z&;)Qd!MvP>w+@PAvY;aKFThs!^$T!0#Jrq^cNvP*XQAo*3*e7^{X%rfq3}q*@Jd6V z2f}BOiY(lI7$y2es0sgig#15vCd|+MlbEN+67U$tJd*ILOODLIG+}_JF378P9w!Mo z{MWy~IPez-{^G!29Qca^f6IYa4SXG^p{Em9$Lxgrgsl6q)9x3~x!b5;aJ2z{#P;sq zyIXPhe#JcpvAYkd@86@oS4Cm>ZuQ-}e>mDI`8N$5oiEzk`uyJ;ct~AL0S%;tBVe3u s-M!Aa+K3&x=zQMhSAEBGb~dohxr;t7?`(uu;*04YGd!Ah`0Ta+1KMZT<^TWy literal 49099 zcmeFaWmuHk8$LRSiiilJqJWeVN(<68VgS<8ASvC_H4NB@grIbafOK~Z9n#$(-OUig zFq}1md;j+ToX_W6=Yw9?-W!Rvp7q52+|T{4m+h-4FG+ln@*)HRA(obU{2ct}83b}Z z>B1TCnX%covfw|2c2b%S5D3v#>_2#r$mpBklN63mG#r&}O&nbe?2RE0jcg5#>7=a< zOpTu#8yLC0YB3gpKoA zWVZR}-;e1eCkX!ii{Q4{zaKY$+5dMxXu#hs{yGW*`I}J?$ltnv`~?tzl7EpF0{Po7 zAb%Sx1oD?2Kp=nV!C!iS<<5WU!C!iS&4a)6;4eJ@Y4DdG{J%#Jh9kY>8h3&_ac zE)jwJJQ+K>jVo5~3+L-zj>N70&JeWhh|E8atwsoB)70~?8uGU}{r|o>RjrQe*r_5b zH(x2xZUm5Sbl~}P$je;YoMs@cA%X@$B_}>HL5ZraycW@tJS#8SK`P}Pp+y6Y(V{(P znEB}$-aT7+`UjsXxgI>L5Rj(R;AX}va4{w5oa2ENt<$`}^QnTt<796luG1)-T`pe@ zW4TtDH#CVpfk3AID+>NsEqwc58PV=P`XcFnRl(Z-%7p^|(Gf5FS77|R$tfB^{<`ar zBmQRBA4mMHlRu953oCydLH8GhPdx$s+fPm%;r+M${;7?>MBO0372tv9lm7tdNA+$&Q0N zuaA}Pp7iyOj}N&nMk4Hu`zw6#36g0^XhaU;Fm8&dt?+JRz2o+HgGV23!g0>#mc`u} zs1KIibB-2pn>Jdwq35}^l-jYg(23~DDby=2v0MmXJuGZP$|+))pK}*YJBU^+kK&}RDMWg@m=7-o2T#3h zZMt@y*s%onb0vJIdtMzG5!MnD!bq=d!93u!&R$h3VW@#JN_oBTsq8$9iN*VK#>Pp{ z#UU5c4>ek2X4ac$$;GWOJ0-buPsP+LHm6$y#fu(x^JNrR?SgoixMDA%c-J!E zG!4{KZI31l{stelxs{cO(F2ae^p#zd6LP6ZflXV?tG51L$VSiM^^Zn*E}av{wpCjw z7cn^uhER`vH(Fl8(t@0XN~r!Oc17q^5boB;U{d))Ta@jZB2p)6A#OApRq}8_9Rkib zSd1uin|W`k@lz<~ZL*kJf7X>ltaG7L2eI_nb7ei`9xnIjtV%20L5N!|_9|rd?dha| zHD0)FtVY>#Xzk5@s*{YoBo%9Jx$!eC^622DOm@Or`4F(zMK1ba*>aTl9(JK?XYz5T zsHcD+)Vv|!WaE^XeAc-Tru*3mgLVU{#KlxSyies+;O2qxBzy{UU_q6E%B*Zd&~*gP5F(jtbHF36(7=I)aBvw zu2c+_{H4>M#s%oNu;YHTn@L@APfyP;4^!>8{B_Q2V_TbVh-|A$e>ywqXnTCXYhf?n zU-dwxemi!tVaw9|zKOkf<1NC3+rCOw+e30~xom@k3bm`uT3#kk6~tD?6*0Lkdi43J zynyGARPlM>fvgx|KARD%pICp=>9t_*=y=<9d1fTJZv~G^`WIZby$kZ`XN8ONt1S{; z*Dd4Tq9+664kk;%k(aFQA}_&^g=^<;VNKr-#@*LBpzSf31o`WlWRTwdF%=}mOdz{l z_R#3KL!ospQhu6UhnccX6xjn_f(z~CTm5CPXXb>zH zrx-rULnHKC!t?ZgAbYW?oSC^HzM1t1yE^3X@##-6R#I+&j94EjtuvdA%bk09NrIf( z++EyH+g7M&krex_P_fe^;0XdF(=4+3U3dDgwuaugPW<4q<}QE|Hk(RdyGtj9Bo1u0 z-{V4XQ3901YRmOrmUSC!X>Ru)5MCvgaE=jpb-PvAF0WS!ySR4+JvLLk(Jlv4%I4-v zJDt{i*AA_J&6Z!1a5jduuXJ=M2VmrzRQ^{3}B8(QPQPXknKQAME%2yLka9 zTe!~F59ywcyB5;i7at(;gj=m8~`n+ZYE|_$kIEjfa8x)jguA#9PTXR&v;j`AK zB$N*pLu=kN`m=8YWhMX; zR;b>rOh`(^Jy+Wo2n2vc!t_O1~S9 z%-*}sV>%WRE671#12-P5J%NC(H1@q@9+Fo@xrcGj-LV-xMr$Td#qcGHny*` zCbl!XP%M?A<~!rUSXC$a%Ysl&`&L%&8;fyb0+9`m^o=J3$W;f8_FDu@93v93HbC4S z;SyVrEn-#YMLt9AS$lOuC8u`LUehstQ#CQ_*Pxk;>{h?-Yl`GekD3s6->k-5qP@(z z^FA0^_)2ojk5?1j#cisWn+;@VNMBIe(> zuu=E+yp1clTURUAuI|C)+_@2b-mC7Wwoe)gs*a~aQdDkEnORtnn;Ms=TMS<7xD|oj zSCQ^C=VZVEz^5-9RVi6O*Q0n=HmAgQ*FIIM%@dmqbVuU%IlLVh2C@d=cuc+|?JF~sf>L*N-vtDGchNYKi*;vy z01vi4EQdH;qneDc(k<;P&2?DVD8OB$4cf;AoU|`J+gcA)a|cF6t=-bvTJ@T7Bfytf z59i(w?Ql$Ep?hXqlmm^se$A0&U{;!rLyVrOl9SwW2A~tko!A=kmY~Dg^bx?72>N%e zz)^mhb9={RW#-*qIj>%}lF(x6umv%QnfO&vip?UxV~!cwgxB_z)zC>!Q!O0GT2*CF7z}h z7y*<3&{iK@qC3i8iK*OgkC-bl8vOa?4uAs+af9iRvi^LN6OE;Bv`xF;ld>lH=&(Ot zopkEJ)<(G~#4DEUHqbrxX$9<*d>)620t9LaVTu*rk4V6NFXZl7++G)q0IdTAL~`h= zwVo_6SsHc|ht%E@o1%}f-}n%ik6!u2d}r0S==Ab)o;(iml=}MnLD%m<*_fT)ai%+i zYnX30+uq9o_SF2pFI>QV!AK9xWhrr55y%fkQnme_AoTf0% z2~bNQEdO-46o!D2?-{W=>fVU%BvRecl|2Xil{c z?rYwsLu-&ZXqZ^iL+%p)-B1fQPDy~{D3IHy_4|}Dq|$Ol&D4c4Gtvfc-}POkAOwW0fP+Cr}8i8)Q44DV+A zKRz$)H2Hlw67&-LYq%Z${6zeuF>JXilCpIhcUow`DKSKh7N6*o7gvFz27}@$TZ7Do zo^r6=bdXFtjl`N`K9P|P0cW7+K>|nr!*J$Q7Fu>E!Q_mzK&aeS5;5N%Uut?oK!Qnu z(|mPTe9rCp+Zzx3os!S-WA}OQwy}1nXn|o5jHl^BY9G9a=H!5qW8E+9L@WZe8LVwv zwv_zv&G~U|*RGGFHtZ~vW^a8CfG?wk_lMu9*CMlquK``cC%B9rVOKz1 z=SEih8PSjN>IsCopaF@>fZ7|?i!M^I@a zyey>NVqsqSC-p5Io#H2GaGL!{%uq2WG8K$vcBPyeO82c7a}1WAfn~VcX(7A5^Do(9 zUND3#Chun%6LsN_ta&YRx)JLR)hFuemV+JClZS#!<*~afM$#>t+UH3qdF|iUeSHKD z@k2fZaN$3w_3hT~N~xxN7(8`!cB<~fr1OfMDbXKJf4tJpuiH7047$WO)==TI5ZHGx zM@rJtp6sfadT48(RJJ<3;~4kZ6{bisPLz4}e$v^}cZ39z^%&$&Xkb8`kkj%^pcaZj zxP(QI^#n}c{ZhlE-!;CI&e-qzVP=^d2a(kg&E!8GA z5L$3J^9+L3=hG+imredM`*=WwDAC{^_Q;q^6o+lrr+Hee41W$v5=&~8LPS>kOx8W%NrR1H_uL1|# zs<=^Y}*6e<%fQgV8p6u&#qTBB%Q?8v&Uc;@ikN~_JPvP zm|UWj8vLV~6QMEU8%bWNRI@G0EH6}B*l|$3$O-JrJFx5moIQIAQeqF{a70oOb@eJz zC)00o%N?{%V5)qCXPD*LFfe6Gj}pwrId*UFY1_`PGl~^1qy~Z(kSYbqTI1bRpuN?; zq^DaZvBV{GJhUNzj2lZsRjr*DgsH|LX9V+2ohQ;T_XNjLHfmsATa(IJf0b88d%8{=;X@&wu~oRfqRpsBa?NWxJeZdsJ)vOuQ* z^qYHcU?ar0_((5T__xA8Tb4UG}g;v-nPBt4)b6{ z{p1H%1k%rI>mv>+owQgli@E&|#F?i;L2n`#=n<@H8Er$I>{$Kjott!yn{xvz4h59IT<)UVO4Y6w&2ypSqPZ9U2VjJUOqghO@`^Fxz8JUCg-kB0-Wr} z=lT~roYWo2h)k1jbx*}|W5Mx|bn8K*BQbh#NMlT=sLx8{(bqXD#^lvv5z;@?ku z#w5@~IN8u&+;-+WT`oX4LcLl~I$pV4o7cvub5Ij}L|=RcbR*d!7*^79h>y$KW|~uF zfEqkIUes0PIP(v5oJn4f;U;xV zoNr+1Ka5;`CaHf6x;QhvaJP(h&q@_<;lOqF-Q4w99t!Q*_2&*IonQ>iuES|hFhig{ z_4GXNJ(!CF!XnE?OiYBes#^PKk&_GL50-JJD@rjq){at#rA3 zEpJy#Vq$wk8$g^#XB_|K!ylQIFM$fzX2tsD?nHrkVw{4vShr_}G00a@%+T}W8r9)K zhYu8P^tM0NU7mNErEl$(0w(wF@Q$TBuP%bp<=tQ}kk34UfcO4&+bMgZZJqvAu`e5E zgZ6p-Ew;W^8w2C`ayM4UIPX<66#zIM)3)ZeUIwS_RdG9gFfF{NE|Fg{zI&a0@Q&!6 z>_T_?6g|T98=Se$7UF=N8{$peP-&0Z_*|w%B3@!Q9Kw2S$0+<1#GaF@&x6s42?4pN z%sm{X>p!TCIX0-NJtCG66V+T^UgjCz^QRQ7&A}gR;ri_PBW$+sU_l-&b5-I zTV&lEKq(xC=eA|dz0KnuSZNp2~4;T1SVgt#S8AE1anUA6;tBZVPLip%7Y%9ok|D z!{}WqQeXA|`nF8?Nd;muwNcd;qbTg`@$^8XB(1Q}4!#v&t!?H(dkG0snSACxEpch? z^tH>kJ>u`rBQM5$J3N0#K&pn8#8B)F3Wj#|E$^y=z0xc#8Z3^qIE%9NqU1X8&c1Xt zatzD&PKIeIcFHdk{!*PcDWcQ&1SV=mL|C~!_!b#TE@o+IvmST3e{ehDgVBAbjMju& z67IX$Q4+`?Uw2XXni|I)HzD({W;u=Z43vy!&ur`CWpNj4gtTTO1m&!0z3Ft!R`ku{ z2l~q5W0S+#FWl7BJAsIsl5H-3x;~X()N`J@;MR(LpPHWt)bcA%0fvxr3kC2Sa&vm; z)P6yPtE*=)5Dpyt`q;nst?XB^?YJzky-s!B85k4&#M?d65Lm5IcPBB5&xVfV%yZ>n z_7fU}!1i^^^rv9ii{fVfxIPVmNSN5!No9A*&YvR{6W9)`H_|04=^99=;^xkMrZcDd z&Bp#RoxW~1=e`0}JxlQY4Y&!Bmvhd0707#T5$<2Ix)j1YF`>2^S$y3ai1H7eTEZ#^ z5C3O#r6`TgG(?)fGpX>qdxM`wM^AY*>9ir#M9*cT9zQ5CNfMU`k0Ajpfjh-~nq3 z+@zrx*DKM>7PfXZRd|FxT!k-FlR>HADpm$t)LEFc=oA!;hSsY?H)2w$x zcAl5FC7Ba5BKGSc;Y);D|k@5$Pin6I1Ykz|nEqXeR-swM|!7i+^8M zX|$MUQV@Z(2i}|7sHIsROo$pvr@d0`x|-*e_(HomF+oI z+t=A2iw%Af+nk?2zX%GAoaRih6dLFn?>(9+v}{jNpqMrF0coyJ@x<((xLt6q6srud zxQl@38<+@g7Y`feIZU_<@^QGi_4FJt(wo1W-+0R9?0`mimGScNF~RxjjC6PGzm+ze zFMDNiu6l&1ay&MdjehD|{f`62S9ok&fiCM5G)k>U>~oDsygMYa(cvY>!+I(cPq_>c8Et*`u|WB&RRC&E9>6)oEX2eXj-NWxo= z_ag=?o{JrBS>-7%c6QA!!|mf1rQT}^mVC$(ilDEl(4#tev5d|tZc5aNq0KaCKWZ4xwf2)`(-OkV8IrKgZ?Kx! zF4CH2OdqCUTMtq0uT;;{U9T`I5JHHalcy{^!!6Gn*AL5Dd?TE&X|iB098Z97Ny|*# zI~lrH(5s$Vyz=T>sgs^$pV#oxSkguuYk!_9a6D&XQIUK`_TkKB71fI1@cLvz`iD5G z1A)+UJ#c*#R+4IN4(b7UGry?wOH{GJOmhEHx}*5wHb3U@pl-Sf~h{ zmeD55f`(+eRaT8Gzp-+Tov#UuNn;texI(BJ6W4Iw(9=2R>7@7t5c^bFB6dk!H-PaF!j_y3uQn@$C1q{H zF9#NQde{K4c|`4!l$5NZT6VHO+c(7-IAV1_0X4`SG5Gnt1}`=Wlr6c&tHE?HrLcb)2WlcXh{aZ5*BZ7NWfR&-$}8Gt(?uGX^QO zl$2W7FV`ggK_dMn*A9c+NeX=LvP7Gak{UYBMiW2a*6tIGt*vU(XfuI$x8Dr_-TPf# zia$T00Xz!n0x4_CE8@=7bM;cx(;x?kPegQN$FH_N7;3iS|ARWdK4}>_bD@EF z-5>6$Wr&-|FQwzxElQ^-&p&Vv?M-r;?C?xEkzb_sw&eKog=iqV-R5v_O#FjnAB@`@ z7WD1Bp;Xb9*H}uNdU1g#tKf(j1Q^e8y*zQ(x<5lLU(r$XGf2uYkH>%jt*D}9> zi;mFy8^EVGw+=gxoc0#Gy1G0(T%wW!W87mBShinsXD?L=lt-WA?o9?gjIrU#>@8^Z ze){w@+4v0|oU0z;?q8OZC=){TEL+_Q$LuKZwHB9-A0AAY6`TH~lIhJoPtGx6n}A{+HgKi_ct!XZs%P z3!Q!Qr$}G(^6(_ej1BhaF@wCR)BA4pc0}wX72$jFh-T03i=O_;`#`7E`kVb@_s6Qe)sVb|{agGfm@1+yF;bbnn>y0{iVQiCSL^su0sM0E0TT}lZ z=HZGFG!JPpNbAq6)c*Zucx;5iG3^xRHjz~{ZHQZYgX+APiXQ9ET6_@+lndZrXsMoP zJIHWnmy`a!IW|kR6cD(hrGs`vP+*JW>>m9TU*Qn9D~!FRy|@p9^ekW-wxZ5QHt zWKwFWu~*-2!*_CG-`(^e1 zetw|pGN%0Q=u!Vn>)Pk_M7b3IwuTt`iU_fJ*DV|(ON=|)c9 zJ0}n}Xf}^80XhXk3)iq}SxzeG`xP-|m1Z|GHc1QoW=VCycY-ERHvYN#;a+A`9ZNAL zd4BnQs`I13OZ*Db@_E(UTJ!dV1ghHjg)X`XzR@#bA3WdTGTL`0`9rnh9}cGYw8>`q zdgR4R5c2qqFDD{?u_$pH0G#44Dp74h-uyiIB}{S&0N8RlB$);x@)xip!{_hQWR+f| zP6w@tYB3K}C%(E{E)y@4>A>RW)-PsR)47Hy_tGeYVY5~@e0R-bnpk=5eOdX*hV%$Dn0<^kg>d#mQcWqD~f=AHYS zW$R_`pZkiiOq5+|tq&bix0HqS$vC5hMOw=qs&IIG9Da<7>K(St6Q;EC@T_V#%}0- zzI_OhwLYbzWpDRr<={$eC*Peel#9-+wNptKenT&)`v8>5 z$%5`n!xnkTh)?yUz|R-2nHwVt`MB-zhKoLf&aNQF^5yfvLZnbDXU4Ly{rsw6t{vcM zv(FlL{vplri@a*~81#i;lIn^of$eXy-+J!+T=UQXNg>xnc^U5JNCS|mT6I3uZTYU# zH+9uA-2U95{B5}XC*UrDsFKooAIRtN3P#d*12A%BW#u~*>j1_-NXu-1x>OL)4)skc za*{1>`e$2+@^`*--fnX8a)WBLUjt>VC$D0=;Uo%EWiZ3^+`vfh*5NYr+v7SJknYm2 zt`HKC{umh%n#!%n*xuPq-%S7cb2gIT3>ScWr;nh;6s+e5g9aTe9-=(hiLG7c%8ga$ zPnih|9We{;jSdNa8gn%Ot3|Rwn@+kzNL%{%@6QjK;WR0j{AV(9gdw4qn4A(aXO3O= zFPycXou{_6`hDppbhG@&BV=Li!#V@U-!FbLNCn*36`tt(8Ojn&B9fg@L#6e>)v4jx zwiShL_6C>}4X0N3K3{Zk(=gUpd($`bRyJe4H>Ho@m#Tm19~^`1Xx^fhBZGvixF}?! zH5@FC^p~?gztjn68w4E-LazicM)Fczw~xb@(8wP5TpfW$FtOcJhQx+-rW_VCF$(&M zXg8+L=Y!Lxr)Lw@W%{^B2OY{4qV9UQ!+)nWj{P%K{LY%Z*~hpJCK}Llc(&bP{RB=o zTV*6^!}~Uy7h}()KV@Q%>{XX`>Rz zX^I#NF;>r6@r}U{LLpT3XHdC-qMg%n?3y5h{&cOZdgc;g^!PN&5%3qPN!pB(+|x%J z>q@LQA_(Ta>iUH0^WL5X{`L27s@K|2otvqwEU)A1HzY=DX?Nt?7_R>#WY{6bV!9b^ z9zQOZV&-bK8x#{`D}p_zFnJ z0VCR%-1|^vumGLcW8d50X2Ye$NS;qC-)C)ZZr6$h#yx(7CM0tM4%2@W5P-*?o0k5We`YLv>gHu zQHB!2Iuo#y+!3*->i8F~PYu9;t}$9{!?c#K%zp@b_r1;!T-lHk)+gm;|4|W`xx14) zY&bEz#<1|LhQ{~2SKC6v{}LsciP_1wJV?Cz3>Vd*f%ON3$g@Gb@NQEAOQXx1KvlsC zAJ?-7_c7%j^|ixR_vhw}f513c#^!?WxCip{?jcEnoPS4vbYmb4_v10`^lM;oP~XNE zL}aw_8=WD3v7qKH_-_rmiRb{PpaY_e;m;eo!a(j`yx;gxw)E(=)gNvR1oY}JN7~`!BZT1RY3TnbESrx zhY%VSn`QtJua=gHCr(}Tq0-gBf6VrVEFqi1PPwe?Y}@coFcwyd`E422xWYj7ftkx^ z`X-LRz6V|b#O8u@(fVk_hOCSn{X(Y)3jE>#xzOkmCTx5r)%r~4QS4pe1$LjikKXW0@&0#hb4vt+ z+bn`-vL*LyTf+)+&7QoV$ZTS^$HcF+Zv^R zIC&?zObu&2!21|(LCuIkcO0vy(OPu*vZG&-KYEYHR52pFWPHHOHyU3EHMf3Anrll9 zl`*y*8G~dZgntKWTLg63=abb-=Rd`#&z<{9M(PYid2xLS=~`A_yVmB?(@iix&Ei1& zGmZ4ZE@-qCM-a2AF~QPl{K75=R-)%dv#(!^dNkA$p0wdt-$5Orq+Yw>k`|kT)Kzg$ zS8HDW7OLdiy7bT36`vb51Ipv$N6?~Vr^;Ky80)&rNapPmG9M)uWdkkbQ97Pk%?h$* ztVv_>ky&q}NUjErg1UT{pMjY8N|ui%6BIFJ*4>vSXpX$i}FG^{Dm^Pv54% z6VwuUYUAoZhzZ@_m}Dk|Y(4HssP>l&PlPA($xe(LTHC-s|aZhx6T z;tOOg00)LsZ{e$WzpnrZBls*2s5ssTD4%v)Wd9O_Cpq^c6@ZG5((ViYp*Il7+t^KM zGHRIqZJ_VcUfZ~FL1md`vdW2vkRbkxgK6i-hLN0CR|!cpx%r!UH-%O7R*!T1{{ zMp0jk9A32et}!w4Jf3q^zVkoZArMmcU}kA&kMm$%aa*57tUJVHV!}RxBP4siMpXI#;s*31 zNEvq#9m6SS#G_Wc@Hc}i4&|(8EslV&1Ug9ky%xU8UGj^i{1a{0{1}Jlu5f(;mB{Ur zMnc0dI=?-`n{a)=rc+9z=<6>x$uU3Y0F#ia7jrEwM;t*_-wh~P6@^DeT42*(E>CNvy^W15>m309dPC_w_p`FIb3+0VP{1a} z*Dq!dN_O~fG+7QBok{h7q-AlDFwPGk-Hng-3mw$Y{WbrLQP?LFS!g>sHgXW)i_G$T zt>7hq<<(g#JZN>`;OwuXfqBJ=6klvy&%fz7auVZodQv?zJ#E-vn3`=+TK+nkN)N6V zCmX*SUEExhAS)*ye75Z$MGpc=a72D!HO%F+8hc!8V0U9g$k}T2HtFK&pa}dw0G{b= zB|nYOIJ)S!D3wOt((zHWQas4+_mwS~X@)xLv}~turPAeJr1@&$fd7%%+zsX-)qnbL zpS}kt%WS(F;jmJ80^n zBP5^!yx_}tEIq{*RQ23-ucv%=I_qi?!EG0X)q!+ie5QMe{68t$EJY*L)lG1oUxYjI zTf{w5pjjP(0SuV`xu%9bS0fl{YH-(m>vry81(2P;*!_M5#K77sBr2-c);W_uHHX); zB&_XSDm)uxmq*f8*?K9y6uz%dHwhgGlX~t zEp4n^^=mNFy%{>VGCIe$=~`BB89)NucEbjg?lxusj5|!B8grTg+J*M-G}u~bI=wX) z3lo)GwpCfxo%eUQuiR|fZ2noT@CNOe%G$tlxITV|(3SFkfYUhJR{#|aiMzue*iylj zVej$|dMJ~cUXj)#SA-|EOk2W1&U)|Om$#Pe?5bJspXW94=#yJW=4a`{6Y)we*C9|F4dfD*b8rcw#%YQRJD*&@FFg?M$m5^I5m?G_wj$^eu93udFZ4Ib4}|mQyQ{A<5nL z@n86%r*osD=X-KFJaOtvHxP8ps2gxhA|l7?9%w3laT{$P8$RhtZ28=h%u+PJ@{Xwb zDnL4rWBeT}VC`@NIIKNVJXTurs(O4e=&mAYougX~yfE zMWXV!d2luG5zJ_)V5`c;HTmbBo)xn3td)}?*1vrLBYNr~O`|Xf9_x}_y{cnu!?b6y zwmx2-kpuY`j`2R}R!D`FfQNfvBEt_B3DTfMynS0oclrPsYOS}@;2bawS(M2@Mr!Z~ z9vnBdO2kC>*-JBX{r3HSgHs4d^Y~R<$U;#?)vXT&li8X(O`Vqqq~y&>Siy6-yxKGW z?Jv+(Xsu1>$ImaIwFab58W=%8EDQh>B@B!fdDay0m7}8*AGe$N6B0L+LqfQH>LXjG z^a%CHn8__2SD6S2Vi$pQ(J$YW!`V5rgqlqZo@8W8>`)*wY?r&irq4bZNDK|C8VgD;Pe$pbU+dg zv4)g|jJ>IEXb85G4u*Z%n5dMg^+^FDxl*T>iCZDb;9=>nG4Lqd^R>^?3N5MM^($8% z*2WE1dhAP`ar}2wb=a7I1w*P885T00Jn8NxK$Uk5Y=KdyAmI5jOLm$g*k-DTK$8nD zMtFm1ueVTp=F-AB6XoYGjApM?D$H&<0xn6FjrSg%8k}>06b^Y2usA)G*;ypKCe$YXnW~X1CS%=r)uYA zzNw%*#qx;TZgbl|)Pap}kcCFFhnxc`p@ELIar>>;eA0*fwjgc5%;csyaQhZQ`+?#VP-m`;E{4)|De(+LA4nuD@igo zHt=*$BcnSvis6=Pa@w=vJN%u7w#%-{GoS=9qVBKOr)D9Qhy4pzl>&3*MQ7F`GfsFy();k{HY+3!<3N08UYC-sd zT!6R*?kp7i3jxH{;S8hq8nD#^ai#NQL1U0AWA+oLL1G)zyLH*g%RnsrX7#^%hoY_w zTqmrg3e=An2F@)ykx&jgy$3cgOb!^opQHw-NoMn+zz`)KYXNVxp91Wt^bLzsF=sez(}VM zY;fZW?!^i(3K3U3wwYK#2emCcx^($ykwSf4Yi}PuYpeDd(@@Cl+wb3@D?7Rsu1bYd zPp{v&k*PV39wSP>xLKZUWf866u#jVLvW@m@cOU=KWJAL?(X>x?y6|BNZKK1&T&HR0 zT``DJXX(z3_1Fig<>jnxc0}|2u>x+-hVdW=VhZf(qnL@Kq@+5qq-4sRg1F{MC*z}q zwwV3(fEefV@$nyV@3kO4-@Gpp`U-I{P1Hbeu`B)|{&gP+hfdiGb#+>{QU2di1fi)a zFCS!0DRbf6cl&TD{}TyhSngJ*%wD9Z3L*=YRJ8@^FbkWug0>wKWg88nU+myoJ**V?*)4 zEao;Y*MM`UQhPV1nX$3>J>YV>{3iMjZYL=ALie~t`pW{iL`-lz~~XcbH7?! z_t!RZ)Pq>3-94gT(4QK^yBzxE6|r^A6-aa1^z?U?w1fgjje`VrKHjSGmnq<2&<(j5 zZp~)fYX0LK=7F(Pf7&40(;y)}`T0$knW7Gtc$TpS@VWPuA-5dcVw^WbndM{i2qBOX zdQp0W2Bi|SWSM_l*h}fE67mo9>Zf^m{xb)wpYfPDwRrLYB*^+|qdH=DyG-MKl}8T) zwIoQdH{d37ab-Rdgf>odLNVtFa4Jf(UjcsfYh#rJx_(+Nmky-eL~J$IJx}(58}Qh) z1mJtoBbzhME-vhu>FVHD2%wTJUKsa&>eK6pfiKNDohvuo0I#oTV61*w@-Q5_q+NLN z#ufX2D=?~Nzav@CZR6q+#(M;z`y4v2nkzy&UXY7Cyb;w;3-K0Xr4Taeu01^3JH9{O zmJJ-0psH`IJTo@pM&`w!h=-vmRlaXszmvN8rpIp zaCN4i$UnCp%bECGU+9Yi1(UDppM8zTKw4a*{WvJcKYq=MZHvi}-|5$^#z?dy@SSEw zy@er^?fm6iKQDj_)e*f0=UG@-o?A-=fCDX}o!gh^Yg-PVEcBm?+Qhg2dg{T(q5Icg z5m(y6KcMjMm$})MxK66;!7_i>XPG}?3GH|UvYG0+|1!y2m)7PhHI2(6jv6(1wdLFk z`^zg)JLTOb_7l+*_YF&6BqgVn*Zeu{)wfr~u+^bgNCNL5zJe@Hc4~4jtl?fD4f;S- z*zR#~xX&hNT5nh#x0dc?S5>)z1gJnqg3T5NI{ZYZ)E}5B=Hzz7AoTa&-(tk&J;l{Q z>dt`UHIOp&0Glh3djkL#(7ER*_s@W2WCZW4Zg*w)Qw=PvlvR$77F}1dUG=+xC{TYC zaGG|X4-Yeo8@O#ejwY@$E2N5(hJlx0&Qq=I^qO9ae*2SB{Sch!C53xUg;pg$q&Z_} zq2nEe*}=yzX~d;-znnaF*cOVNN+~GRT*ErcZndWb8;Ua&h$)F@wY+2i#00N%}VO?=)EHN zX504ro*sizOj4{@p02GG-PCD5y##BfrAFLr?HX7(=dJM|nKD@}?uyqK$P+c*;-BI{ z@W+RE!PzAxX0->)$=6e}vfz`Wm>LJpj87}?HJ;WFQ3)G)W172PI_ot5$#m(Nv@}DC zK~x2_;x*FhU`LZ0CKZYW5|-E^M%E}wJ%HD85a`LfwnET{UewUX$==jE?#nBK9bpcJ zeM1BzcUIQdEv#M`D4+2?jZMVvj<&78rP<2Pnfqmls1AVh$bR)Sxt_27G=(-WOSVVi zs7?Hs50^*JLLvDJeeWrxzKWCI;Pj%f%J}~9&9z(G$c%$u(RwzSqvhr=Ehi9yiQ*iy zfAZ-E;Y|8W8O$7}8&4W7?jnChYM!tfdqCsGqm3xHr5@ubNHf)Uw3}^Lyw4}dA$!b@ zO!nvzaOjJ}A=c-ejrO{t)~L^1_%WgmBOQc-&u|Y9xUB~8Ng8q406P-yqy5;hdF9TV zcBeEsZqS@KG%V9uom^bnGh~_Q5rH;n7^fdw({JF*nno5X(PuRYP~e#Hg4@Vq zgw6=)>opQq$Aa0q{bOWaW8c5uF2DGvqXik<&+0tds(G_aQg8UX+D=CUBzF9724$I@ zw8%)`|KNY#a-5f|a*m)}A>K9+=)1vdw4)QlDByHL+xg`DyD8Uhd8NhORle9J=V|HPXp{P|WG_CLkAlctN84v%q^!tgT;zl0ChNSiB41{VBRUEyIU{Hg22 zl-H~>Iu+uS?8wrsc7LvIRMO(m&3=0J9*9D46u}3-{TLLw;57lcmE7z@JOxT0BNh%0 ztpYnQ^+AMWN_%cWRaKE3mG(}Fj~)%2x7qd-e%-Jx8kpzrr@fGkD-#TA5WylK4cKm#K4gFkZY?PB*)s@O$qHlTo#O&Zg|9N-$sQtC0muDV%`A{9aTYU#V zOwV3*4e~p!p&h2r*#Rp9MxgLPsIAV>Rp|S%#)u{+l1Qtcw&2|8^K978Lq`P~@Gen^8}-R-=+ghDf?U@P*x&F>p|H!u-Arfc^9EY6zNG+58OTa>8{Oy3At z7OuOrYtTfHCRYnUikCn8(;VR(==Yt7A*-v9VJ;7O>u{d{TH&-2dYBhVHYZmiYx~k{ z5a^q<;$G;autCPzoz>xdX0Y+m4CZi^LYQjy_GutKfo#7r*ad?Bezgh`#OL`q-job* zArryiY@@}X;)wueyOZ4zq}Jh?VJ_Fr-yq>ZwG=+OINX+M@gnIp{$whE3D6ATf-Yhj zk!VUaK3?52N5P>RBzu)rfNOeC{<~Fh!Xv{9>fw>H8YajYDlt!xRT6vQK4R&V>J;ku z!RK}?6rx3puRPc-ya>seyMFWNv1rceMmZA$eZYrQN3Q?{?xeDjuLqz{yfoN&@ac zeQpXT-yYHc=)vmB#jFs0!LiJBr#+2$w-G?>NTG*IMa1tw5-*UL0ME}+wQ58EiH$b+ zwE(wf(i?3n*I+N`eg?^8Qj0sQ)U;gpPkpaJBMa;q2(nOt)4h2*&doifA%r~du+%fB zzeWas;Q977ex(ZFNACxaoou!rSMW1*ddXI;BZ7Ko7E&tN2udY!Fp_6B@%kO;pI+mv z?t2iQbFEx}f2WPie!>E38hvQ}$p|ky0ygIY3b2MSGugy2JQw4L4uAdO6?n-{|K+>? ztGzD`hw}a3)*@}FBxRRMAu1ue7F0ro>_*wLC4089q*SsNvddZtA&qrxWtZL9hwQr; zgE3>C&zOF{Y6Dn-igcG{sRRHcT`;fhX9qd=i*n)bap!C#AVML2=#e-g!WO=ZVWj zUj-ZGD0f1OtAN_4BQ1pT( zI`JFvjoI7iB_kuoeyC~}9dj)EB}lyqLv7}UmQ}2H`#Md?Ik|I;9iEiEE);m1th6(= z+*s!eXDy@MY04)+1Kz!H9~!UD{4V<-D!9NjMu`}+m*zV?SE1RhE+N+MYgH4>q50Il zLA_!6(Je7REuEd6?l0*^`*SuI4`)K8u8>J(a#E6hh0o?=3q$5j{i5Z&iF=S$N^cM1Q?Cg%FqrsJuYaI%V?NjpS6$64 z2DbKsFJFv|EhOZB1AL*xo#1t#g>cMHHHK!gjPIB*y<_6Stj3)6#@bNDQ}fs1#?0)K z6NR{>=&`NJvG?I-_Oa?KguI4G=)cEDGHe;ex+0ov`L7gChWeYyj}JU!GXI=v_tKo)1_ zW*FJ8w8)0AD+NuTtPYI2fR?P5vhGor_;GUd>4p?TcO>^eR(0jC*kT`0mVEIa@PR*G z=;zsbn;#j?1=xdtwk&v^T=b;w^}BTIor%ruTHkV@I0hr@` z{0bbW>>IwYbo#Jq%An7T8nVvbmo#x-fXZm1Y zUv(b3^CnY3{i{M)6s~Xp2w``Ep4#^L9uqN#4^+6n1|K;0OKz)-u~_C(9qXv z@mlg}GCzO)^FXKR<@So|XpiPHS~jb>=wsshc(jPgM}UT^Zupbk>#^k3D^3MDNhHK` zW}oAo!g|N7_odh>3%3^ARk^Y4to>5VNBRRitLkljsg2&m$0DE)^{PRZcSK{Q?M@P> zu3rArR^(<*D~}}-yZXAaEcS_Xb!GNNKS=42evWrx-+S!S?Vf1)wRQ0wWrC97&Av+R-(Y-0BVh_qxyQj$+{QX^X4j4DWcX(}T^~o- z>@ZPt^Bgp?OO6 z=vqc*X5i{BX+XtYsGYfUo8ED{(2jLAUd29urehne1yTi2-fmP`)fpNBx3<;~=l~=R zx}(zSAcG8Ul}J%hrUM?PgGQ;pTMaHs25Qj}tRLUHwdwg8x;KR-SQp^&rIgR4viHZ3 zR=~;+EtqTPKC4SrF4{|@qv2U&!{gt+{#>8JBMVd+k2Pk5Z_p8jvWn+{$}DU)3+#_S zr-yn1sWR#nlQiU!Fg|XkkSJF#P(3;^?9?^@cXn(skp>sFRWW}i{&0R`G4<`ZoIIed zm+w)dK7#xwBu8KWQhrXK?!)Dp2#wApJBV|uyC?U{Jx%^nt=ZLa=b$qwsr0VmjR|Ed z%=g`FPnHH;PUkTsKRIzCcig$e)_pGUye?a%%YI^7KcFcJ7LYm>`|l$GZKSR={2?8L zUnSzhu|3D+@x(t_iHwe-i@c21j}5rsNs7>zGA*Cn^BPhd;Ii}_+|herY-qg=qFzx8 zUHA1m6p$_8+!o{#t{f?peN!fbVKHuJb~9T~o_TQY^?!U%8Ul(-**{(YPHpSczH9qKUrL5;+aw17gv?dxB1ii?#wCesh|zMPoRI#42;ZMYWVK`llxMMXv3a#h(g5jU%7 zdYE!QtEHrx=10Gqfn`GjzM?8)H_W}+?(6I0fb$qt!DEiTxRUoD&vHVjH2s7s@=ia{ z$o6BbghLnFB^40xwX8!boC2(u&mzinhgMv90|NzS?}eEB`#?y)=F)HcgUu}i15Ob8 zN09OncM2!wLj(e!0JJ%G&UFZy#*4QN`cs=5W|jjx`a_~bs`5`^%CqHmqUK}B887w; z*S3U(UD!ne{923%hV>Nwpq0U3@-sBKW@#lO{2<3Achwfe8(zWx?BSHwveor0{q;B~ zy!Ss>!*BGl%U1(}Zra~hzJPI3zVmq0hftK-p4Plq?up}ryINY=oAL|cft<{jd=q^T za<^IFb|LWocA_#|d*Nx5ANR@#HXJC$0m`K1%qc9?>EX$twp>+cngY?$kI2U%CV=4a z9ol7b-Da*oqzVY1XbNk1ob;9JH&a=!iItTseKi7M4*GlP2@&%>Y*y{nT@HQeFTR^&t)HHnk~0vyyFgVpkXO20Ji5Pm!rqZoKH4wQr$GAw)cnJp)PUW zDZy4L`AWvxXh_tZ!Xib^DgkulQ6K2{U75k@w>qKQ%;@hb6$gQ+Hexix(Boc{twwG9 z9jH2-u}%rmh`O-IPyhs^q9?B9%d1k98zd{^|nB?Fd zrZZ6)+Jg)Q(B5P=>O*1EZgfVyrK_=5TbYQxSdWWd9kjBw-q#~9 z#HYKeP#yGnuUHps+kOwex4#?JBgRg*3mKLiH+rw#Ymwd*E5WOEM7c`EV8HGZDs$d zYn=u4k%oqmerhM-Mz62!LAF{`006%lCk5LnM`ED}D$h)Ja!sa&&=XHWuy>Sb(S=5I z^5~_F-JU`3z>b>v27(vIiU;witk`&>{tT(S_M5|muwPXF9w59Um{n;B0Sw8pT(2qqV{#7?dmddIo>Y&EI-=iWoN3mfCPkkw?zYc8DbkBwb3G zlr7BqFHwJ#E5AT#!1e`7T*z|T2GLk4uVF~8Ye8$)mI_(l=aq}s$77xcuKcYC-v5;E zE%Kv_TJ&hmO)=usv&3iESBI-cNNGMXBWX6aeLsjj?*|TsUEg9sM^d(m9_q|!6+rp{ z8H+GJ@QtC$j0re|>es^d%CnI*%g1rA42i{h1)0&@Owf69?)hCUw za*X+}F=*YzIJR0;ftc*$a&3PNn#3`scX|hCUd$V-fvG-Nn$ks{Th!~ZqcjdJ_xJ@I zFm#9U-^+L9K78r`Re%TsAV4B>>;C663^h(H$%E_z?cbkEt1`HxwZDYlL(#chd`I+4 zjby?ujHYPTrN#Wl`m}0jElEpSZZ0*yKMC?E#mb~R1FDG?==h|k_>x_yC`}-?!Lmm- zcY5SG-E);Ex7N^hqKj;aEXXpDH7e*eFJ0T(xUf&?Cl-QpuIOgys$*kggFW;^sBEGT z|4O@B7hsT(;pfHs!xsT)pV9O zJ2`sY&{9KTo%MVgi6$}B#J&eD84?T)agS;Kf}Acy;!7Yq{q~UwYsLqX?t4v;hZx{|HbimkR-&-+~7#Op8sm2lwCX(xjFilNoaOCw`i!^#a- zO5MU(l^;QRO0lbv#~N|x(SWW~xN!;8E7pLIfhnA-MkS11npe`vZI$dNA{g*0K;NJU zVb&6W(brH6C3+9EXdy7hS{Cnr1lId1psBEz6!lMAdJ)tK5R>4eIZ$as-I0HgrL0QNdUF2>%UbQ^?&4w71n+y2**b=0``hlXr40g6>KcW7IM5gPt7FIw4 z0d)I7dQwYCq2e7=Qs)h?7iC36JawrvlUX!MLc2qiFBn|B#buIgpdn1O3YbF>gjg|L5ST)G=c{(8&&#w%}G0Ui%rL{(c@EXdWdN^XxBf22Bx5{ z{+8?C7%0iwBD*wa4Bz(zIG8BGy~h6mLwyw2J^s!^&^xx&1NUc)`bvULZU@AMS+sr# z_x^lsY14@su~;kc0BJvd&S%ORp_ak#w!r9)*I!3)d9L|LI2fdOWK*z~;!UsnhN0j0 z3mO!i8q}19+-Rmu5SYs{LCc-2Ve6cb3jWnKkxHi4i75y&BFrI})+%7!y7_fFJs`59 zaK+;~;n=J)wA7_~VOPo=L0=h1slv{{n3Y8t5zt2R?@E6|*JSfujB zI<_3?m9qQL;&AWuOH#6E%)dgwKaw~;wa=Lw?LdQO0u38ROZYn6e^fTs7m(V$fq z17pUh3&RMQt3EwDRru_4Mh2q<=CqCJAZNg_-@&UV-O9yzI+m8~54fc~_ay6Ia)fE2 z6S>B)D>Z?-L*mdQPOm%RA$;!|8VZfB#!-v}aOaH3yX=~u$>PZ%zA&Y8+x}eXAoD|9 z-w@Hhn*TQ-Z3ga$H0mOJ&}DM*u#`E2tEJAsti<2TYDF+Ut??Y{g;9?iOgnQ9%E0`7 z2~E<((U#eAlv@XgS$(%#HWSW-IS{fRfjM8dB2YXjDbDL|1hBO-mIvEQ92ecp2KfcX zy%Og0(mY>of<=3Ax>>~h_f2cju&0MPNQ;!A^aR3MsIkF^Z9hI0HV8=_jKzj7`XOM1 z8iLgv9LRC_-R~+V#5S<8;aQsFG$WRO+7^mcxZg@myA3gk%X+ZJynW_5Ldu2+6AgT0 ze_LD07nrLG%EcDuFL!Vi{D2x(6Miiy$Ys4ZS)6dcRVx5(B$%wkcDeWKx<>Q#U(p

    5q+yaA)8n75jY&ws25`5&YY6{G3ZT7>_PB1B&a~Y`e+o)+lnFM`Kx>4 z$HHQbf-XvWg+ts4e-|Gk9GX>bKa=q9R`kOb{jlET6H5Z!#m%iOUYozhagrrknQl>s zAr%IgP9;!=grL=_RPwz>euMjri)3t0y2Z;eJJUoM(#z^m^4kqXiYo8f+V&kInON@} z>iu(@hjDG(f`FSUYs_0iPJFCtjoi=fJayQn{_*qi9w9ff=il#BxA}rXv=Js9sQNJ`Tm zL`A(t8)1DtWxh5mdr?Vy&@Ci%6EA;an@hq0bB-uSH#35;xgvrHib-D4%`S-O_46TW ztm^dpT(4XldgE`tet0PWt>AWUaB%mA+xXJ}?}=C-7C*{Xk@|8`zSG@1a906h=-I-& z4mY{DI+EMEo^#jM@z-Af&2kw83*q3Yo^Rl}*t##>OO7}7WzHloXT276LF20#}OzE<35Z-x@aC-CCubm}$y*A6M)X5V6(gp>6IL{M0RXRMEmq@r=!xBgS z;TTf~o`|I-1uU+Oy_IElVsH%#RV-7RIkkGb+n*ec{E_?lueqjN9Hc1BA(c7Bmw{3J zhTlUjKL2B}I{5B0=N=3`Vo-N9lB%#vF!JpgRq%fsKfe=W_-0jVu9rktZ2CJ2s+4{@ z9l*hEu z7*BaXc%t5^qD(I6>Z*(Ut- zDpc*YiLj4Q=*ODqCnxo0`>;909u2p{gXLDn3a8J<{&;niGNF|SyZpk9Pm9Jl*tvhb`z zUd4C!h1d9~5(~+}Vmscbk|@*@XT_NgSs6LCl)Af4)Wn&Q(SLnY`wh{SG8v=Kh&IV(g)v`Uy? zV?`Li7)%m%r|e=_;DB&%a@PKPz(YUDn^i(zHRZo>bWUVS;(U6thSnRjLW#}oNFjqT zJ>~$j@ovVih3;_AG{S$89*c>2R|(@>;UMqP*kfRHD>p^N0o|z%zc7;*-)92pkWqut z(0Cu5!5-7ivf3P1_H?Z_2jp$%PkADPFHdOc_H=0LNuf+856QuqNL_a`*Xs%HzfCm> z!#cP~a0ldZK3kkX4MzLD&H(VRi;F|rZS-k1OV;$-VzLrvv}r=4M62Dy0YVof>q{;- z5G#gx6~{f-c1@7ppD&8nK=gi}5`h}xTIL{o zWd_SA2S~YI<3z5((T9JEQiXo-`46 zA6NM@<+=h3?1h}Mfe}M<;qOun;)!&^>XDehA%W-Z!|PPJMGO6hoa!M>eb) z46oPQ{~AkS$)5Ol`5WZT9e+FvEr2_Lh_DL8qLPsW3&hMkChWh z#W2~L0HU#%9VW$3P|AIt_#%i=&dq8$rmk?$>)+0ld6S=lM)E?zcchLWU5l+UlkF?3{!(^J8j?d{of?Q>ry0Ei&N_o|2ewnI^(B}?HWfIDPsRk{;$+;#*s|*kPz)P* zmjeBye``1aIKHzB{~PLX4))+&dOW77-K_aDhxD^WInrVpkXH~kK9uUU0eUZZ_P2qu zD1@2pU@j@Q&Hh6Dy4&+a62o8FrTgGd`W3*!)RvXD%A^I*ztz-Kj#c*LNZtczfe3x# zlJ#igl#!`Uaa!ud&b!~#3>{#Ax3=6P@-V&yVOmkrYf$Vm@o0pJdvrxP$#*YQPBoW7 zZO)=}bx(Kah1+q?kQ+yD?@tENx`itxFUK_Vue}S>foAvNcfwXJgH5C0{eGb}F6fBj>^P zOPS&iE{l%M>$7lZk5~@HeGZrel@%+!mS(MgBLzDYDD;VU*u2Cxz}fw;EJ1#Y6E8u3 zyw@kC2Tzt#4%;vo_++c3eOuXo;2tSKT);p0xWv<;@=0M{#e>37BO=L_4G2%C@x~q; z7M72dWX^DU9j%TVE?rSZTUr=8tuKxB-#(A%3Azl!-#;LW!jL)@!&*F*7W0JDgH#hS z_5H6Gait-EMYvi@Wp3e0NXP?4+|?~2^?gevH@<3m)m!$aF5m#NzWh@dibc%3h0mS6 z16x2RD|sC4epgr_^_7_V)uxHpmP0O`o35soUN9U~P*8x81={nT^y8NDY*34()P#A> zn`47k*`)}hSPXZj_(Ywn(M+H1uVyc1mf?=FoRCUOiX{Jt-#|`HJt=n?(0v=X;yT5)B|6q}sR-PcJC|Ks=z|kY3qzF#x5SHv37(BsjP^7n=wre>^2fyjyx%Yt*E<`*M?b zw$6aH01YOj+z-*Lpwv7ap{4i`hV;4Qt0*YR;GD1)Jg{@{ljDzUn&KQv$%-q(hA!q6 zkb7P2KVnuir8Hq2cn!IOJ9@_-(lCNKHCoJ5(ed}OM7!RJ^k(8lLNn*|?45C8fiG1s ziC!|I3&Nl-e>>@ti#6 zPwp|BEh714Sw}_a1y&sp>x-17%|!fYq;(Rbh$~k5MM2J-osI>1C^q;E*F!MaY z;FkMAeAdd2J|DLzz|Y_NK3#a8Pao(O#84snj;0T=x+U;bxe@9Ji(6(byUdL}qg}t0 zE18g5948KZj9G&g5gnt&yu@|`jGy3{ogs?k0j%82Z{ELz&cLQBN208e?She)XSz`1 zJ&uvctICzabtQ0GmneK1dZIT`9o)QTPRkkDwnFDQ(pcrQ)E={mfAunReSZJvpK>KS zW*!W9ubx2Y<{b1Sdx}7ZG-9H`M5wM6+TVIL8ItqSsdRpbp1!%4Pvop*R03c;AC`w+ zH9oPm7g^o|z>AGk>43o?G@bCd811PQSD1!U=fI7MXeW3a^n9(gbh>dq^a=kt)$1?3{ zJ_D#({S8rFS8^6yvMlFgvyUEqmBxIonXl5y7Mro5%Xt>eku!6dIR{HiZTh~9AA{*b zQ+NaTP>vukx&fE1rqA^xmqA2=QuJ98=T(-w$MlqBa**nqj*9pb&yG0I#%b3fzQp`f zs)?w_!^CFQSllPU1nHb-Q-4QlGO9KCX=zPij_KYf_wBB(5eFACWv?r@q)ExvNXf9L zV{L1?{nUV(urnC8fXU-L&sS^$Kkvb`l6d+sBbg}EjoGNb`d7W$);|kO?w$s5f|Eo1 z##Rm8?<K)RJe|4Kp|VoXPv>6K8mazkP%4-kl;ugSI)a& z&Zio2kR$0RE`BRjkA?vKB#-s4!={CwR7#D(s~ksU9)%~e{_)h^*qvu#JOhE+$sY$pkswiXu1)d|yv&^jG@f6Yr@5Lq1I|KAumfjYUs#d2DhjQp zM6i|%fAiIuN(dZ*?hXq_=5JbHTYysqIoCa$*$Q3Ik@7Ql?+#?Hk2XiKJUSI3bh45%L9cj zviIA*_rp5DY$M<{4wR$!e{NHhdcS81N+A%&Z;_R^FAl|w7ueV=Xn(?lM#fUpi#8LO z7Z_m6Vx<Es5vP6yXZ|zfJA|&lOU)_ z*e$(Rp}`qS1}uUo`#vcF`u&^@*{EsSEb_SLJJmG{A99?f<(c{3(&B`DUn;h5uei?K(vaZ;WuD@~6yv3!#3H|S^&xS4cP zi2Asq%0}(ptrk60rU4MU;30`GE@66L9qTDWgf)p$gbZQTFLnu+6qgmw|yK^Z6wt%|J`hpu1fw z10IaBQ8G@A3m@*`l=~qCTC?xh3cI@vKCpL5T`e|+Fwc~ZKR0!g6y4c`m%pUERjbw* zraT(qqydJFHv{&VGF68w5C$d@C|8^4DAiUv$Mwv1$LtAx}>+)q(L_0;U6^h5x=<#8yF}!Dc`pJKDYlZ>{a;phh>a=)ncd_N~@HdpUh) zm(C?9q&gDbbY72}+GjINKdXKn}jsIDv4)G!`6JW4^-QXO@ta91a(QCYYNg0KZ zHiF*+#wI&@?VCv$rX`W*rqfB)eYZ(R!-YH<@788FV>*02O@g;7=u-}eI1tjGxSP*F zaRqU%LI%AFwe5qUzYE+G%#WFO?_3<6-FUY5YVbImxM0VE?C@`Dzgs>bRS&;^o&L~l zz%9u6)?fA->yff7L7_S`TV! zLK{yRZsR8H+6o*kQp7wqFdz$cvqXMcYOT{mn%`+}h*!j3&PYDUe&yQ*^KVct9|RZ51p?!@vOLg~Bv&r9HPr3~!A3qp!Sa(zx@z=5Z9 z4T>#T>W>>F_w4w?9H*a(I8#?aExu_Hs6N38s zXP=v>A|M^f&Hm;aQnIqoo|>u~BfYTyv;{EzEa3vI0)nQzB$3v0(Z}7UWWb>W)&C(2 z?ImO)8BAw6cdU9IfF!m_#~EwiJn-kiEhdhJUNEW82W@z*dKp^SVq<0|LX>ABRnQw* zAzJxxcViM7!>N(1n7@(S6rtTLyhf9UxxZ|+|T9T z*OuXW!z*dm45+cpl?*55+$p1#q4y#ld|L&V+t_U$2LZj(?>V@>xv(rXEe;0~VCraF zHupfrpBpogg&QI3H=j*+Um>NGR5Ixh@DAWW3F4Gs-{do(e9TksGfWIF6 z5kEW~>}(qcP#$yc6iX3?kFK4?{(>hrT|3HEpP~A@nXU+2C{3*>NO!U;>r=c+!C+2` zDu^H#-veFFKjJ-{u7YDsYaly3E86oSjj9yg(p8iob&$b^@d~#?J^@A%!?5ol>C1rd zHj6pa>QpnCo15@nYp?DYsc_6En4!A3Ayx0%gDsJ7CNSTlHy>TFW;(i-lRVKJ*YFx8 z(7qfW5bYuJBVNWqGH0QoXSMeHA5Z-R_B)8$3Y-u79qhQB#PRlMizc|{UEHgbPzwk^ zz?oIN+$LcR#F&v%y14m9@y9KRT=ZOa@J_%1>9sSfYu*AYChyzTAAB=sp6an!9I7DW zs$#&ta~ysOi(Ombm`2X_BF*%Oq|42&arcq@0sYKL6wDkj{JjelH(R1+j z(>zi`@aA%$C`%!&XgavmGW_~&Pb^}E)u4Va`O6VM8st2sSV#W+5_T6#+Qn{IMkz8> zxdBd}J;w5~yk52Qgbm=KYo~+%#ng-$QKAIyi{4v09KFgy8iR8*tzJmd3I-GjRdNEs zLCy_aLh)shT5oTWO=my6nUpg%^_HE1L^MYeKvF_@u4rPN^-H~q!mFDyu)8M!Xn50(2&MZWS|U;BZxIyE(KHFcj{N!rewG6Kntu^;Zi0d4rl=)?0|D{19?n%eI@MVRXnR! z`;gqM!c*#Byx#tX zW_t~2w$}jCVLJ}C*ML$L{}Tu0u9z^$|;*-^jQ{s{2!Y%kPyM8UIl{x^9+tU#LYN~S(02{J1E zyV#C0H!o=q*@5ndwd-sUEhVAy6@+fAJk2ir}3yRkwTY&X{b zC84I|(AL-g_lYj%?Q zHfG9q+rO~AO535aoe%#ND7UM|cB$O%58GXo?jKm7q51zNA5?dKx2oAmx$o!1%=Db( zD{Yi#yHmf6l>WzPhJ*SdimjhG<0y}or+#&}ZvVo6r&-?HtMvb1XzV - - - - - - + \ No newline at end of file From d62345bbaaeb8bd93c5451739316dd0df90d3ba3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 12 Apr 2020 23:50:32 +0300 Subject: [PATCH 368/743] Show information about not loaded dictionaries in system.columns, system.tables and in "SHOW TABLES" executed in any database with the "Dictionary" engine. --- contrib/poco | 2 +- src/Common/ErrorCodes.cpp | 1 + src/Databases/DatabaseDictionary.cpp | 67 ++--- src/Databases/DatabaseDictionary.h | 2 + src/Databases/DatabaseOrdinary.cpp | 25 +- src/Databases/DatabaseWithDictionaries.cpp | 250 +++++++++--------- src/Databases/DatabaseWithDictionaries.h | 25 +- src/Databases/DatabasesCommon.cpp | 5 +- src/Databases/DatabasesCommon.h | 1 - src/Databases/DictionaryAttachInfo.h | 18 ++ src/Databases/IDatabase.h | 31 ++- .../getDictionaryConfigurationFromAST.h | 1 - .../ExternalDictionariesLoader.cpp | 14 + src/Interpreters/ExternalDictionariesLoader.h | 10 +- src/Interpreters/ExternalLoader.cpp | 121 ++++----- src/Interpreters/ExternalLoader.h | 40 ++- ...ExternalLoaderDatabaseConfigRepository.cpp | 29 +- src/Interpreters/InterpreterCreateQuery.cpp | 8 +- src/Interpreters/InterpreterDropQuery.cpp | 27 +- src/Storages/StorageDictionary.cpp | 132 ++++----- src/Storages/StorageDictionary.h | 43 +-- src/Storages/System/StorageSystemColumns.cpp | 2 +- .../System/StorageSystemDictionaries.cpp | 9 +- src/Storages/System/StorageSystemModels.cpp | 4 +- src/Storages/System/StorageSystemTables.cpp | 2 +- ...8_dictionaries_from_dictionaries.reference | 1 + .../0_stateless/01048_exists_query.sql | 2 +- ...01224_no_superfluous_dict_reload.reference | 7 +- .../01224_no_superfluous_dict_reload.sql | 8 +- .../01225_drop_dictionary_as_table.sql | 6 +- ...1225_show_create_table_from_dictionary.sql | 2 +- 31 files changed, 478 insertions(+), 417 deletions(-) create mode 100644 src/Databases/DictionaryAttachInfo.h diff --git a/contrib/poco b/contrib/poco index ddca76ba495..7d605a1ae5d 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit ddca76ba4956cb57150082394536cc43ff28f6fa +Subproject commit 7d605a1ae5d878294f91f68feb62ae51e9a04426 diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 27381811a8d..1f10893b08e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -491,6 +491,7 @@ namespace ErrorCodes extern const int CANNOT_ASSIGN_ALTER = 517; extern const int CANNOT_COMMIT_OFFSET = 518; extern const int NO_REMOTE_SHARD_AVAILABLE = 519; + extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE = 520; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 9e7788bf846..a0f52a8f39a 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -15,6 +16,18 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; + extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY; +} + +namespace +{ + StoragePtr createStorageDictionary(const String & database_name, const ExternalLoader::LoadResult & load_result) + { + if (!load_result.config) + return nullptr; + DictionaryStructure dictionary_structure = ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); + return StorageDictionary::create(StorageID(database_name, load_result.name), load_result.name, dictionary_structure); + } } DatabaseDictionary::DatabaseDictionary(const String & name_) @@ -26,29 +39,12 @@ DatabaseDictionary::DatabaseDictionary(const String & name_) Tables DatabaseDictionary::listTables(const Context & context, const FilterByNameFunction & filter_by_name) { Tables tables; - ExternalLoader::LoadResults load_results; - if (filter_by_name) + auto load_results = context.getExternalDictionariesLoader().getLoadResults(filter_by_name); + for (auto & load_result : load_results) { - /// If `filter_by_name` is set, we iterate through all dictionaries with such names. That's why we need to load all of them. - load_results = context.getExternalDictionariesLoader().tryLoad(filter_by_name); - } - else - { - /// If `filter_by_name` isn't set, we iterate through only already loaded dictionaries. We don't try to load all dictionaries in this case. - load_results = context.getExternalDictionariesLoader().getCurrentLoadResults(); - } - - for (const auto & load_result: load_results) - { - /// Load tables only from XML dictionaries, don't touch other - if (load_result.object && load_result.repository_name.empty()) - { - auto dict_ptr = std::static_pointer_cast(load_result.object); - auto dict_name = dict_ptr->getName(); - const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); - auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - tables[dict_name] = StorageDictionary::create(StorageID(getDatabaseName(), dict_name), ColumnsDescription{columns}, context, true, dict_name); - } + auto storage = createStorageDictionary(getDatabaseName(), load_result); + if (storage) + tables.emplace(storage->getStorageID().table_name, storage); } return tables; } @@ -64,15 +60,8 @@ StoragePtr DatabaseDictionary::tryGetTable( const Context & context, const String & table_name) const { - auto dict_ptr = context.getExternalDictionariesLoader().tryGetDictionary(table_name, true /*load*/); - if (dict_ptr) - { - const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); - auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - return StorageDictionary::create(StorageID(getDatabaseName(), table_name), ColumnsDescription{columns}, context, true, table_name); - } - - return {}; + auto load_result = context.getExternalDictionariesLoader().getLoadResult(table_name); + return createStorageDictionary(getDatabaseName(), load_result); } DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) @@ -82,7 +71,7 @@ DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context & bool DatabaseDictionary::empty(const Context & context) const { - return !context.getExternalDictionariesLoader().hasCurrentlyLoadedObjects(); + return !context.getExternalDictionariesLoader().hasObjects(); } ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context, @@ -92,15 +81,17 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context, { WriteBufferFromString buffer(query); - const auto & dictionaries = context.getExternalDictionariesLoader(); - auto dictionary = throw_on_error ? dictionaries.getDictionary(table_name) - : dictionaries.tryGetDictionary(table_name, true /*load*/); - if (!dictionary) + auto load_result = context.getExternalDictionariesLoader().getLoadResult(table_name); + if (!load_result.config) + { + if (throw_on_error) + throw Exception{"Dictionary " + backQuote(table_name) + " doesn't exist", ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY}; return {}; + } - auto names_and_types = StorageDictionary::getNamesAndTypes(dictionary->getStructure()); + auto names_and_types = StorageDictionary::getNamesAndTypes(ExternalDictionariesLoader::getDictionaryStructure(*load_result.config)); buffer << "CREATE TABLE " << backQuoteIfNeed(database_name) << '.' << backQuoteIfNeed(table_name) << " ("; - buffer << StorageDictionary::generateNamesAndTypesDescription(names_and_types.begin(), names_and_types.end()); + buffer << StorageDictionary::generateNamesAndTypesDescription(names_and_types); buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")"; } diff --git a/src/Databases/DatabaseDictionary.h b/src/Databases/DatabaseDictionary.h index b586cb1403f..0001a59efa6 100644 --- a/src/Databases/DatabaseDictionary.h +++ b/src/Databases/DatabaseDictionary.h @@ -43,6 +43,8 @@ public: ASTPtr getCreateDatabaseQuery(const Context & context) const override; + bool shouldBeEmptyOnDetach() const override { return false; } + void shutdown() override; protected: diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 11c4a4400cd..ff375a26b13 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -74,18 +75,24 @@ namespace void tryAttachDictionary( - Context & context, - const ASTCreateQuery & query, - DatabaseOrdinary & database) + const ASTPtr & query, + DatabaseOrdinary & database, + const String & metadata_path) { - assert(query.is_dictionary); + auto & create_query = query->as(); + assert(create_query.is_dictionary); try { - database.attachDictionary(query.table, context); + Poco::File meta_file(metadata_path); + auto config = getDictionaryConfigurationFromAST(create_query); + time_t modification_time = meta_file.getLastModified().epochTime(); + database.attachDictionary(create_query.table, DictionaryAttachInfo{query, config, modification_time}); } catch (Exception & e) { - e.addMessage("Cannot attach table '" + backQuote(query.table) + "' from query " + serializeAST(query)); + e.addMessage("Cannot attach dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(create_query.table) + + " from metadata file " + metadata_path + + " from query " + serializeAST(*query)); throw; } } @@ -173,12 +180,12 @@ void DatabaseOrdinary::loadStoredObjects( /// Attach dictionaries. attachToExternalDictionariesLoader(context); - for (const auto & name_with_query : file_names) + for (const auto & [name, query] : file_names) { - auto create_query = name_with_query.second->as(); + auto create_query = query->as(); if (create_query.is_dictionary) { - tryAttachDictionary(context, create_query, *this); + tryAttachDictionary(query, *this, getMetadataPath() + name); /// Messages, so that it's not boring to wait for the server to load for a long time. logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch); diff --git a/src/Databases/DatabaseWithDictionaries.cpp b/src/Databases/DatabaseWithDictionaries.cpp index 6673fdf8075..aec8b66e572 100644 --- a/src/Databases/DatabaseWithDictionaries.cpp +++ b/src/Databases/DatabaseWithDictionaries.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include #include @@ -24,46 +26,80 @@ namespace ErrorCodes { extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY; extern const int TABLE_ALREADY_EXISTS; - extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_DICTIONARY; extern const int DICTIONARY_ALREADY_EXISTS; extern const int FILE_DOESNT_EXIST; - extern const int CANNOT_GET_CREATE_TABLE_QUERY; } -void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const Context & context) +void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const DictionaryAttachInfo & attach_info) { String full_name = getDatabaseName() + "." + dictionary_name; { std::lock_guard lock(mutex); - if (!dictionaries.emplace(dictionary_name).second) + auto [it, inserted] = dictionaries.emplace(dictionary_name, attach_info); + if (!inserted) throw Exception("Dictionary " + full_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); + + /// Attach the dictionary as table too. + try + { + attachTableUnlocked( + dictionary_name, + StorageDictionary::create( + StorageID(getDatabaseName(), dictionary_name), + full_name, + ExternalDictionariesLoader::getDictionaryStructure(*attach_info.config))); + } + catch (...) + { + dictionaries.erase(it); + throw; + } } CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, full_name, static_cast(ExternalLoaderStatus::NOT_LOADED)); + /// ExternalLoader::reloadConfig() will find out that the dictionary's config has been added /// and in case `dictionaries_lazy_load == false` it will load the dictionary. - const auto & external_loader = context.getExternalDictionariesLoader(); - external_loader.reloadConfig(getDatabaseName(), full_name); + external_loader->reloadConfig(getDatabaseName(), full_name); } -void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name, const Context & context) +void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name) +{ + DictionaryAttachInfo attach_info; + detachDictionaryImpl(dictionary_name, attach_info); +} + +void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info) { String full_name = getDatabaseName() + "." + dictionary_name; + { std::lock_guard lock(mutex); auto it = dictionaries.find(dictionary_name); if (it == dictionaries.end()) - throw Exception("Dictionary " + full_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + throw Exception("Dictionary " + full_name + " doesn't exist.", ErrorCodes::UNKNOWN_DICTIONARY); + attach_info = std::move(it->second); dictionaries.erase(it); + + /// Detach the dictionary as table too. + try + { + detachTableUnlocked(dictionary_name); + } + catch (...) + { + dictionaries.emplace(dictionary_name, std::move(attach_info)); + throw; + } } CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, full_name); + /// ExternalLoader::reloadConfig() will find out that the dictionary's config has been removed /// and therefore it will unload the dictionary. - const auto & external_loader = context.getExternalDictionariesLoader(); - external_loader.reloadConfig(getDatabaseName(), full_name); - + external_loader->reloadConfig(getDatabaseName(), full_name); } void DatabaseWithDictionaries::createDictionary(const Context & context, const String & dictionary_name, const ASTPtr & query) @@ -85,8 +121,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S /// A dictionary with the same full name could be defined in *.xml config files. String full_name = getDatabaseName() + "." + dictionary_name; - const auto & external_loader = context.getExternalDictionariesLoader(); - if (external_loader.getCurrentStatus(full_name) != ExternalLoader::Status::NOT_EXIST) + if (external_loader->getCurrentStatus(full_name) != ExternalLoader::Status::NOT_EXIST) throw Exception( "Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); @@ -117,23 +152,22 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S /// Add a temporary repository containing the dictionary. /// We need this temp repository to try loading the dictionary before actually attaching it to the database. - auto temp_repository - = const_cast(external_loader) /// the change of ExternalDictionariesLoader is temporary - .addConfigRepository(std::make_unique( - getDatabaseName(), dictionary_metadata_tmp_path, getDictionaryConfigurationFromAST(query->as()))); + auto temp_repository = external_loader->addConfigRepository(std::make_unique( + getDatabaseName(), dictionary_metadata_tmp_path, getDictionaryConfigurationFromAST(query->as()))); bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true); if (!lazy_load) { /// load() is called here to force loading the dictionary, wait until the loading is finished, /// and throw an exception if the loading is failed. - external_loader.load(full_name); + external_loader->load(full_name); } - attachDictionary(dictionary_name, context); + auto config = getDictionaryConfigurationFromAST(query->as()); + attachDictionary(dictionary_name, DictionaryAttachInfo{query, config, time(nullptr)}); SCOPE_EXIT({ if (!succeeded) - detachDictionary(dictionary_name, context); + detachDictionary(dictionary_name); }); /// If it was ATTACH query and file with dictionary metadata already exist @@ -142,94 +176,31 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S /// ExternalDictionariesLoader doesn't know we renamed the metadata path. /// So we have to manually call reloadConfig() here. - external_loader.reloadConfig(getDatabaseName(), full_name); + external_loader->reloadConfig(getDatabaseName(), full_name); /// Everything's ok. succeeded = true; } -void DatabaseWithDictionaries::removeDictionary(const Context & context, const String & dictionary_name) +void DatabaseWithDictionaries::removeDictionary(const Context &, const String & dictionary_name) { - detachDictionary(dictionary_name, context); - - String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); + DictionaryAttachInfo attach_info; + detachDictionaryImpl(dictionary_name, attach_info); try { + String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); Poco::File(dictionary_metadata_path).remove(); CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, getDatabaseName() + "." + dictionary_name); } catch (...) { /// If remove was not possible for some reason - attachDictionary(dictionary_name, context); + attachDictionary(dictionary_name, attach_info); throw; } } -StoragePtr DatabaseWithDictionaries::tryGetTableImpl(const Context & context, const String & table_name, bool load) const -{ - if (auto table_ptr = DatabaseWithOwnTablesBase::tryGetTable(context, table_name)) - return table_ptr; - - if (isDictionaryExist(context, table_name)) - /// We don't need lock database here, because database doesn't store dictionary itself - /// just metadata - return getDictionaryStorage(context, table_name, load); - - return {}; -} -StoragePtr DatabaseWithDictionaries::tryGetTable(const Context & context, const String & table_name) const -{ - return tryGetTableImpl(context, table_name, true /*load*/); -} - -ASTPtr DatabaseWithDictionaries::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const -{ - ASTPtr ast; - bool has_table = tryGetTableImpl(context, table_name, false /*load*/) != nullptr; - auto table_metadata_path = getObjectMetadataPath(table_name); - try - { - ast = getCreateQueryFromMetadata(context, table_metadata_path, throw_on_error); - } - catch (const Exception & e) - { - if (!has_table && e.code() == ErrorCodes::FILE_DOESNT_EXIST && throw_on_error) - throw Exception{"Table " + backQuote(table_name) + " doesn't exist", - ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY}; - else if (throw_on_error) - throw; - } - return ast; -} - -DatabaseTablesIteratorPtr DatabaseWithDictionaries::getTablesWithDictionaryTablesIterator( - const Context & context, const FilterByNameFunction & filter_by_dictionary_name) -{ - /// NOTE: it's not atomic - auto tables_it = getTablesIterator(context, filter_by_dictionary_name); - auto dictionaries_it = getDictionariesIterator(context, filter_by_dictionary_name); - - Tables result; - while (tables_it && tables_it->isValid()) - { - result.emplace(tables_it->name(), tables_it->table()); - tables_it->next(); - } - - while (dictionaries_it && dictionaries_it->isValid()) - { - auto table_name = dictionaries_it->name(); - auto table_ptr = getDictionaryStorage(context, table_name, false /*load*/); - if (table_ptr) - result.emplace(table_name, table_ptr); - dictionaries_it->next(); - } - - return std::make_unique(result); -} - DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_dictionary_name) { std::lock_guard lock(mutex); @@ -237,9 +208,9 @@ DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterato return std::make_unique(dictionaries); Dictionaries filtered_dictionaries; - for (const auto & dictionary_name : dictionaries) + for (const auto & dictionary_name : dictionaries | boost::adaptors::map_keys) if (filter_by_dictionary_name(dictionary_name)) - filtered_dictionaries.emplace(dictionary_name); + filtered_dictionaries.emplace_back(dictionary_name); return std::make_unique(std::move(filtered_dictionaries)); } @@ -249,44 +220,84 @@ bool DatabaseWithDictionaries::isDictionaryExist(const Context & /*context*/, co return dictionaries.find(dictionary_name) != dictionaries.end(); } -StoragePtr DatabaseWithDictionaries::getDictionaryStorage(const Context & context, const String & table_name, bool load) const -{ - auto dict_name = database_name + "." + table_name; - const auto & external_loader = context.getExternalDictionariesLoader(); - auto dict_ptr = external_loader.tryGetDictionary(dict_name, load); - if (dict_ptr) - { - const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); - auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - return StorageDictionary::create(StorageID(database_name, table_name), ColumnsDescription{columns}, context, true, dict_name); - } - return nullptr; -} - ASTPtr DatabaseWithDictionaries::getCreateDictionaryQueryImpl( const Context & context, const String & dictionary_name, bool throw_on_error) const { - ASTPtr ast; - - auto dictionary_metadata_path = getObjectMetadataPath(dictionary_name); - ast = getCreateQueryFromMetadata(context, dictionary_metadata_path, throw_on_error); - if (!ast && throw_on_error) { - /// Handle system.* tables for which there are no table.sql files. - bool has_dictionary = isDictionaryExist(context, dictionary_name); - - auto msg = has_dictionary ? "There is no CREATE DICTIONARY query for table " : "There is no metadata file for dictionary "; - - throw Exception(msg + backQuote(dictionary_name), ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY); + /// Try to get create query ifg for an attached dictionary. + std::lock_guard lock{mutex}; + auto it = dictionaries.find(dictionary_name); + if (it != dictionaries.end()) + { + ASTPtr ast = it->second.create_query->clone(); + auto & create_query = ast->as(); + create_query.attach = false; + create_query.database = getDatabaseName(); + return ast; + } } - return ast; + /// Try to get create query for non-attached dictionary. + ASTPtr ast; + try + { + auto dictionary_metadata_path = getObjectMetadataPath(dictionary_name); + ast = getCreateQueryFromMetadata(context, dictionary_metadata_path, throw_on_error); + } + catch (const Exception & e) + { + if (throw_on_error && (e.code() != ErrorCodes::FILE_DOESNT_EXIST)) + throw; + } + + if (ast) + { + const auto * create_query = ast->as(); + if (create_query && create_query->is_dictionary) + return ast; + } + if (throw_on_error) + throw Exception{"Dictionary " + backQuote(dictionary_name) + " doesn't exist", + ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY}; + return nullptr; +} + + +Poco::AutoPtr DatabaseWithDictionaries::getDictionaryConfiguration(const String & dictionary_name) const +{ + std::lock_guard lock(mutex); + auto it = dictionaries.find(dictionary_name); + if (it != dictionaries.end()) + return it->second.config; + throw Exception("Dictionary " + backQuote(dictionary_name) + " doesn't exist", ErrorCodes::UNKNOWN_DICTIONARY); +} + +time_t DatabaseWithDictionaries::getObjectMetadataModificationTime(const String & object_name) const +{ + { + std::lock_guard lock(mutex); + auto it = dictionaries.find(object_name); + if (it != dictionaries.end()) + return it->second.modification_time; + } + return DatabaseOnDisk::getObjectMetadataModificationTime(object_name); +} + + +bool DatabaseWithDictionaries::empty(const Context &) const +{ + std::lock_guard lock{mutex}; + return tables.empty() && dictionaries.empty(); } void DatabaseWithDictionaries::shutdown() { + { + std::lock_guard lock(mutex); + dictionaries.clear(); + } detachFromExternalDictionariesLoader(); DatabaseOnDisk::shutdown(); } @@ -295,8 +306,9 @@ DatabaseWithDictionaries::~DatabaseWithDictionaries() = default; void DatabaseWithDictionaries::attachToExternalDictionariesLoader(Context & context) { - database_as_config_repo_for_external_loader = context.getExternalDictionariesLoader().addConfigRepository( - std::make_unique(*this, context)); + external_loader = &context.getExternalDictionariesLoader(); + database_as_config_repo_for_external_loader + = external_loader->addConfigRepository(std::make_unique(*this, context)); } void DatabaseWithDictionaries::detachFromExternalDictionariesLoader() diff --git a/src/Databases/DatabaseWithDictionaries.h b/src/Databases/DatabaseWithDictionaries.h index 50e4dca671f..fe78aa98a19 100644 --- a/src/Databases/DatabaseWithDictionaries.h +++ b/src/Databases/DatabaseWithDictionaries.h @@ -8,9 +8,9 @@ namespace DB class DatabaseWithDictionaries : public DatabaseOnDisk { public: - void attachDictionary(const String & name, const Context & context) override; + void attachDictionary(const String & dictionary_name, const DictionaryAttachInfo & attach_info) override; - void detachDictionary(const String & name, const Context & context) override; + void detachDictionary(const String & dictionary_name) override; void createDictionary(const Context & context, const String & dictionary_name, @@ -18,15 +18,15 @@ public: void removeDictionary(const Context & context, const String & dictionary_name) override; - StoragePtr tryGetTable(const Context & context, const String & table_name) const override; - - ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const override; - - DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name) override; + bool isDictionaryExist(const Context & context, const String & dictionary_name) const override; DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name) override; - bool isDictionaryExist(const Context & context, const String & dictionary_name) const override; + Poco::AutoPtr getDictionaryConfiguration(const String & /*name*/) const override; + + time_t getObjectMetadataModificationTime(const String & object_name) const override; + + bool empty(const Context & context) const override; void shutdown() override; @@ -39,16 +39,17 @@ protected: void attachToExternalDictionariesLoader(Context & context); void detachFromExternalDictionariesLoader(); - StoragePtr getDictionaryStorage(const Context & context, const String & table_name, bool load) const; + void detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info); ASTPtr getCreateDictionaryQueryImpl(const Context & context, const String & dictionary_name, bool throw_on_error) const override; -private: - ext::scope_guard database_as_config_repo_for_external_loader; + std::unordered_map dictionaries; - StoragePtr tryGetTableImpl(const Context & context, const String & table_name, bool load) const; +private: + ExternalDictionariesLoader * external_loader = nullptr; + ext::scope_guard database_as_config_repo_for_external_loader; }; } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 9d9c0707a7c..1b54424b484 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -27,7 +27,7 @@ bool DatabaseWithOwnTablesBase::isTableExist( const String & table_name) const { std::lock_guard lock(mutex); - return tables.find(table_name) != tables.end() || dictionaries.find(table_name) != dictionaries.end(); + return tables.find(table_name) != tables.end(); } StoragePtr DatabaseWithOwnTablesBase::tryGetTable( @@ -58,7 +58,7 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Con bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const { std::lock_guard lock(mutex); - return tables.empty() && dictionaries.empty(); + return tables.empty(); } StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name) @@ -125,7 +125,6 @@ void DatabaseWithOwnTablesBase::shutdown() std::lock_guard lock(mutex); tables.clear(); - dictionaries.clear(); } DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase() diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 3bf7460da01..0515c16deaf 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -42,7 +42,6 @@ public: protected: mutable std::mutex mutex; Tables tables; - Dictionaries dictionaries; Poco::Logger * log; DatabaseWithOwnTablesBase(const String & name_, const String & logger); diff --git a/src/Databases/DictionaryAttachInfo.h b/src/Databases/DictionaryAttachInfo.h new file mode 100644 index 00000000000..b2214d26f3c --- /dev/null +++ b/src/Databases/DictionaryAttachInfo.h @@ -0,0 +1,18 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +struct DictionaryAttachInfo +{ + ASTPtr create_query; + Poco::AutoPtr config; + time_t modification_time; +}; + +} diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 1dc1f9eb36b..9ef649dff58 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -5,8 +5,11 @@ #include #include #include +#include #include +#include +#include #include #include #include @@ -18,11 +21,10 @@ namespace DB class Context; struct Settings; struct ConstraintsDescription; -class ColumnsDescription; struct IndicesDescription; struct TableStructureWriteLockHolder; class ASTCreateQuery; -using Dictionaries = std::set; +using Dictionaries = std::vector; namespace ErrorCodes { @@ -74,9 +76,14 @@ private: public: DatabaseDictionariesSnapshotIterator() = default; DatabaseDictionariesSnapshotIterator(Dictionaries & dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {} - DatabaseDictionariesSnapshotIterator(Dictionaries && dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {} + DatabaseDictionariesSnapshotIterator(const std::unordered_map & dictionaries_) + { + boost::range::copy(dictionaries_ | boost::adaptors::map_keys, std::back_inserter(dictionaries)); + it = dictionaries.begin(); + } + void next() { ++it; } bool isValid() const { return !dictionaries.empty() && it != dictionaries.end(); } @@ -140,12 +147,6 @@ public: return std::make_unique(); } - /// Get an iterator to pass through all the tables and dictionary tables. - virtual DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name = {}) - { - return getTablesIterator(context, filter_by_name); - } - /// Is the database empty. virtual bool empty(const Context & context) const = 0; @@ -192,7 +193,7 @@ public: /// Add dictionary to the database, but do not add it to the metadata. The database may not support this method. /// If dictionaries_lazy_load is false it also starts loading the dictionary asynchronously. - virtual void attachDictionary(const String & /*name*/, const Context & /*context*/) + virtual void attachDictionary(const String & /* dictionary_name */, const DictionaryAttachInfo & /* attach_info */) { throw Exception("There is no ATTACH DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } @@ -204,7 +205,7 @@ public: } /// Forget about the dictionary without deleting it. The database may not support this method. - virtual void detachDictionary(const String & /*name*/, const Context & /*context*/) + virtual void detachDictionary(const String & /*name*/) { throw Exception("There is no DETACH DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } @@ -260,6 +261,11 @@ public: return getCreateDictionaryQueryImpl(context, name, true); } + virtual Poco::AutoPtr getDictionaryConfiguration(const String & /*name*/) const + { + throw Exception(getEngineName() + ": getDictionaryConfiguration() is not supported", ErrorCodes::NOT_IMPLEMENTED); + } + /// Get the CREATE DATABASE query for current database. virtual ASTPtr getCreateDatabaseQuery(const Context & /*context*/) const = 0; @@ -276,6 +282,9 @@ public: /// Returns metadata path of a concrete table if the database supports it, empty string otherwise virtual String getObjectMetadataPath(const String & /*table_name*/) const { return {}; } + /// All tables and dictionaries should be detached before detaching the database. + virtual bool shouldBeEmptyOnDetach() const { return true; } + /// Ask all tables to complete the background threads they are using and delete all table objects. virtual void shutdown() = 0; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.h b/src/Dictionaries/getDictionaryConfigurationFromAST.h index bb48765c492..7dd672ccf4b 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.h +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.h @@ -11,5 +11,4 @@ using DictionaryConfigurationPtr = Poco::AutoPtr #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -33,6 +34,19 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( return DictionaryFactory::instance().create(name, config, key_in_config, context, dictionary_from_database); } + +DictionaryStructure +ExternalDictionariesLoader::getDictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config) +{ + return {config, key_in_config + ".structure"}; +} + +DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const ObjectConfig & config) +{ + return getDictionaryStructure(*config.config, config.key_in_config); +} + + void ExternalDictionariesLoader::resetAll() { #if USE_MYSQL diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 4a54a9963e7..e69046706a3 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -23,14 +23,14 @@ public: return std::static_pointer_cast(load(name)); } - DictPtr tryGetDictionary(const std::string & name, bool load) const + DictPtr tryGetDictionary(const std::string & name) const { - if (load) - return std::static_pointer_cast(tryLoad(name)); - else - return std::static_pointer_cast(getCurrentLoadResult(name).object); + return std::static_pointer_cast(tryLoad(name)); } + static DictionaryStructure getDictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config = "dictionary"); + static DictionaryStructure getDictionaryStructure(const ObjectConfig & config); + static void resetAll(); protected: diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 893d9aa61f9..4b83616fe50 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -94,15 +94,6 @@ namespace }; } -struct ExternalLoader::ObjectConfig -{ - Poco::AutoPtr config; - String key_in_config; - String repository_name; - bool from_temp_repository = false; - String path; -}; - /** Reads configurations from configuration repository and parses it. */ @@ -141,7 +132,7 @@ public: settings = settings_; } - using ObjectConfigsPtr = std::shared_ptr>; + using ObjectConfigsPtr = std::shared_ptr>>; /// Reads all repositories. ObjectConfigsPtr read() @@ -176,8 +167,9 @@ private: struct FileInfo { Poco::Timestamp last_update_time = 0; - std::vector> objects; // Parsed contents of the file. bool in_use = true; // Whether the `FileInfo` should be destroyed because the correspondent file is deleted. + Poco::AutoPtr file_contents; // Parsed contents of the file. + std::unordered_map objects; }; struct RepositoryInfo @@ -280,14 +272,15 @@ private: } LOG_TRACE(log, "Loading config file '" << path << "'."); - auto file_contents = repository.load(path); + file_info.file_contents = repository.load(path); + auto & file_contents = *file_info.file_contents; /// get all objects' definitions Poco::Util::AbstractConfiguration::Keys keys; - file_contents->keys(keys); + file_contents.keys(keys); /// for each object defined in repositories - std::vector> object_configs_from_file; + std::unordered_map objects; for (const auto & key : keys) { if (!startsWith(key, settings.external_config)) @@ -297,7 +290,7 @@ private: continue; } - String object_name = file_contents->getString(key + "." + settings.external_name); + String object_name = file_contents.getString(key + "." + settings.external_name); if (object_name.empty()) { LOG_WARNING(log, path << ": node '" << key << "' defines " << type_name << " with an empty name. It's not allowed"); @@ -306,14 +299,14 @@ private: String database; if (!settings.external_database.empty()) - database = file_contents->getString(key + "." + settings.external_database, ""); + database = file_contents.getString(key + "." + settings.external_database, ""); if (!database.empty()) object_name = database + "." + object_name; - object_configs_from_file.emplace_back(object_name, ObjectConfig{file_contents, key, {}, {}, {}}); + objects.emplace(object_name, key); } - file_info.objects = std::move(object_configs_from_file); + file_info.objects = std::move(objects); file_info.last_update_time = update_time_from_repository; file_info.in_use = true; return true; @@ -333,33 +326,36 @@ private: need_collect_object_configs = false; // Generate new result. - auto new_configs = std::make_shared>(); + auto new_configs = std::make_shared>>(); for (const auto & [repository, repository_info] : repositories) { for (const auto & [path, file_info] : repository_info.files) { - for (const auto & [object_name, object_config] : file_info.objects) + for (const auto & [object_name, key_in_config] : file_info.objects) { auto already_added_it = new_configs->find(object_name); if (already_added_it == new_configs->end()) { - auto & new_config = new_configs->emplace(object_name, object_config).first->second; - new_config.from_temp_repository = repository->isTemporary(); - new_config.repository_name = repository->getName(); - new_config.path = path; + auto new_config = std::make_shared(); + new_config->config = file_info.file_contents; + new_config->key_in_config = key_in_config; + new_config->repository_name = repository->getName(); + new_config->from_temp_repository = repository->isTemporary(); + new_config->path = path; + new_configs->emplace(object_name, std::move(new_config)); } else { const auto & already_added = already_added_it->second; - if (!already_added.from_temp_repository && !repository->isTemporary()) + if (!already_added->from_temp_repository && !repository->isTemporary()) { LOG_WARNING( log, type_name << " '" << object_name << "' is found " - << (((path == already_added.path) && (repository->getName() == already_added.repository_name)) + << (((path == already_added->path) && (repository->getName() == already_added->repository_name)) ? ("twice in the same file '" + path + "'") - : ("both in file '" + already_added.path + "' and '" + path + "'"))); + : ("both in file '" + already_added->path + "' and '" + path + "'"))); } } } @@ -440,13 +436,10 @@ public: else { const auto & new_config = new_config_it->second; - bool config_is_same = isSameConfiguration(*info.object_config.config, info.object_config.key_in_config, *new_config.config, new_config.key_in_config); - info.object_config = new_config; + bool config_is_same = isSameConfiguration(*info.config->config, info.config->key_in_config, *new_config->config, new_config->key_in_config); + info.config = new_config; if (!config_is_same) { - /// Configuration has been changed. - info.object_config = new_config; - if (info.triedToLoad()) { /// The object has been tried to load before, so it is currently in use or was in use @@ -531,7 +524,7 @@ public: /// Returns the load result of the object. template - ReturnType getCurrentLoadResult(const String & name) const + ReturnType getLoadResult(const String & name) const { std::lock_guard lock{mutex}; const Info * info = getInfo(name); @@ -543,13 +536,13 @@ public: /// Returns all the load results as a map. /// The function doesn't load anything, it just returns the current load results as is. template - ReturnType getCurrentLoadResults(const FilterByNameFunction & filter) const + ReturnType getLoadResults(const FilterByNameFunction & filter) const { std::lock_guard lock{mutex}; return collectLoadResults(filter); } - size_t getNumberOfCurrentlyLoadedObjects() const + size_t getNumberOfLoadedObjects() const { std::lock_guard lock{mutex}; size_t count = 0; @@ -562,7 +555,7 @@ public: return count; } - bool hasCurrentlyLoadedObjects() const + bool hasLoadedObjects() const { std::lock_guard lock{mutex}; for (auto & name_info : infos) @@ -581,6 +574,12 @@ public: return names; } + size_t getNumberOfObjects() const + { + std::lock_guard lock{mutex}; + return infos.size(); + } + /// Tries to load a specified object during the timeout. template ReturnType tryLoad(const String & name, Duration timeout) @@ -698,7 +697,7 @@ public: private: struct Info { - Info(const String & name_, const ObjectConfig & object_config_) : name(name_), object_config(object_config_) {} + Info(const String & name_, const std::shared_ptr & config_) : name(name_), config(config_) {} bool loaded() const { return object != nullptr; } bool failed() const { return !object && exception; } @@ -737,8 +736,7 @@ private: result.loading_start_time = loading_start_time; result.last_successful_update_time = last_successful_update_time; result.loading_duration = loadingDuration(); - result.origin = object_config.path; - result.repository_name = object_config.repository_name; + result.config = config; return result; } else @@ -750,7 +748,7 @@ private: String name; LoadablePtr object; - ObjectConfig object_config; + std::shared_ptr config; TimePoint loading_start_time; TimePoint loading_end_time; TimePoint last_successful_update_time; @@ -784,7 +782,7 @@ private: results.reserve(infos.size()); for (const auto & [name, info] : infos) { - if (filter(name)) + if (!filter || filter(name)) { auto result = info.template getLoadResult(); if constexpr (std::is_same_v) @@ -838,7 +836,7 @@ private: bool all_ready = true; for (auto & [name, info] : infos) { - if (!filter(name)) + if (filter && !filter(name)) continue; if (info.state_id >= min_id) @@ -955,7 +953,7 @@ private: previous_version_as_base_for_loading = nullptr; /// Need complete reloading, cannot use the previous version. /// Loading. - auto [new_object, new_exception] = loadSingleObject(name, info->object_config, previous_version_as_base_for_loading); + auto [new_object, new_exception] = loadSingleObject(name, *info->config, previous_version_as_base_for_loading); if (!new_object && !new_exception) throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR); @@ -1296,9 +1294,9 @@ void ExternalLoader::enablePeriodicUpdates(bool enable_) periodic_updater->enable(enable_); } -bool ExternalLoader::hasCurrentlyLoadedObjects() const +bool ExternalLoader::hasLoadedObjects() const { - return loading_dispatcher->hasCurrentlyLoadedObjects(); + return loading_dispatcher->hasLoadedObjects(); } ExternalLoader::Status ExternalLoader::getCurrentStatus(const String & name) const @@ -1307,30 +1305,35 @@ ExternalLoader::Status ExternalLoader::getCurrentStatus(const String & name) con } template -ReturnType ExternalLoader::getCurrentLoadResult(const String & name) const +ReturnType ExternalLoader::getLoadResult(const String & name) const { - return loading_dispatcher->getCurrentLoadResult(name); + return loading_dispatcher->getLoadResult(name); } template -ReturnType ExternalLoader::getCurrentLoadResults(const FilterByNameFunction & filter) const +ReturnType ExternalLoader::getLoadResults(const FilterByNameFunction & filter) const { - return loading_dispatcher->getCurrentLoadResults(filter); + return loading_dispatcher->getLoadResults(filter); } -ExternalLoader::Loadables ExternalLoader::getCurrentlyLoadedObjects() const +ExternalLoader::Loadables ExternalLoader::getLoadedObjects() const { - return getCurrentLoadResults(); + return getLoadResults(); } -ExternalLoader::Loadables ExternalLoader::getCurrentlyLoadedObjects(const FilterByNameFunction & filter) const +ExternalLoader::Loadables ExternalLoader::getLoadedObjects(const FilterByNameFunction & filter) const { - return getCurrentLoadResults(filter); + return getLoadResults(filter); } -size_t ExternalLoader::getNumberOfCurrentlyLoadedObjects() const +size_t ExternalLoader::getNumberOfLoadedObjects() const { - return loading_dispatcher->getNumberOfCurrentlyLoadedObjects(); + return loading_dispatcher->getNumberOfLoadedObjects(); +} + +size_t ExternalLoader::getNumberOfObjects() const +{ + return loading_dispatcher->getNumberOfObjects(); } template @@ -1456,10 +1459,10 @@ ExternalLoader::LoadablePtr ExternalLoader::createObject( return create(name, *config.config, config.key_in_config, config.repository_name); } -template ExternalLoader::LoadablePtr ExternalLoader::getCurrentLoadResult(const String &) const; -template ExternalLoader::LoadResult ExternalLoader::getCurrentLoadResult(const String &) const; -template ExternalLoader::Loadables ExternalLoader::getCurrentLoadResults(const FilterByNameFunction &) const; -template ExternalLoader::LoadResults ExternalLoader::getCurrentLoadResults(const FilterByNameFunction &) const; +template ExternalLoader::LoadablePtr ExternalLoader::getLoadResult(const String &) const; +template ExternalLoader::LoadResult ExternalLoader::getLoadResult(const String &) const; +template ExternalLoader::Loadables ExternalLoader::getLoadResults(const FilterByNameFunction &) const; +template ExternalLoader::LoadResults ExternalLoader::getLoadResults(const FilterByNameFunction &) const; template ExternalLoader::LoadablePtr ExternalLoader::tryLoad(const String &, Duration) const; template ExternalLoader::LoadResult ExternalLoader::tryLoad(const String &, Duration) const; diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index a9a94ca615e..bcf01eb6625 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -53,17 +53,25 @@ public: using Duration = std::chrono::milliseconds; using TimePoint = std::chrono::system_clock::time_point; + struct ObjectConfig + { + Poco::AutoPtr config; + String key_in_config; + String repository_name; + bool from_temp_repository = false; + String path; + }; + struct LoadResult { Status status = Status::NOT_EXIST; String name; LoadablePtr object; - String origin; TimePoint loading_start_time; TimePoint last_successful_update_time; Duration loading_duration; std::exception_ptr exception; - std::string repository_name; + std::shared_ptr config; }; using LoadResults = std::vector; @@ -99,26 +107,32 @@ public: /// Returns the result of loading the object. /// The function doesn't load anything, it just returns the current load result as is. template , void>> - ReturnType getCurrentLoadResult(const String & name) const; + ReturnType getLoadResult(const String & name) const; using FilterByNameFunction = std::function; /// Returns all the load results as a map. /// The function doesn't load anything, it just returns the current load results as is. template , void>> - ReturnType getCurrentLoadResults() const { return getCurrentLoadResults(alwaysTrue); } + ReturnType getLoadResults() const { return getLoadResults(FilterByNameFunction{}); } template , void>> - ReturnType getCurrentLoadResults(const FilterByNameFunction & filter) const; + ReturnType getLoadResults(const FilterByNameFunction & filter) const; /// Returns all loaded objects as a map. /// The function doesn't load anything, it just returns the current load results as is. - Loadables getCurrentlyLoadedObjects() const; - Loadables getCurrentlyLoadedObjects(const FilterByNameFunction & filter) const; + Loadables getLoadedObjects() const; + Loadables getLoadedObjects(const FilterByNameFunction & filter) const; /// Returns true if any object was loaded. - bool hasCurrentlyLoadedObjects() const; - size_t getNumberOfCurrentlyLoadedObjects() const; + bool hasLoadedObjects() const; + size_t getNumberOfLoadedObjects() const; + + /// Returns true if there is no object. + bool hasObjects() const { return getNumberOfObjects() == 0; } + + /// Returns number of objects. + size_t getNumberOfObjects() const; static constexpr Duration NO_WAIT = Duration::zero(); static constexpr Duration WAIT = Duration::max(); @@ -139,7 +153,7 @@ public: /// The function does nothing for already loaded objects, it just returns them. /// The function doesn't throw an exception if it's failed to load something. template , void>> - ReturnType tryLoadAll(Duration timeout = WAIT) const { return tryLoad(alwaysTrue, timeout); } + ReturnType tryLoadAll(Duration timeout = WAIT) const { return tryLoad(FilterByNameFunction{}, timeout); } /// Loads a specified object. /// The function does nothing if it's already loaded. @@ -157,7 +171,7 @@ public: /// The function does nothing for already loaded objects, it just returns them. /// The function throws an exception if it's failed to load something. template , void>> - ReturnType loadAll() const { return load(alwaysTrue); } + ReturnType loadAll() const { return load(FilterByNameFunction{}); } /// Loads or reloads a specified object. /// The function reloads the object if it's already loaded. @@ -174,7 +188,7 @@ public: /// Load or reloads all objects. Not recommended to use. /// The function throws an exception if it's failed to load or reload something. template , void>> - ReturnType loadOrReloadAll() const { return loadOrReload(alwaysTrue); } + ReturnType loadOrReloadAll() const { return loadOrReload(FilterByNameFunction{}); } /// Reloads objects by filter which were tried to load before (successfully or not). /// The function throws an exception if it's failed to load or reload something. @@ -197,10 +211,8 @@ private: void checkLoaded(const LoadResult & result, bool check_no_errors) const; void checkLoaded(const LoadResults & results, bool check_no_errors) const; - static bool alwaysTrue(const String &) { return true; } Strings getAllTriedToLoadNames() const; - struct ObjectConfig; LoadablePtr createObject(const String & name, const ObjectConfig & config, const LoadablePtr & previous_version) const; class LoadablesConfigReader; diff --git a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp index 10f99262da7..7b3d57b192a 100644 --- a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp @@ -1,30 +1,30 @@ #include -#include -#include #include + namespace DB { - namespace ErrorCodes { extern const int UNKNOWN_DICTIONARY; } + namespace { -String trimDatabaseName(const std::string & loadable_definition_name, const IDatabase & database) -{ - const auto & dbname = database.getDatabaseName(); - if (!startsWith(loadable_definition_name, dbname)) - throw Exception( - "Loadable '" + loadable_definition_name + "' is not from database '" + database.getDatabaseName(), ErrorCodes::UNKNOWN_DICTIONARY); - /// dbname.loadable_name - ///--> remove <--- - return loadable_definition_name.substr(dbname.length() + 1); -} + String trimDatabaseName(const std::string & loadable_definition_name, const IDatabase & database) + { + const auto & dbname = database.getDatabaseName(); + if (!startsWith(loadable_definition_name, dbname)) + throw Exception( + "Loadable '" + loadable_definition_name + "' is not from database '" + database.getDatabaseName(), ErrorCodes::UNKNOWN_DICTIONARY); + /// dbname.loadable_name + ///--> remove <--- + return loadable_definition_name.substr(dbname.length() + 1); + } } + ExternalLoaderDatabaseConfigRepository::ExternalLoaderDatabaseConfigRepository(IDatabase & database_, const Context & context_) : name(database_.getDatabaseName()) , database(database_) @@ -34,8 +34,7 @@ ExternalLoaderDatabaseConfigRepository::ExternalLoaderDatabaseConfigRepository(I LoadablesConfigurationPtr ExternalLoaderDatabaseConfigRepository::load(const std::string & loadable_definition_name) { - String dictname = trimDatabaseName(loadable_definition_name, database); - return getDictionaryConfigurationFromAST(database.getCreateDictionaryQuery(context, dictname)->as()); + return database.getDictionaryConfiguration(trimDatabaseName(loadable_definition_name, database)); } bool ExternalLoaderDatabaseConfigRepository::exists(const std::string & loadable_definition_name) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 37e2c8c5945..be7bd238025 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -45,6 +45,8 @@ #include #include +#include + #include #include @@ -703,7 +705,11 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create) } if (create.attach) - database->attachDictionary(dictionary_name, context); + { + auto config = getDictionaryConfigurationFromAST(create); + auto modification_time = database->getObjectMetadataModificationTime(dictionary_name); + database->attachDictionary(dictionary_name, DictionaryAttachInfo{query_ptr, config, modification_time}); + } else database->createDictionary(context, dictionary_name, query_ptr); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 91783352842..71730c23788 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -188,7 +188,7 @@ BlockIO InterpreterDropQuery::executeToDictionary( { /// Drop dictionary from memory, don't touch data and metadata context.checkAccess(AccessType::DROP_DICTIONARY, database_name, dictionary_name); - database->detachDictionary(dictionary_name, context); + database->detachDictionary(dictionary_name); } else if (kind == ASTDropQuery::Kind::Truncate) { @@ -254,21 +254,26 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS bool drop = kind == ASTDropQuery::Kind::Drop; context.checkAccess(AccessType::DROP_DATABASE, database_name); - /// DETACH or DROP all tables and dictionaries inside database - for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) + if (database->shouldBeEmptyOnDetach()) { - String current_table_name = iterator->name(); - executeToTable(database_name, current_table_name, kind, false, false, false); - } + /// DETACH or DROP all tables and dictionaries inside database. + /// First we should DETACH or DROP dictionaries because StorageDictionary + /// must be detached only by detaching corresponding dictionary. + for (auto iterator = database->getDictionariesIterator(context); iterator->isValid(); iterator->next()) + { + String current_dictionary = iterator->name(); + executeToDictionary(database_name, current_dictionary, kind, false, false, false); + } - for (auto iterator = database->getDictionariesIterator(context); iterator->isValid(); iterator->next()) - { - String current_dictionary = iterator->name(); - executeToDictionary(database_name, current_dictionary, kind, false, false, false); + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + String current_table_name = iterator->name(); + executeToTable(database_name, current_table_name, kind, false, false, false); + } } /// DETACH or DROP database itself - DatabaseCatalog::instance().detachDatabase(database_name, drop); + DatabaseCatalog::instance().detachDatabase(database_name, drop, database->shouldBeEmptyOnDetach()); } } diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 86831593d54..9b2c5784d85 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -1,73 +1,48 @@ -#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 NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int THERE_IS_NO_COLUMN; - extern const int UNKNOWN_TABLE; + extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE; } - -StorageDictionary::StorageDictionary( - const StorageID & table_id_, - const ColumnsDescription & columns_, - const Context & context, - bool attach, - const String & dictionary_name_) - : IStorage(table_id_) - , dictionary_name(dictionary_name_) - , logger(&Poco::Logger::get("StorageDictionary")) +namespace { - setColumns(columns_); - - if (!attach) + void checkNamesAndTypesCompatibleWithDictionary(const String & dictionary_name, const ColumnsDescription & columns, const DictionaryStructure & dictionary_structure) { - const auto & dictionary = context.getExternalDictionariesLoader().getDictionary(dictionary_name); - const DictionaryStructure & dictionary_structure = dictionary->getStructure(); - checkNamesAndTypesCompatibleWithDictionary(dictionary_structure); + auto dictionary_names_and_types = StorageDictionary::getNamesAndTypes(dictionary_structure); + std::set names_and_types_set(dictionary_names_and_types.begin(), dictionary_names_and_types.end()); + + for (const auto & column : columns.getOrdinary()) + { + if (names_and_types_set.find(column) == names_and_types_set.end()) + { + std::string message = "Not found column "; + message += column.name + " " + column.type->getName(); + message += " in dictionary " + backQuote(dictionary_name) + ". "; + message += "There are only columns "; + message += StorageDictionary::generateNamesAndTypesDescription(dictionary_names_and_types); + throw Exception(message, ErrorCodes::THERE_IS_NO_COLUMN); + } + } } } -void StorageDictionary::checkTableCanBeDropped() const -{ - throw Exception("Cannot detach dictionary " + backQuoteIfNeed(dictionary_name) + " as table, use DETACH DICTIONARY query.", ErrorCodes::UNKNOWN_TABLE); -} - -Pipes StorageDictionary::read( - const Names & column_names, - const SelectQueryInfo & /*query_info*/, - const Context & context, - QueryProcessingStage::Enum /*processed_stage*/, - const size_t max_block_size, - const unsigned /*threads*/) -{ - auto dictionary = context.getExternalDictionariesLoader().getDictionary(dictionary_name); - auto stream = dictionary->getBlockInputStream(column_names, max_block_size); - auto source = std::make_shared(stream); - /// TODO: update dictionary interface for processors. - Pipes pipes; - pipes.emplace_back(std::move(source)); - return pipes; -} NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure & dictionary_structure) { @@ -103,25 +78,55 @@ NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure return dictionary_names_and_types; } -void StorageDictionary::checkNamesAndTypesCompatibleWithDictionary(const DictionaryStructure & dictionary_structure) const -{ - auto dictionary_names_and_types = getNamesAndTypes(dictionary_structure); - std::set names_and_types_set(dictionary_names_and_types.begin(), dictionary_names_and_types.end()); - for (const auto & column : getColumns().getOrdinary()) +String StorageDictionary::generateNamesAndTypesDescription(const NamesAndTypesList & list) +{ + std::stringstream ss; + bool first = true; + for (const auto & name_and_type : list) { - if (names_and_types_set.find(column) == names_and_types_set.end()) - { - std::string message = "Not found column "; - message += column.name + " " + column.type->getName(); - message += " in dictionary " + dictionary_name + ". "; - message += "There are only columns "; - message += generateNamesAndTypesDescription(dictionary_names_and_types.begin(), dictionary_names_and_types.end()); - throw Exception(message, ErrorCodes::THERE_IS_NO_COLUMN); - } + if (!std::exchange(first, false)) + ss << ", "; + ss << name_and_type.name << ' ' << name_and_type.type->getName(); } + return ss.str(); } + +StorageDictionary::StorageDictionary( + const StorageID & table_id_, + const String & dictionary_name_, + const DictionaryStructure & dictionary_structure_) + : IStorage(table_id_) + , dictionary_name(dictionary_name_) +{ + setColumns(ColumnsDescription{getNamesAndTypes(dictionary_structure_)}); +} + + +void StorageDictionary::checkTableCanBeDropped() const +{ + throw Exception("Cannot detach dictionary " + backQuote(dictionary_name) + " as table, use DETACH DICTIONARY query.", ErrorCodes::CANNOT_DETACH_DICTIONARY_AS_TABLE); +} + +Pipes StorageDictionary::read( + const Names & column_names, + const SelectQueryInfo & /*query_info*/, + const Context & context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t max_block_size, + const unsigned /*threads*/) +{ + auto dictionary = context.getExternalDictionariesLoader().getDictionary(dictionary_name); + auto stream = dictionary->getBlockInputStream(column_names, max_block_size); + auto source = std::make_shared(stream); + /// TODO: update dictionary interface for processors. + Pipes pipes; + pipes.emplace_back(std::move(source)); + return pipes; +} + + void registerStorageDictionary(StorageFactory & factory) { factory.registerStorage("Dictionary", [](const StorageFactory::Arguments & args) @@ -133,8 +138,11 @@ void registerStorageDictionary(StorageFactory & factory) args.engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args.engine_args[0], args.local_context); String dictionary_name = args.engine_args[0]->as().value.safeGet(); - return StorageDictionary::create( - args.table_id, args.columns, args.context, args.attach, dictionary_name); + const auto & dictionary = args.context.getExternalDictionariesLoader().getDictionary(dictionary_name); + const DictionaryStructure & dictionary_structure = dictionary->getStructure(); + checkNamesAndTypesCompatibleWithDictionary(dictionary_name, args.columns, dictionary_structure); + + return StorageDictionary::create(args.table_id, dictionary_name, dictionary_structure); }); } diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 87826304166..225c0b3120d 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -1,23 +1,12 @@ #pragma once #include -#include -#include #include -#include -#include -namespace Poco -{ -class Logger; -} - namespace DB { struct DictionaryStructure; -struct IDictionaryBase; -class ExternalDictionaries; class StorageDictionary final : public ext::shared_ptr_helper, public IStorage { @@ -35,42 +24,16 @@ public: unsigned threads) override; static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); - - template - static std::string generateNamesAndTypesDescription(ForwardIterator begin, ForwardIterator end) - { - std::string description; - { - WriteBufferFromString buffer(description); - bool first = true; - for (; begin != end; ++begin) - { - if (!first) - buffer << ", "; - first = false; - - buffer << begin->name << ' ' << begin->type->getName(); - } - } - - return description; - } + static String generateNamesAndTypesDescription(const NamesAndTypesList & list); private: - using Ptr = MultiVersion::Version; - String dictionary_name; - Poco::Logger * logger; - - void checkNamesAndTypesCompatibleWithDictionary(const DictionaryStructure & dictionary_structure) const; protected: StorageDictionary( const StorageID & table_id_, - const ColumnsDescription & columns_, - const Context & context, - bool attach, - const String & dictionary_name_); + const String & dictionary_name_, + const DictionaryStructure & dictionary_structure); }; } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 26e2376c3f7..43594f5355a 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -303,7 +303,7 @@ Pipes StorageSystemColumns::read( const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getTablesWithDictionaryTablesIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { const String & table_name = iterator->name(); storages.emplace(std::piecewise_construct, diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index a3b32324700..4c54353c44d 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -53,7 +53,7 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con const bool check_access_for_dictionaries = !access->isGranted(AccessType::SHOW_DICTIONARIES); const auto & external_dictionaries = context.getExternalDictionariesLoader(); - for (const auto & load_result : external_dictionaries.getCurrentLoadResults()) + for (const auto & load_result : external_dictionaries.getLoadResults()) { const auto dict_ptr = std::dynamic_pointer_cast(load_result.object); @@ -66,9 +66,10 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con else { short_name = load_result.name; - if (!load_result.repository_name.empty() && startsWith(short_name, load_result.repository_name + ".")) + String repository_name = load_result.config ? load_result.config->repository_name : ""; + if (!repository_name.empty() && startsWith(short_name, repository_name + ".")) { - database = load_result.repository_name; + database = repository_name; short_name = short_name.substr(database.length() + 1); } } @@ -81,7 +82,7 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con res_columns[i++]->insert(database); res_columns[i++]->insert(short_name); res_columns[i++]->insert(static_cast(load_result.status)); - res_columns[i++]->insert(load_result.origin); + res_columns[i++]->insert(load_result.config ? load_result.config->path : ""); std::exception_ptr last_exception = load_result.exception; diff --git a/src/Storages/System/StorageSystemModels.cpp b/src/Storages/System/StorageSystemModels.cpp index 306829cf6de..9fae9803b96 100644 --- a/src/Storages/System/StorageSystemModels.cpp +++ b/src/Storages/System/StorageSystemModels.cpp @@ -28,13 +28,13 @@ NamesAndTypesList StorageSystemModels::getNamesAndTypes() void StorageSystemModels::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { const auto & external_models_loader = context.getExternalModelsLoader(); - auto load_results = external_models_loader.getCurrentLoadResults(); + auto load_results = external_models_loader.getLoadResults(); for (const auto & load_result : load_results) { res_columns[0]->insert(load_result.name); res_columns[1]->insert(static_cast(load_result.status)); - res_columns[2]->insert(load_result.origin); + res_columns[2]->insert(load_result.config ? load_result.config->path : ""); if (load_result.object) { diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 81ff6a03e12..d50a5fe4185 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -226,7 +226,7 @@ protected: const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); if (!tables_it || !tables_it->isValid()) - tables_it = database->getTablesWithDictionaryTablesIterator(context); + tables_it = database->getTablesIterator(context); const bool need_lock_structure = needLockStructure(database, getPort().getHeader()); diff --git a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.reference b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.reference index 87dc6a5b6bf..4a22b3a52cf 100644 --- a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.reference +++ b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.reference @@ -9,6 +9,7 @@ dict1 dict2 dict3 +dict4 table_for_dict dict1 dict2 diff --git a/tests/queries/0_stateless/01048_exists_query.sql b/tests/queries/0_stateless/01048_exists_query.sql index 9a4c0558b60..700b4f5983d 100644 --- a/tests/queries/0_stateless/01048_exists_query.sql +++ b/tests/queries/0_stateless/01048_exists_query.sql @@ -32,7 +32,7 @@ EXISTS TABLE db_01048.t_01048; -- Dictionaries are tables as well. But not all t EXISTS DICTIONARY db_01048.t_01048; -- But dictionary-tables cannot be dropped as usual tables. -DROP TABLE db_01048.t_01048; -- { serverError 60 } +DROP TABLE db_01048.t_01048; -- { serverError 520 } DROP DICTIONARY db_01048.t_01048; EXISTS db_01048.t_01048; EXISTS TABLE db_01048.t_01048; diff --git a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.reference b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.reference index 5321624de02..524fbdd26fc 100644 --- a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.reference +++ b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.reference @@ -16,4 +16,9 @@ CREATE TABLE dict_db_01224_dictionary.`dict_db_01224.dict` `val` UInt64 ) ENGINE = Dictionary(`dict_db_01224.dict`) -LOADED +NOT_LOADED +Dictionary 1 CREATE DICTIONARY dict_db_01224.dict (`key` UInt64 DEFAULT 0, `val` UInt64 DEFAULT 10) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'dict_data\' PASSWORD \'\' DB \'dict_db_01224\')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()) +NOT_LOADED +key UInt64 +val UInt64 +NOT_LOADED diff --git a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql index a6eed6f072c..cf8b2a471c4 100644 --- a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql +++ b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql @@ -1,6 +1,7 @@ DROP DATABASE IF EXISTS dict_db_01224; DROP DATABASE IF EXISTS dict_db_01224_dictionary; CREATE DATABASE dict_db_01224; +CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary; CREATE TABLE dict_db_01224.dict_data (key UInt64, val UInt64) Engine=Memory(); CREATE DICTIONARY dict_db_01224.dict @@ -21,10 +22,15 @@ SELECT status FROM system.dictionaries WHERE database = 'dict_db_01224' AND name SHOW CREATE TABLE dict_db_01224.dict FORMAT TSVRaw; SELECT status FROM system.dictionaries WHERE database = 'dict_db_01224' AND name = 'dict'; -CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary; SHOW CREATE TABLE dict_db_01224_dictionary.`dict_db_01224.dict` FORMAT TSVRaw; SELECT status FROM system.dictionaries WHERE database = 'dict_db_01224' AND name = 'dict'; +SELECT engine, metadata_path LIKE '%/metadata/dict\_db\_01224/dict.sql', create_table_query FROM system.tables WHERE database = 'dict_db_01224' AND name = 'dict'; +SELECT status FROM system.dictionaries WHERE database = 'dict_db_01224' AND name = 'dict'; + +SELECT name, type FROM system.columns WHERE database = 'dict_db_01224' AND table = 'dict'; +SELECT status FROM system.dictionaries WHERE database = 'dict_db_01224' AND name = 'dict'; + DROP DICTIONARY dict_db_01224.dict; SELECT status FROM system.dictionaries WHERE database = 'dict_db_01224' AND name = 'dict'; diff --git a/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql b/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql index 045775aec2b..866f2dff56b 100644 --- a/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql +++ b/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql @@ -13,8 +13,8 @@ LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); SYSTEM RELOAD DICTIONARY dict_db_01225.dict; -DROP TABLE dict_db_01225.dict; -- { serverError 60; } --- Regression: --- Code: 1000. DB::Exception: Received from localhost:9000. DB::Exception: File not found: ./metadata/dict_db_01225/dict.sql. + +DROP TABLE dict_db_01225.dict; -- { serverError 520; } DROP DICTIONARY dict_db_01225.dict; + DROP DATABASE dict_db_01225; diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index 7550d5292d0..a494511ebd8 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -15,7 +15,7 @@ LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.dict` FORMAT TSVRaw; -SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 36; } +SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 487; } DROP DATABASE dict_db_01225; DROP DATABASE dict_db_01225_dictionary; From 96b5e7d3df02e9db42f9f0603bf0240a8ec6d734 Mon Sep 17 00:00:00 2001 From: Vitaly Date: Sat, 18 Apr 2020 13:05:43 +0300 Subject: [PATCH 369/743] Update default_libs.cmake --- cmake/freebsd/default_libs.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/freebsd/default_libs.cmake b/cmake/freebsd/default_libs.cmake index 2bb76c6a761..d0dea46102d 100644 --- a/cmake/freebsd/default_libs.cmake +++ b/cmake/freebsd/default_libs.cmake @@ -4,7 +4,7 @@ if (NOT COMPILER_CLANG) message (FATAL_ERROR "FreeBSD build is supported only for Clang") endif () -execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-x86_64.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread") From ef5f83eef8bdffde3a929daab706bbda4662694d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 18 Apr 2020 14:26:57 +0300 Subject: [PATCH 370/743] Update MsgPackRowInputFormat.cpp --- .../Formats/Impl/MsgPackRowInputFormat.cpp | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 0748af68523..3e112fb1ce6 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : IRowInputFormat(header_, in_, std::move(params_)), buf(in), parser(visitor), data_types(header_.getDataTypes()) {} -void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type) +void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type) // NOLINT { while (!info_stack.empty()) { @@ -35,7 +35,7 @@ void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type) info_stack.push(Info{column, type}); } -void MsgPackVisitor::insert_integer(UInt64 value) +void MsgPackVisitor::insert_integer(UInt64 value) // NOLINT { Info & info = info_stack.top(); switch (info.type->getTypeId()) @@ -92,37 +92,37 @@ void MsgPackVisitor::insert_integer(UInt64 value) } } -bool MsgPackVisitor::visit_positive_integer(UInt64 value) +bool MsgPackVisitor::visit_positive_integer(UInt64 value) // NOLINT { insert_integer(value); return true; } -bool MsgPackVisitor::visit_negative_integer(Int64 value) +bool MsgPackVisitor::visit_negative_integer(Int64 value) // NOLINT { insert_integer(value); return true; } -bool MsgPackVisitor::visit_str(const char* value, size_t size) +bool MsgPackVisitor::visit_str(const char* value, size_t size) // NOLINT { info_stack.top().column.insertData(value, size); return true; } -bool MsgPackVisitor::visit_float32(Float32 value) +bool MsgPackVisitor::visit_float32(Float32 value) // NOLINT { assert_cast(info_stack.top().column).insertValue(value); return true; } -bool MsgPackVisitor::visit_float64(Float64 value) +bool MsgPackVisitor::visit_float64(Float64 value) // NOLINT { assert_cast(info_stack.top().column).insertValue(value); return true; } -bool MsgPackVisitor::start_array(size_t size) +bool MsgPackVisitor::start_array(size_t size) // NOLINT { auto nested_type = assert_cast(*info_stack.top().type).getNestedType(); ColumnArray & column_array = assert_cast(info_stack.top().column); @@ -133,13 +133,13 @@ bool MsgPackVisitor::start_array(size_t size) return true; } -bool MsgPackVisitor::end_array() +bool MsgPackVisitor::end_array() // NOLINT { info_stack.pop(); return true; } -void MsgPackVisitor::parse_error(size_t, size_t) +void MsgPackVisitor::parse_error(size_t, size_t) // NOLINT { throw Exception("Error occurred while parsing msgpack data.", ErrorCodes::INCORRECT_DATA); } From bdc8dfe04be16c9ac36c578e72da6213b048b8c8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 18 Apr 2020 14:48:51 +0300 Subject: [PATCH 371/743] Added a change from @azat in #10277 --- tests/integration/test_settings_constraints_distributed/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index 58a780ef16d..86456f8a099 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -103,4 +103,5 @@ def test_insert_clamps_settings(): distributed.query("INSERT INTO proxy VALUES (toDate('2020-02-20'), 2, 2)") distributed.query("INSERT INTO proxy VALUES (toDate('2020-02-21'), 2, 2)", settings={"max_memory_usage": 5000000}) + distributed.query("SYSTEM FLUSH DISTRIBUTED proxy") assert_eq_with_retry(distributed, "SELECT COUNT() FROM proxy", "4") From e0a121e8d58a3e8f2fbd122e2e53419a08d9bb16 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 18 Apr 2020 14:56:19 +0300 Subject: [PATCH 372/743] Update simpleaggregatefunction.md --- .../nested_data_structures/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md b/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md index e172986745d..4e086541053 100644 --- a/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md +++ b/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md @@ -15,7 +15,7 @@ Currently, the following aggregate functions are supported: - [`groupBitOr`](../../query_language/agg_functions/reference.md#groupbitor) - [`groupBitXor`](../../query_language/agg_functions/reference.md#groupbitxor) -- Type of the `SimpleAggregateFunction(func, Type)` is `Type` itself, so you do not need to apply functions with `-Merge`/`-State` suffixes. +- Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. - `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function. **Parameters** From 71a7a74d259228e9794953eba6f31e0a4d570c1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Apr 2020 15:31:59 +0300 Subject: [PATCH 373/743] Fix test --- tests/queries/0_stateless/01246_least_greatest_generic.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01246_least_greatest_generic.sql b/tests/queries/0_stateless/01246_least_greatest_generic.sql index 74d095a66e7..f0dceabfcb5 100644 --- a/tests/queries/0_stateless/01246_least_greatest_generic.sql +++ b/tests/queries/0_stateless/01246_least_greatest_generic.sql @@ -15,8 +15,10 @@ SELECT greatest(1, inf); SELECT least(0., -0.); SELECT least(toNullable(123), 456); -SELECT LEAST(-1, 18446744073709551615); -- { serverError 386 } -SELECT LEAST(-1., 18446744073709551615); -- { serverError 386 } +-- This can be improved +SELECT LEAST(-1, 18446744073709551615); -- { serverError 43 } +SELECT LEAST(-1., 18446744073709551615); -- { serverError 43 } + SELECT LEAST(-1., 18446744073709551615.); SELECT greatest(-1, 1, 4294967295); From 4f245dc2867a5557d8212eb9ff8cefff839c549f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Apr 2020 15:35:00 +0300 Subject: [PATCH 374/743] Added performance test --- tests/performance/least_greatest_hits.xml | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 tests/performance/least_greatest_hits.xml diff --git a/tests/performance/least_greatest_hits.xml b/tests/performance/least_greatest_hits.xml new file mode 100644 index 00000000000..464656b0201 --- /dev/null +++ b/tests/performance/least_greatest_hits.xml @@ -0,0 +1,9 @@ + + + test.hits + + + SELECT count() FROM test.hits WHERE NOT ignore(least(URL, Referer)) + SELECT count() FROM test.hits WHERE NOT ignore(greatest(URL, Referer, Title)) + SELECT count() FROM test.hits WHERE NOT ignore(greatest(ClientIP, RemoteIP)) + From 7fbd7e953316bedaaebfd92aa60ff2eeb84dc73e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 18 Apr 2020 15:41:15 +0300 Subject: [PATCH 375/743] Update materialized_view_parallel_insert.xml --- tests/performance/materialized_view_parallel_insert.xml | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/performance/materialized_view_parallel_insert.xml b/tests/performance/materialized_view_parallel_insert.xml index a430c4e480c..4b71354dec3 100644 --- a/tests/performance/materialized_view_parallel_insert.xml +++ b/tests/performance/materialized_view_parallel_insert.xml @@ -1,10 +1,4 @@ - - - 2 - - - hits_10m_single From 96c2882db98e5a9b5ef7593883331013104dca41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Apr 2020 15:37:20 +0300 Subject: [PATCH 376/743] One more test #4767 --- .../0_stateless/01247_least_greatest_filimonov.reference | 3 +++ tests/queries/0_stateless/01247_least_greatest_filimonov.sql | 3 +++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/01247_least_greatest_filimonov.reference create mode 100644 tests/queries/0_stateless/01247_least_greatest_filimonov.sql diff --git a/tests/queries/0_stateless/01247_least_greatest_filimonov.reference b/tests/queries/0_stateless/01247_least_greatest_filimonov.reference new file mode 100644 index 00000000000..5b3b2abada8 --- /dev/null +++ b/tests/queries/0_stateless/01247_least_greatest_filimonov.reference @@ -0,0 +1,3 @@ +2 +767 +C diff --git a/tests/queries/0_stateless/01247_least_greatest_filimonov.sql b/tests/queries/0_stateless/01247_least_greatest_filimonov.sql new file mode 100644 index 00000000000..b845d65dcb9 --- /dev/null +++ b/tests/queries/0_stateless/01247_least_greatest_filimonov.sql @@ -0,0 +1,3 @@ +SELECT GREATEST(2,0); +SELECT GREATEST(34.0,3.0,5.0,767.0); +SELECT GREATEST('B','A','C'); From 65714a3014bc2e152f8fd6f03b8f557fc2c751a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Apr 2020 15:39:08 +0300 Subject: [PATCH 377/743] One more test #4767 --- .../01248_least_greatest_mixed_const.reference | 10 ++++++++++ .../0_stateless/01248_least_greatest_mixed_const.sql | 1 + 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/01248_least_greatest_mixed_const.reference create mode 100644 tests/queries/0_stateless/01248_least_greatest_mixed_const.sql diff --git a/tests/queries/0_stateless/01248_least_greatest_mixed_const.reference b/tests/queries/0_stateless/01248_least_greatest_mixed_const.reference new file mode 100644 index 00000000000..bbdc93bd5ee --- /dev/null +++ b/tests/queries/0_stateless/01248_least_greatest_mixed_const.reference @@ -0,0 +1,10 @@ +0 6 +1 6 +2 6 +3 6 +4 6 +4 6 +4 6 +4 7 +4 8 +4 9 diff --git a/tests/queries/0_stateless/01248_least_greatest_mixed_const.sql b/tests/queries/0_stateless/01248_least_greatest_mixed_const.sql new file mode 100644 index 00000000000..3fcf20623d6 --- /dev/null +++ b/tests/queries/0_stateless/01248_least_greatest_mixed_const.sql @@ -0,0 +1 @@ +SELECT least(4, number, 6), greatest(4, number, 6) FROM numbers(10); From fd085b58e0e1f07adc263850b43045ae61064fc2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Apr 2020 15:44:06 +0300 Subject: [PATCH 378/743] Fix readme in performance tests #10345 --- tests/performance/README.md | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/tests/performance/README.md b/tests/performance/README.md index a797b88a321..4774d21b42a 100644 --- a/tests/performance/README.md +++ b/tests/performance/README.md @@ -1,6 +1,6 @@ ## ClickHouse performance tests -This directory contains `.xml`-files with performance tests for `clickhouse-performance-test` tool. +This directory contains `.xml`-files with performance tests for @akuzm tool. ### How to write performance test @@ -8,8 +8,6 @@ First of all you should check existing tests don't cover your case. If there are You have to specify `preconditions`. It contains table names. Only `hits_100m_single`, `hits_10m_single`, `test.hits` are available in CI. -The most important part of test is `stop_conditions`. Also you should always specify `total_time_ms` metric. Endless tests will be ignored by CI. - You can use `substitions`, `create`, `fill` and `drop` queries to prepare test. You can find examples in this folder. Take into account, that these tests will run in CI which consists of 56-cores and 512 RAM machines. Queries will be executed much faster than on local laptop. @@ -18,9 +16,4 @@ If your test continued more than 10 minutes, please, add tag `long` to have an o ### How to run performance test -You have to run clickhouse-server and after you can start testing: - -``` -$ clickhouse-performance-test --input-file my_lovely_test1.xml --input-file my_lovely_test2.xml -$ clickhouse-performance-test --input-file /my_lovely_test_dir/ -``` +TODO @akuzm \ No newline at end of file From 785d2c0e01985ad95550affad43257fc14115c3e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Apr 2020 15:53:49 +0300 Subject: [PATCH 379/743] Remove garbage from performance tests --- tests/performance/IPv4.xml | 9 --------- tests/performance/IPv6.xml | 9 --------- tests/performance/agg_functions_min_max_any.xml | 10 ---------- tests/performance/and_function.xml | 10 ---------- tests/performance/arithmetic.xml | 10 ---------- tests/performance/array_auc.xml | 5 ----- tests/performance/array_element.xml | 5 ----- tests/performance/array_join.xml | 5 ----- tests/performance/array_reduce.xml | 5 ----- tests/performance/base64.xml | 8 -------- tests/performance/base64_hits.xml | 8 -------- tests/performance/basename.xml | 10 ---------- tests/performance/bitCount.xml | 6 ------ tests/performance/bit_operations_fixed_string.xml | 10 ---------- .../bit_operations_fixed_string_numbers.xml | 10 ---------- tests/performance/bloom_filter.xml | 5 ----- tests/performance/bounding_ratio.xml | 6 ------ tests/performance/cidr.xml | 10 ---------- tests/performance/codecs_float_insert.xml | 10 ---------- tests/performance/codecs_float_select.xml | 10 ---------- tests/performance/codecs_int_insert.xml | 10 ---------- tests/performance/codecs_int_select.xml | 10 ---------- tests/performance/collations.xml | 10 ---------- tests/performance/column_column_comparison.xml | 10 ---------- tests/performance/columns_hashing.xml | 10 ---------- tests/performance/complex_array_creation.xml | 5 ----- tests/performance/conditional.xml | 5 ----- tests/performance/consistent_hashes.xml | 5 ----- tests/performance/constant_column_comparison.xml | 10 ---------- tests/performance/constant_column_search.xml | 10 ---------- tests/performance/count.xml | 5 ----- tests/performance/cpu_synthetic.xml | 10 ---------- tests/performance/cryptographic_hashes.xml | 8 -------- tests/performance/date_parsing.xml | 10 ---------- tests/performance/date_time.xml | 5 ----- tests/performance/date_time_64.xml | 10 ---------- tests/performance/decimal_aggregates.xml | 5 ----- tests/performance/early_constant_folding.xml | 9 --------- tests/performance/empty_string_deserialization.xml | 5 ----- tests/performance/empty_string_serialization.xml | 5 ----- tests/performance/entropy.xml | 8 -------- tests/performance/first_significant_subdomain.xml | 10 ---------- tests/performance/fixed_string16.xml | 10 ---------- tests/performance/float_formatting.xml | 8 -------- tests/performance/float_parsing.xml | 8 -------- tests/performance/format_date_time.xml | 5 ----- tests/performance/functions_coding.xml | 5 ----- tests/performance/functions_geo.xml | 5 ----- tests/performance/general_purpose_hashes.xml | 10 ---------- tests/performance/general_purpose_hashes_on_UUID.xml | 10 ---------- tests/performance/generate_table_function.xml | 5 ----- tests/performance/great_circle_dist.xml | 5 ----- tests/performance/group_array_moving_sum.xml | 8 -------- tests/performance/h3.xml | 6 ------ tests/performance/if_array_num.xml | 5 ----- tests/performance/if_array_string.xml | 5 ----- tests/performance/if_string_const.xml | 5 ----- tests/performance/if_string_hits.xml | 10 ---------- tests/performance/if_to_multiif.xml | 5 ----- tests/performance/information_value.xml | 8 -------- tests/performance/insert_values_with_expressions.xml | 10 ---------- tests/performance/inserts_arrays_lowcardinality.xml | 10 ---------- tests/performance/int_parsing.xml | 10 ---------- tests/performance/jit_large_requests.xml | 5 ----- tests/performance/jit_small_requests.xml | 10 ---------- tests/performance/joins_in_memory.xml | 5 ----- tests/performance/joins_in_memory_pmj.xml | 5 ----- tests/performance/json_extract_rapidjson.xml | 10 ---------- tests/performance/json_extract_simdjson.xml | 10 ---------- tests/performance/leftpad.xml | 10 ---------- tests/performance/linear_regression.xml | 5 ----- tests/performance/logical_functions_large.xml | 10 ---------- tests/performance/logical_functions_medium.xml | 10 ---------- tests/performance/logical_functions_small.xml | 10 ---------- tests/performance/math.xml | 10 ---------- tests/performance/merge_table_streams.xml | 9 --------- tests/performance/merge_tree_huge_pk.xml | 10 ---------- tests/performance/merge_tree_many_partitions.xml | 10 ---------- tests/performance/merge_tree_many_partitions_2.xml | 10 ---------- tests/performance/merge_tree_simple_select.xml | 5 ----- tests/performance/mingroupby-orderbylimit1.xml | 5 ----- tests/performance/modulo.xml | 5 ----- tests/performance/ngram_distance.xml | 10 ---------- tests/performance/number_formatting_formats.xml | 10 ---------- tests/performance/nyc_taxi.xml | 10 ---------- tests/performance/order_by_decimals.xml | 10 ---------- tests/performance/order_by_read_in_order.xml | 10 ---------- tests/performance/order_by_single_column.xml | 10 ---------- tests/performance/parallel_insert.xml | 5 ----- tests/performance/parse_engine_file.xml | 10 ---------- tests/performance/pre_limit_no_sorting.xml | 10 ---------- tests/performance/prewhere.xml | 10 ---------- tests/performance/random_printable_ascii.xml | 5 ----- tests/performance/range.xml | 5 ----- tests/performance/read_hits_with_aio.xml | 5 ----- tests/performance/right.xml | 8 -------- tests/performance/round_down.xml | 8 -------- tests/performance/round_methods.xml | 8 -------- tests/performance/scalar.xml | 9 --------- tests/performance/select_format.xml | 10 ---------- tests/performance/set.xml | 8 -------- tests/performance/set_hits.xml | 9 --------- tests/performance/set_index.xml | 9 --------- tests/performance/simple_join_query.xml | 8 -------- tests/performance/slices_hits.xml | 10 ---------- tests/performance/sort.xml | 5 ----- tests/performance/string_join.xml | 5 ----- tests/performance/string_set.xml | 5 ----- tests/performance/string_sort.xml | 10 ---------- tests/performance/sum_map.xml | 5 ----- tests/performance/synthetic_hardware_benchmark.xml | 5 ----- tests/performance/trim_numbers.xml | 8 -------- tests/performance/trim_urls.xml | 8 -------- tests/performance/trim_whitespace.xml | 5 ----- tests/performance/uniq.xml | 6 ------ tests/performance/url_hits.xml | 8 -------- .../performance/vectorize_aggregation_combinators.xml | 8 -------- tests/performance/visit_param_extract_raw.xml | 5 ----- tests/performance/website.xml | 9 --------- 119 files changed, 932 deletions(-) diff --git a/tests/performance/IPv4.xml b/tests/performance/IPv4.xml index 8f5b61d70c9..b3f6cf52584 100644 --- a/tests/performance/IPv4.xml +++ b/tests/performance/IPv4.xml @@ -1,14 +1,5 @@ - - - 30000 - - - 5000 - 60000 - - CREATE TABLE IF NOT EXISTS ips_v4(ip String) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY tuple() - 10000 - - diff --git a/tests/performance/bit_operations_fixed_string.xml b/tests/performance/bit_operations_fixed_string.xml index 90df91f1025..c08761ba8fc 100644 --- a/tests/performance/bit_operations_fixed_string.xml +++ b/tests/performance/bit_operations_fixed_string.xml @@ -1,15 +1,5 @@ - - - 3 - 10000 - - - 5 - 60000 - - diff --git a/tests/performance/bit_operations_fixed_string_numbers.xml b/tests/performance/bit_operations_fixed_string_numbers.xml index 779aea19cdc..e10e665ac81 100644 --- a/tests/performance/bit_operations_fixed_string_numbers.xml +++ b/tests/performance/bit_operations_fixed_string_numbers.xml @@ -1,15 +1,5 @@ - - - 3 - 10000 - - - 5 - 60000 - - SELECT count() FROM numbers(10000000) WHERE NOT ignore(bitXor(reinterpretAsFixedString(number), reinterpretAsFixedString(number + 1))) SELECT count() FROM numbers(10000000) WHERE NOT ignore(bitXor(reinterpretAsFixedString(number), reinterpretAsFixedString(0xabcd0123cdef4567))) diff --git a/tests/performance/bloom_filter.xml b/tests/performance/bloom_filter.xml index 079b7a43da3..3d9096afb03 100644 --- a/tests/performance/bloom_filter.xml +++ b/tests/performance/bloom_filter.xml @@ -1,10 +1,5 @@ - - - 30000 - - DROP TABLE IF EXISTS test_bf CREATE TABLE test_bf (`id` int, `ary` Array(String), INDEX idx_ary ary TYPE bloom_filter(0.01) GRANULARITY 8192) ENGINE = MergeTree() ORDER BY id diff --git a/tests/performance/bounding_ratio.xml b/tests/performance/bounding_ratio.xml index 0d0adfaea45..113c9c4dc14 100644 --- a/tests/performance/bounding_ratio.xml +++ b/tests/performance/bounding_ratio.xml @@ -1,11 +1,5 @@ - - - - 10000 - - SELECT boundingRatio(number, number) FROM numbers(1000000) SELECT (argMax(number, number) - argMin(number, number)) / (max(number) - min(number)) FROM numbers(1000000) diff --git a/tests/performance/cidr.xml b/tests/performance/cidr.xml index 938734e3709..a83a7e19182 100644 --- a/tests/performance/cidr.xml +++ b/tests/performance/cidr.xml @@ -1,15 +1,5 @@ - - - 3 - 10000 - - - 5 - 60000 - - diff --git a/tests/performance/codecs_float_insert.xml b/tests/performance/codecs_float_insert.xml index 706a2f3c0a0..273b5c07b67 100644 --- a/tests/performance/codecs_float_insert.xml +++ b/tests/performance/codecs_float_insert.xml @@ -1,15 +1,5 @@ - - - 10 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/codecs_float_select.xml b/tests/performance/codecs_float_select.xml index 4c2f671a90e..8dc78e5f90f 100644 --- a/tests/performance/codecs_float_select.xml +++ b/tests/performance/codecs_float_select.xml @@ -1,15 +1,5 @@ - - - 10 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/codecs_int_insert.xml b/tests/performance/codecs_int_insert.xml index 1226d9020a0..8efdc031b33 100644 --- a/tests/performance/codecs_int_insert.xml +++ b/tests/performance/codecs_int_insert.xml @@ -1,15 +1,5 @@ - - - 10 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/codecs_int_select.xml b/tests/performance/codecs_int_select.xml index 8054c2b2de4..a8ed2e90069 100644 --- a/tests/performance/codecs_int_select.xml +++ b/tests/performance/codecs_int_select.xml @@ -1,15 +1,5 @@ - - - 10 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/collations.xml b/tests/performance/collations.xml index 03d77fa5e27..17b2d36b7e3 100644 --- a/tests/performance/collations.xml +++ b/tests/performance/collations.xml @@ -1,15 +1,5 @@ - - - 5 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/column_column_comparison.xml b/tests/performance/column_column_comparison.xml index 7559e03e506..88ceda7bf83 100644 --- a/tests/performance/column_column_comparison.xml +++ b/tests/performance/column_column_comparison.xml @@ -8,16 +8,6 @@ - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/columns_hashing.xml b/tests/performance/columns_hashing.xml index ca330b0d435..414c85c3853 100644 --- a/tests/performance/columns_hashing.xml +++ b/tests/performance/columns_hashing.xml @@ -9,16 +9,6 @@ - - - 5 - 60000 - - - 10 - 150000 - - SELECT count() FROM numbers(1000000) WHERE NOT ignore(geohashEncode((number % 150)*1.1 - 75, (number * 3.14 % 300)*1.1 - 150)) diff --git a/tests/performance/general_purpose_hashes.xml b/tests/performance/general_purpose_hashes.xml index 458e646f3a7..d636de1ddaa 100644 --- a/tests/performance/general_purpose_hashes.xml +++ b/tests/performance/general_purpose_hashes.xml @@ -1,15 +1,5 @@ - - - 3 - 10000 - - - 5 - 60000 - - diff --git a/tests/performance/general_purpose_hashes_on_UUID.xml b/tests/performance/general_purpose_hashes_on_UUID.xml index 9b749ae79e0..020e2a3c134 100644 --- a/tests/performance/general_purpose_hashes_on_UUID.xml +++ b/tests/performance/general_purpose_hashes_on_UUID.xml @@ -1,15 +1,5 @@ - - - 3 - 10000 - - - 5 - 60000 - - diff --git a/tests/performance/generate_table_function.xml b/tests/performance/generate_table_function.xml index c53ec285bf5..8de711304ae 100644 --- a/tests/performance/generate_table_function.xml +++ b/tests/performance/generate_table_function.xml @@ -1,10 +1,5 @@ - - - 10000 - - SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8') LIMIT 10000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8', 0, 10, 10) LIMIT 10000000); diff --git a/tests/performance/great_circle_dist.xml b/tests/performance/great_circle_dist.xml index 3b2aac65230..ff297cffc58 100644 --- a/tests/performance/great_circle_dist.xml +++ b/tests/performance/great_circle_dist.xml @@ -1,10 +1,5 @@ - - - 10000 - - SELECT count() FROM numbers(1000000) WHERE NOT ignore(greatCircleDistance((rand(1) % 360) * 1. - 180, (number % 150) * 1.2 - 90, (number % 360) + toFloat64(rand(2)) / 4294967296 - 180, (rand(3) % 180) * 1. - 90)) diff --git a/tests/performance/group_array_moving_sum.xml b/tests/performance/group_array_moving_sum.xml index 76b6b5bb980..465306d837d 100644 --- a/tests/performance/group_array_moving_sum.xml +++ b/tests/performance/group_array_moving_sum.xml @@ -1,12 +1,4 @@ - - - 30000 - - - 60000 - - 30000000000 diff --git a/tests/performance/h3.xml b/tests/performance/h3.xml index 104e777fcc5..ce00ebbc9ec 100644 --- a/tests/performance/h3.xml +++ b/tests/performance/h3.xml @@ -1,11 +1,5 @@ - - - - 10000 - - SELECT count() FROM zeros(100000) WHERE NOT ignore(geoToH3(37.62 + rand(1) / 0x100000000, 55.75 + rand(2) / 0x100000000, 15)) diff --git a/tests/performance/if_array_num.xml b/tests/performance/if_array_num.xml index 4ecd1e66daa..f3f418b809c 100644 --- a/tests/performance/if_array_num.xml +++ b/tests/performance/if_array_num.xml @@ -1,10 +1,5 @@ - - - 10000 - - SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : [4, 5]) diff --git a/tests/performance/if_array_string.xml b/tests/performance/if_array_string.xml index 40302131665..6713822e5a4 100644 --- a/tests/performance/if_array_string.xml +++ b/tests/performance/if_array_string.xml @@ -1,10 +1,5 @@ - - - 10000 - - SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? ['Hello', 'World'] : ['a', 'b', 'c']) diff --git a/tests/performance/if_string_const.xml b/tests/performance/if_string_const.xml index 7e273db36d8..69dd8f75463 100644 --- a/tests/performance/if_string_const.xml +++ b/tests/performance/if_string_const.xml @@ -1,10 +1,5 @@ - - - 10000 - - SELECT count() FROM zeros(1000000) WHERE NOT ignore(rand() % 2 ? 'hello' : 'world') SELECT count() FROM zeros(1000000) WHERE NOT ignore(rand() % 2 ? 'hello' : '') diff --git a/tests/performance/if_string_hits.xml b/tests/performance/if_string_hits.xml index ec9ea39f7cf..ca23d710185 100644 --- a/tests/performance/if_string_hits.xml +++ b/tests/performance/if_string_hits.xml @@ -1,15 +1,5 @@ - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/if_to_multiif.xml b/tests/performance/if_to_multiif.xml index 373318c316c..88fc38d48f0 100644 --- a/tests/performance/if_to_multiif.xml +++ b/tests/performance/if_to_multiif.xml @@ -1,10 +1,5 @@ - - - 10000 - - - - - 10 - 3000 - - - 100 - 60000 - - file('test_all_expr_matches.values', Values, 'd DateTime, i UInt32, s String, ni Nullable(UInt64), ns Nullable(String), ars Array(String)') diff --git a/tests/performance/inserts_arrays_lowcardinality.xml b/tests/performance/inserts_arrays_lowcardinality.xml index bca5c858576..40617fb9593 100644 --- a/tests/performance/inserts_arrays_lowcardinality.xml +++ b/tests/performance/inserts_arrays_lowcardinality.xml @@ -1,14 +1,4 @@ - - - 5 - 12000 - - - 50 - 60000 - - CREATE TABLE lot_of_string_arrays_src (`id` UInt64, `col00` Array(String), `col01` Array(String), `col02` Array(String), `col03` Array(String), `col04` Array(String), `col05` Array(String), `col06` Array(String), `col07` Array(String), `col08` Array(String), `col09` Array(String), `col10` Array(String), `col11` Array(String), `col12` Array(String), `col13` Array(String), `col14` Array(String), `col15` Array(String), `col16` Array(String), `col17` Array(String), `col18` Array(String), `col19` Array(String), `col20` Array(String), `col21` Array(String), `col22` Array(String), `col23` Array(String), `col24` Array(String), `col25` Array(String), `col26` Array(String), `col27` Array(String), `col28` Array(String), `col29` Array(String), `col30` Array(String), `col31` Array(String), `col32` Array(String), `col33` Array(String), `col34` Array(String), `col35` Array(String), `col36` Array(String), `col37` Array(String), `col38` Array(String), `col39` Array(String), `col40` Array(String), `col41` Array(String), `col42` Array(String), `col43` Array(String), `col44` Array(String), `col45` Array(String), `col46` Array(String), `col47` Array(String), `col48` Array(String), `col49` Array(String)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; CREATE TABLE lot_of_string_arrays_dst_lowcardinality (`id` UInt64, `col00` Array(LowCardinality(String)), `col01` Array(LowCardinality(String)), `col02` Array(LowCardinality(String)), `col03` Array(LowCardinality(String)), `col04` Array(LowCardinality(String)), `col05` Array(LowCardinality(String)), `col06` Array(LowCardinality(String)), `col07` Array(LowCardinality(String)), `col08` Array(LowCardinality(String)), `col09` Array(LowCardinality(String)), `col10` Array(LowCardinality(String)), `col11` Array(LowCardinality(String)), `col12` Array(LowCardinality(String)), `col13` Array(LowCardinality(String)), `col14` Array(LowCardinality(String)), `col15` Array(LowCardinality(String)), `col16` Array(LowCardinality(String)), `col17` Array(LowCardinality(String)), `col18` Array(LowCardinality(String)), `col19` Array(LowCardinality(String)), `col20` Array(LowCardinality(String)), `col21` Array(LowCardinality(String)), `col22` Array(LowCardinality(String)), `col23` Array(LowCardinality(String)), `col24` Array(LowCardinality(String)), `col25` Array(LowCardinality(String)), `col26` Array(LowCardinality(String)), `col27` Array(LowCardinality(String)), `col28` Array(LowCardinality(String)), `col29` Array(LowCardinality(String)), `col30` Array(LowCardinality(String)), `col31` Array(LowCardinality(String)), `col32` Array(LowCardinality(String)), `col33` Array(LowCardinality(String)), `col34` Array(LowCardinality(String)), `col35` Array(LowCardinality(String)), `col36` Array(LowCardinality(String)), `col37` Array(LowCardinality(String)), `col38` Array(LowCardinality(String)), `col39` Array(LowCardinality(String)), `col40` Array(LowCardinality(String)), `col41` Array(LowCardinality(String)), `col42` Array(LowCardinality(String)), `col43` Array(LowCardinality(String)), `col44` Array(LowCardinality(String)), `col45` Array(LowCardinality(String)), `col46` Array(LowCardinality(String)), `col47` Array(LowCardinality(String)), `col48` Array(LowCardinality(String)), `col49` Array(LowCardinality(String))) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; diff --git a/tests/performance/int_parsing.xml b/tests/performance/int_parsing.xml index 8a6475546bf..a9258875b5e 100644 --- a/tests/performance/int_parsing.xml +++ b/tests/performance/int_parsing.xml @@ -1,15 +1,5 @@ - - - 3 - 10000 - - - 5 - 60000 - - diff --git a/tests/performance/jit_large_requests.xml b/tests/performance/jit_large_requests.xml index 4b390c36e20..805b7f2edb1 100644 --- a/tests/performance/jit_large_requests.xml +++ b/tests/performance/jit_large_requests.xml @@ -1,10 +1,5 @@ - - - 100 - - CREATE TABLE jit_test ( diff --git a/tests/performance/jit_small_requests.xml b/tests/performance/jit_small_requests.xml index 7a4bedf6832..f90415371ce 100644 --- a/tests/performance/jit_small_requests.xml +++ b/tests/performance/jit_small_requests.xml @@ -1,15 +1,5 @@ - - - 5 - 10000 - - - 5000 - 60000 - - diff --git a/tests/performance/joins_in_memory.xml b/tests/performance/joins_in_memory.xml index 67c86f49ff2..f24e7425ecd 100644 --- a/tests/performance/joins_in_memory.xml +++ b/tests/performance/joins_in_memory.xml @@ -1,10 +1,5 @@ - - - 10 - - CREATE TABLE ints (i64 Int64, i32 Int32, i16 Int16, i8 Int8) ENGINE = Memory diff --git a/tests/performance/joins_in_memory_pmj.xml b/tests/performance/joins_in_memory_pmj.xml index 19383467fa1..6742943151e 100644 --- a/tests/performance/joins_in_memory_pmj.xml +++ b/tests/performance/joins_in_memory_pmj.xml @@ -1,10 +1,5 @@ - - - 10 - - CREATE TABLE ints (i64 Int64, i32 Int32, i16 Int16, i8 Int8) ENGINE = Memory diff --git a/tests/performance/json_extract_rapidjson.xml b/tests/performance/json_extract_rapidjson.xml index 9818abb8581..f8d40c1e58d 100644 --- a/tests/performance/json_extract_rapidjson.xml +++ b/tests/performance/json_extract_rapidjson.xml @@ -1,15 +1,5 @@ - - - 3 - 10000 - - - 5 - 60000 - - diff --git a/tests/performance/json_extract_simdjson.xml b/tests/performance/json_extract_simdjson.xml index fa18d43df3e..f9f6df5140e 100644 --- a/tests/performance/json_extract_simdjson.xml +++ b/tests/performance/json_extract_simdjson.xml @@ -1,15 +1,5 @@ - - - 3 - 10000 - - - 5 - 60000 - - diff --git a/tests/performance/leftpad.xml b/tests/performance/leftpad.xml index eb0b09c72ed..3f747054122 100644 --- a/tests/performance/leftpad.xml +++ b/tests/performance/leftpad.xml @@ -9,16 +9,6 @@ - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/linear_regression.xml b/tests/performance/linear_regression.xml index 87fa034d851..a23924939a2 100644 --- a/tests/performance/linear_regression.xml +++ b/tests/performance/linear_regression.xml @@ -1,10 +1,5 @@ - - - 10000 - - test.hits diff --git a/tests/performance/logical_functions_large.xml b/tests/performance/logical_functions_large.xml index e199094c43c..077c88fa32c 100644 --- a/tests/performance/logical_functions_large.xml +++ b/tests/performance/logical_functions_large.xml @@ -1,16 +1,6 @@ 1 - - - 60 - 30000 - - - 120 - 60000 - - 1 diff --git a/tests/performance/logical_functions_medium.xml b/tests/performance/logical_functions_medium.xml index 1c4fd2a24dc..990fe2b6040 100644 --- a/tests/performance/logical_functions_medium.xml +++ b/tests/performance/logical_functions_medium.xml @@ -1,16 +1,6 @@ 1 - - - 60 - 30000 - - - 120 - 60000 - - 1 diff --git a/tests/performance/logical_functions_small.xml b/tests/performance/logical_functions_small.xml index d3d7a2eecca..15c1e87a558 100644 --- a/tests/performance/logical_functions_small.xml +++ b/tests/performance/logical_functions_small.xml @@ -1,16 +1,6 @@ 1 - - - 15 - 20000 - - - 120 - 60000 - - 1 diff --git a/tests/performance/math.xml b/tests/performance/math.xml index 6ab497749f1..78effabaf1e 100644 --- a/tests/performance/math.xml +++ b/tests/performance/math.xml @@ -1,16 +1,6 @@ - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/merge_table_streams.xml b/tests/performance/merge_table_streams.xml index 084fa2da575..01f0444e54c 100644 --- a/tests/performance/merge_table_streams.xml +++ b/tests/performance/merge_table_streams.xml @@ -4,15 +4,6 @@ hits_100m_single - - - 60000 - 3 - - - 30 - - diff --git a/tests/performance/merge_tree_huge_pk.xml b/tests/performance/merge_tree_huge_pk.xml index 78a6cf6308e..2332769b522 100644 --- a/tests/performance/merge_tree_huge_pk.xml +++ b/tests/performance/merge_tree_huge_pk.xml @@ -1,15 +1,5 @@ - - - 10 - 12000 - - - 50 - 60000 - - CREATE TABLE huge_pk ENGINE = MergeTree ORDER BY ( diff --git a/tests/performance/merge_tree_many_partitions.xml b/tests/performance/merge_tree_many_partitions.xml index 33bb12ed22b..d3a5d204d5a 100644 --- a/tests/performance/merge_tree_many_partitions.xml +++ b/tests/performance/merge_tree_many_partitions.xml @@ -3,16 +3,6 @@ CREATE TABLE bad_partitions (x UInt64) ENGINE = MergeTree PARTITION BY x ORDER BY x INSERT INTO bad_partitions SELECT * FROM numbers(10000) - - - 5 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/merge_tree_many_partitions_2.xml b/tests/performance/merge_tree_many_partitions_2.xml index 42bb0ac29c9..6799153ed65 100644 --- a/tests/performance/merge_tree_many_partitions_2.xml +++ b/tests/performance/merge_tree_many_partitions_2.xml @@ -3,16 +3,6 @@ CREATE TABLE bad_partitions (a UInt64, b UInt64, c UInt64, d UInt64, e UInt64, f UInt64, g UInt64, h UInt64, i UInt64, j UInt64, k UInt64, l UInt64, m UInt64, n UInt64, o UInt64, p UInt64, q UInt64, r UInt64, s UInt64, t UInt64, u UInt64, v UInt64, w UInt64, x UInt64, y UInt64, z UInt64) ENGINE = MergeTree PARTITION BY x ORDER BY x INSERT INTO bad_partitions (x) SELECT * FROM numbers_mt(3000) - - - 5 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/merge_tree_simple_select.xml b/tests/performance/merge_tree_simple_select.xml index f38a5241cb5..4d449e878d3 100644 --- a/tests/performance/merge_tree_simple_select.xml +++ b/tests/performance/merge_tree_simple_select.xml @@ -1,11 +1,6 @@ - - - 10 - - CREATE TABLE simple_mergetree (EventDate Date, x UInt64) ENGINE = MergeTree ORDER BY x INSERT INTO simple_mergetree SELECT number, today() + intDiv(number, 10000000) FROM numbers_mt(100000000) diff --git a/tests/performance/mingroupby-orderbylimit1.xml b/tests/performance/mingroupby-orderbylimit1.xml index 306095a7e1c..d6ff3bedbd2 100644 --- a/tests/performance/mingroupby-orderbylimit1.xml +++ b/tests/performance/mingroupby-orderbylimit1.xml @@ -1,10 +1,5 @@ - - - 30000 - - 1 diff --git a/tests/performance/modulo.xml b/tests/performance/modulo.xml index e31de5c1701..77b544ff389 100644 --- a/tests/performance/modulo.xml +++ b/tests/performance/modulo.xml @@ -1,10 +1,5 @@ - - - 10 - - SELECT number % 128 FROM numbers(300000000) FORMAT Null diff --git a/tests/performance/ngram_distance.xml b/tests/performance/ngram_distance.xml index e90f49155b1..f102b90466d 100644 --- a/tests/performance/ngram_distance.xml +++ b/tests/performance/ngram_distance.xml @@ -13,16 +13,6 @@ 20000000000 - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/number_formatting_formats.xml b/tests/performance/number_formatting_formats.xml index c2a9a9c081d..9c9eb601b5a 100644 --- a/tests/performance/number_formatting_formats.xml +++ b/tests/performance/number_formatting_formats.xml @@ -2,16 +2,6 @@ CREATE TABLE IF NOT EXISTS table_{format} (x UInt64) ENGINE = File(`{format}`) - - - 5 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/nyc_taxi.xml b/tests/performance/nyc_taxi.xml index 92a1dd59441..b7c1cf58146 100644 --- a/tests/performance/nyc_taxi.xml +++ b/tests/performance/nyc_taxi.xml @@ -1,15 +1,5 @@ - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/order_by_decimals.xml b/tests/performance/order_by_decimals.xml index 5479181fb08..4889137865d 100644 --- a/tests/performance/order_by_decimals.xml +++ b/tests/performance/order_by_decimals.xml @@ -5,16 +5,6 @@ - - - 5 - 10000 - - - 50 - 60000 - - SELECT toInt32(number) AS n FROM numbers(1000000) ORDER BY n DESC FORMAT Null SELECT toDecimal32(number, 0) AS n FROM numbers(1000000) ORDER BY n FORMAT Null diff --git a/tests/performance/order_by_read_in_order.xml b/tests/performance/order_by_read_in_order.xml index e37e4df4681..5749a49a3aa 100644 --- a/tests/performance/order_by_read_in_order.xml +++ b/tests/performance/order_by_read_in_order.xml @@ -1,15 +1,5 @@ - - - 5 - 10000 - - - 500 - 60000 - - diff --git a/tests/performance/order_by_single_column.xml b/tests/performance/order_by_single_column.xml index 148b14e8959..9b708ea393c 100644 --- a/tests/performance/order_by_single_column.xml +++ b/tests/performance/order_by_single_column.xml @@ -9,16 +9,6 @@ - - - 5 - 10000 - - - 50 - 60000 - - SELECT URL as col FROM hits_100m_single ORDER BY col LIMIT 1000,1 SELECT SearchPhrase as col FROM hits_100m_single ORDER BY col LIMIT 10000,1 diff --git a/tests/performance/parallel_insert.xml b/tests/performance/parallel_insert.xml index 7af2a1418d9..34c45e08bc0 100644 --- a/tests/performance/parallel_insert.xml +++ b/tests/performance/parallel_insert.xml @@ -1,9 +1,4 @@ - - - 2 - - default.hits_10m_single diff --git a/tests/performance/parse_engine_file.xml b/tests/performance/parse_engine_file.xml index a88945125b3..f962cbf4369 100644 --- a/tests/performance/parse_engine_file.xml +++ b/tests/performance/parse_engine_file.xml @@ -4,16 +4,6 @@ INSERT INTO table_{format} SELECT * FROM test.hits LIMIT 100000 - - - 5 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/pre_limit_no_sorting.xml b/tests/performance/pre_limit_no_sorting.xml index e93aef049aa..a1e50f736b8 100644 --- a/tests/performance/pre_limit_no_sorting.xml +++ b/tests/performance/pre_limit_no_sorting.xml @@ -1,14 +1,4 @@ - - - 10 - 200 - - - 100 - 1000 - - SELECT sum(number) FROM (select number from system.numbers_mt limit 1000000000) diff --git a/tests/performance/prewhere.xml b/tests/performance/prewhere.xml index e3350d765ee..40a12a68bb9 100644 --- a/tests/performance/prewhere.xml +++ b/tests/performance/prewhere.xml @@ -1,15 +1,5 @@ - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/random_printable_ascii.xml b/tests/performance/random_printable_ascii.xml index 20868bb4c27..3eb1441a4cc 100644 --- a/tests/performance/random_printable_ascii.xml +++ b/tests/performance/random_printable_ascii.xml @@ -1,10 +1,5 @@ - - - 10000 - - SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomPrintableASCII(10)) diff --git a/tests/performance/range.xml b/tests/performance/range.xml index 95b8455057e..c676f9124ba 100644 --- a/tests/performance/range.xml +++ b/tests/performance/range.xml @@ -1,10 +1,5 @@ - - - 10000 - - SELECT range(number % 100) FROM numbers(10000000) FORMAT Null diff --git a/tests/performance/read_hits_with_aio.xml b/tests/performance/read_hits_with_aio.xml index 850fd0fbadc..1e9a81f7693 100644 --- a/tests/performance/read_hits_with_aio.xml +++ b/tests/performance/read_hits_with_aio.xml @@ -1,10 +1,5 @@ - - - 30000 - - hits_100m_single diff --git a/tests/performance/right.xml b/tests/performance/right.xml index 73030e52f21..ac889e21e73 100644 --- a/tests/performance/right.xml +++ b/tests/performance/right.xml @@ -4,14 +4,6 @@ hits_100m_single - - - 10000 - - - 20000 - - diff --git a/tests/performance/round_down.xml b/tests/performance/round_down.xml index f453467ab2d..c309a767843 100644 --- a/tests/performance/round_down.xml +++ b/tests/performance/round_down.xml @@ -1,13 +1,5 @@ - - - 10000 - - - 20000 - - SELECT count() FROM zeros(10000000) WHERE NOT ignore(roundDuration(rand() % 65536)) diff --git a/tests/performance/round_methods.xml b/tests/performance/round_methods.xml index 0e560b2eae6..fac9c1908b0 100644 --- a/tests/performance/round_methods.xml +++ b/tests/performance/round_methods.xml @@ -1,13 +1,5 @@ - - - 10000 - - - 20000 - - SELECT count() FROM numbers(1000000) WHERE NOT ignore(round(toInt64(number), -2)) diff --git a/tests/performance/scalar.xml b/tests/performance/scalar.xml index e8e487a80da..b50aef8747c 100644 --- a/tests/performance/scalar.xml +++ b/tests/performance/scalar.xml @@ -1,14 +1,5 @@ - - - 30000 - - - 5000 - 60000 - - CREATE TABLE cdp_tags (tag_id String, mid_seqs AggregateFunction(groupBitmap, UInt32)) engine=MergeTree() ORDER BY (tag_id) SETTINGS index_granularity=1 diff --git a/tests/performance/select_format.xml b/tests/performance/select_format.xml index e47d981c4d7..82d5198a71a 100644 --- a/tests/performance/select_format.xml +++ b/tests/performance/select_format.xml @@ -2,16 +2,6 @@ CREATE TABLE IF NOT EXISTS table_{format} ENGINE = File({format}, '/dev/null') AS test.hits - - - 5 - 10000 - - - 100 - 60000 - - diff --git a/tests/performance/set.xml b/tests/performance/set.xml index 345d9c05573..576a26390d1 100644 --- a/tests/performance/set.xml +++ b/tests/performance/set.xml @@ -3,14 +3,6 @@ long - - - 10000 - - - 20000 - - diff --git a/tests/performance/set_hits.xml b/tests/performance/set_hits.xml index 09860aa1cd7..8b9ae1da83b 100644 --- a/tests/performance/set_hits.xml +++ b/tests/performance/set_hits.xml @@ -5,15 +5,6 @@ hits_100m_single - - - 8000 - - - 7000 - 20000 - - SELECT count() FROM hits_100m_single WHERE UserID IN (SELECT UserID FROM hits_100m_single WHERE AdvEngineID != 0) diff --git a/tests/performance/set_index.xml b/tests/performance/set_index.xml index f158c481d93..e1ced2ba53c 100644 --- a/tests/performance/set_index.xml +++ b/tests/performance/set_index.xml @@ -3,15 +3,6 @@ CREATE TABLE test_in (`a` UInt32) ENGINE = MergeTree() ORDER BY a INSERT INTO test_in SELECT number FROM numbers(500000000) - - - 8000 - - - 7000 - 20000 - - SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM numbers(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' diff --git a/tests/performance/simple_join_query.xml b/tests/performance/simple_join_query.xml index 92fdfd23f93..98f2b1eaebf 100644 --- a/tests/performance/simple_join_query.xml +++ b/tests/performance/simple_join_query.xml @@ -1,13 +1,5 @@ - - - 30000 - - - 60000 - - CREATE TABLE join_table(A Int64, S0 String, S1 String, S2 String, S3 String) ENGINE = MergeTree ORDER BY A diff --git a/tests/performance/slices_hits.xml b/tests/performance/slices_hits.xml index 1745df3328c..4a6813579bf 100644 --- a/tests/performance/slices_hits.xml +++ b/tests/performance/slices_hits.xml @@ -1,15 +1,5 @@ - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/sort.xml b/tests/performance/sort.xml index 652dd7f4670..e5781548a37 100644 --- a/tests/performance/sort.xml +++ b/tests/performance/sort.xml @@ -1,10 +1,5 @@ - - - 10 - - CREATE TABLE rand_unlimited_10m_8 (key UInt8) Engine = Memory diff --git a/tests/performance/string_join.xml b/tests/performance/string_join.xml index 6aa2c576b4e..477f62c1327 100644 --- a/tests/performance/string_join.xml +++ b/tests/performance/string_join.xml @@ -1,10 +1,5 @@ - - - 10 - - diff --git a/tests/performance/string_set.xml b/tests/performance/string_set.xml index 7890ab11a4a..bbbfe2d3c2b 100644 --- a/tests/performance/string_set.xml +++ b/tests/performance/string_set.xml @@ -1,10 +1,5 @@ - - - 10 - - diff --git a/tests/performance/string_sort.xml b/tests/performance/string_sort.xml index 6a4e68270f9..71b56bdb9d6 100644 --- a/tests/performance/string_sort.xml +++ b/tests/performance/string_sort.xml @@ -5,16 +5,6 @@ - - - 5 - 10000 - - - 50 - 60000 - - diff --git a/tests/performance/sum_map.xml b/tests/performance/sum_map.xml index a88983fdbea..4f9ce56488c 100644 --- a/tests/performance/sum_map.xml +++ b/tests/performance/sum_map.xml @@ -1,10 +1,5 @@ - - - 30000 - - 1 diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index fc910077c9f..256fd623b3c 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -1,10 +1,5 @@ - - - 12000 - - 30000000000 diff --git a/tests/performance/trim_numbers.xml b/tests/performance/trim_numbers.xml index 62e26f8245a..35cd479d48c 100644 --- a/tests/performance/trim_numbers.xml +++ b/tests/performance/trim_numbers.xml @@ -1,13 +1,5 @@ - - - 10000 - - - 20000 - - diff --git a/tests/performance/trim_urls.xml b/tests/performance/trim_urls.xml index f29d878682f..276a12bc570 100644 --- a/tests/performance/trim_urls.xml +++ b/tests/performance/trim_urls.xml @@ -4,14 +4,6 @@ hits_100m_single - - - 10000 - - - 20000 - - diff --git a/tests/performance/trim_whitespace.xml b/tests/performance/trim_whitespace.xml index 8ec4aeaa54e..049387bbc0c 100644 --- a/tests/performance/trim_whitespace.xml +++ b/tests/performance/trim_whitespace.xml @@ -10,11 +10,6 @@ from numbers_mt(100000000); - - - 30000 - - diff --git a/tests/performance/uniq.xml b/tests/performance/uniq.xml index 0b7c8e58c86..2766c95e6a7 100644 --- a/tests/performance/uniq.xml +++ b/tests/performance/uniq.xml @@ -5,12 +5,6 @@ 30000000000 - - - 5000 - 20000 - - diff --git a/tests/performance/url_hits.xml b/tests/performance/url_hits.xml index f9383eb3910..c8cf119a7d7 100644 --- a/tests/performance/url_hits.xml +++ b/tests/performance/url_hits.xml @@ -4,14 +4,6 @@ hits_100m_single - - - 10000 - - - 20000 - - diff --git a/tests/performance/vectorize_aggregation_combinators.xml b/tests/performance/vectorize_aggregation_combinators.xml index 88870b56d1f..47ac0719bb5 100644 --- a/tests/performance/vectorize_aggregation_combinators.xml +++ b/tests/performance/vectorize_aggregation_combinators.xml @@ -1,14 +1,6 @@ - - - 30000 - - - 60000 - - diff --git a/tests/performance/visit_param_extract_raw.xml b/tests/performance/visit_param_extract_raw.xml index ca46c79c9b5..7be780d5d42 100644 --- a/tests/performance/visit_param_extract_raw.xml +++ b/tests/performance/visit_param_extract_raw.xml @@ -1,9 +1,4 @@ - - - 10000 - - diff --git a/tests/performance/website.xml b/tests/performance/website.xml index c21f09c57d8..6ed60c0860a 100644 --- a/tests/performance/website.xml +++ b/tests/performance/website.xml @@ -5,15 +5,6 @@ hits_100m_single - - - 60000 - 3 - - - 30 - - From 46a5c75bebf6e84ac88d359980e85055e0b8de91 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 18 Apr 2020 16:11:57 +0300 Subject: [PATCH 380/743] Update MergeTreeData.h --- src/Storages/MergeTree/MergeTreeData.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index eb2a0dd8774..90eae080cb1 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -625,7 +625,6 @@ public: return storage_settings.get(); } - /// Get relative table path String getRelativeDataPath() const { return relative_data_path; } /// Get table path on disk From d64c6b2134fdccaed01aca46e157b6728bdbde90 Mon Sep 17 00:00:00 2001 From: Vitaly Date: Sat, 18 Apr 2020 16:54:45 +0300 Subject: [PATCH 381/743] build on FreeBSD fix (amd64) --- cmake/freebsd/default_libs.cmake | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/cmake/freebsd/default_libs.cmake b/cmake/freebsd/default_libs.cmake index d0dea46102d..d60df52bc6d 100644 --- a/cmake/freebsd/default_libs.cmake +++ b/cmake/freebsd/default_libs.cmake @@ -4,7 +4,11 @@ if (NOT COMPILER_CLANG) message (FATAL_ERROR "FreeBSD build is supported only for Clang") endif () -execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-x86_64.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +if (${CMAKE_SYSTEM_PROCESSOR} STREQUAL "amd64") + execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-x86_64.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +else () + execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +endif () set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread") From 14647ed03b154d9237c27bbb29f54ee5b8c91377 Mon Sep 17 00:00:00 2001 From: Andrew Onyshchuk Date: Sat, 18 Apr 2020 15:15:39 -0500 Subject: [PATCH 382/743] Add Avro nested fields insert support --- .../Formats/Impl/AvroRowInputFormat.cpp | 87 +++++++++--------- .../Formats/Impl/AvroRowInputFormat.h | 46 +++++++-- .../queries/0_stateless/01060_avro.reference | 4 + tests/queries/0_stateless/01060_avro.sh | 7 +- .../0_stateless/data_avro/generate_avro.sh | 1 + .../queries/0_stateless/data_avro/nested.avro | Bin 0 -> 341 bytes .../queries/0_stateless/data_avro/nested.avsc | 17 ++++ .../queries/0_stateless/data_avro/nested.json | 1 + 8 files changed, 113 insertions(+), 50 deletions(-) create mode 100644 tests/queries/0_stateless/data_avro/nested.avro create mode 100644 tests/queries/0_stateless/data_avro/nested.avsc create mode 100644 tests/queries/0_stateless/data_avro/nested.json diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 52ceaf063b7..585cc05cba0 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -102,10 +102,6 @@ private: ReadBuffer & in; }; -static void deserializeNoop(IColumn &, avro::Decoder &) -{ -} - /// Insert value with conversion to the column of target type. template static void insertNumber(IColumn & column, WhichDataType type, T value) @@ -441,8 +437,43 @@ AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(avro::NodePtr root_node) } } +void AvroDeserializer::createActions(const Block & header, const avro::NodePtr& node, std::string current_path) +{ + if (node->type() == avro::AVRO_RECORD) + { + for (size_t i = 0; i < node->leaves(); ++i) + { + const auto& field_node = node->leafAt(i); + const auto& field_name = node->nameAt(i); + auto field_path = current_path.empty() ? field_name : current_path + "." + field_name; + createActions(header, field_node, field_path); + } + } + else + { + if (header.has(current_path)) + { + auto target_column_idx = header.getPositionByName(current_path); + const auto& column = header.getByPosition(target_column_idx); + try + { + actions.emplace_back(target_column_idx, createDeserializeFn(node, column.type)); + } + catch (Exception & e) + { + e.addMessage("column " + column.name); + throw; + } + column_found[target_column_idx] = true; + } + else + { + actions.emplace_back(createSkipFn(node)); + } + } +} -AvroDeserializer::AvroDeserializer(const ColumnsWithTypeAndName & columns, avro::ValidSchema schema) +AvroDeserializer::AvroDeserializer(const Block & header, avro::ValidSchema schema) { const auto & schema_root = schema.root(); if (schema_root->type() != avro::AVRO_RECORD) @@ -450,48 +481,23 @@ AvroDeserializer::AvroDeserializer(const ColumnsWithTypeAndName & columns, avro: throw Exception("Root schema must be a record", ErrorCodes::TYPE_MISMATCH); } - field_mapping.resize(schema_root->leaves(), -1); + column_found.resize(header.columns()); + createActions(header, schema_root); - for (size_t i = 0; i < schema_root->leaves(); ++i) + for (size_t i = 0; i < header.columns(); ++i) { - skip_fns.push_back(createSkipFn(schema_root->leafAt(i))); - deserialize_fns.push_back(&deserializeNoop); - } - - for (size_t i = 0; i < columns.size(); ++i) - { - const auto & column = columns[i]; - size_t field_index = 0; - if (!schema_root->nameIndex(column.name, field_index)) + if (!column_found[i]) { - throw Exception("Field " + column.name + " not found in Avro schema", ErrorCodes::THERE_IS_NO_COLUMN); + throw Exception("Field " + header.getByPosition(i).name + " not found in Avro schema", ErrorCodes::THERE_IS_NO_COLUMN); } - auto field_schema = schema_root->leafAt(field_index); - try - { - deserialize_fns[field_index] = createDeserializeFn(field_schema, column.type); - } - catch (Exception & e) - { - e.addMessage("column " + column.name); - throw; - } - field_mapping[field_index] = i; } } void AvroDeserializer::deserializeRow(MutableColumns & columns, avro::Decoder & decoder) const { - for (size_t i = 0; i < field_mapping.size(); i++) + for (const auto& action : actions) { - if (field_mapping[i] >= 0) - { - deserialize_fns[i](*columns[field_mapping[i]], decoder); - } - else - { - skip_fns[i](decoder); - } + action.execute(columns, decoder); } } @@ -499,7 +505,7 @@ void AvroDeserializer::deserializeRow(MutableColumns & columns, avro::Decoder & AvroRowInputFormat::AvroRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : IRowInputFormat(header_, in_, params_) , file_reader(std::make_unique(in_)) - , deserializer(header_.getColumnsWithTypeAndName(), file_reader.dataSchema()) + , deserializer(output.getHeader(), file_reader.dataSchema()) { file_reader.init(); } @@ -626,8 +632,7 @@ static uint32_t readConfluentSchemaId(ReadBuffer & in) AvroConfluentRowInputFormat::AvroConfluentRowInputFormat( const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(header_.cloneEmpty(), in_, params_) - , header_columns(header_.getColumnsWithTypeAndName()) + : IRowInputFormat(header_, in_, params_) , schema_registry(getConfluentSchemaRegistry(format_settings_)) , input_stream(std::make_unique(in)) , decoder(avro::binaryDecoder()) @@ -655,7 +660,7 @@ const AvroDeserializer & AvroConfluentRowInputFormat::getOrCreateDeserializer(Sc if (it == deserializer_cache.end()) { auto schema = schema_registry->getSchema(schema_id); - AvroDeserializer deserializer(header_columns, schema); + AvroDeserializer deserializer(output.getHeader(), schema); it = deserializer_cache.emplace(schema_id, deserializer).first; } return it->second; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index b54c8ecede5..49a34261cfc 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -22,7 +22,7 @@ namespace DB class AvroDeserializer { public: - AvroDeserializer(const ColumnsWithTypeAndName & columns, avro::ValidSchema schema); + AvroDeserializer(const Block & header, avro::ValidSchema schema); void deserializeRow(MutableColumns & columns, avro::Decoder & decoder) const; private: @@ -31,15 +31,46 @@ private: static DeserializeFn createDeserializeFn(avro::NodePtr root_node, DataTypePtr target_type); SkipFn createSkipFn(avro::NodePtr root_node); - /// Map from field index in Avro schema to column number in block header. Or -1 if there is no corresponding column. - std::vector field_mapping; + struct Action + { + enum Type { Deserialize, Skip }; + Type type; + /// Deserialize + int target_column_idx; + DeserializeFn deserialize_fn; + /// Skip + SkipFn skip_fn; - /// How to skip the corresponding field in Avro schema. - std::vector skip_fns; + Action(int target_column_idx_, DeserializeFn deserialize_fn_) + : type(Deserialize) + , target_column_idx(target_column_idx_) + , deserialize_fn(deserialize_fn_) {} - /// How to deserialize the corresponding field in Avro schema. - std::vector deserialize_fns; + Action(SkipFn skip_fn_) + : type(Skip) + , skip_fn(skip_fn_) {} + void execute(MutableColumns & columns, avro::Decoder & decoder) const + { + switch(type) + { + case Deserialize: + deserialize_fn(*columns[target_column_idx], decoder); + break; + case Skip: + skip_fn(decoder); + break; + } + } + }; + + /// Populate actions by recursively traversing root schema + void createActions(const Block & header, const avro::NodePtr& node, std::string current_path = ""); + + /// Bitmap of columns found in Avro schema + std::vector column_found; + /// Deserialize/Skip actions for a row + std::vector actions; /// Map from name of named Avro type (record, enum, fixed) to SkipFn. /// This is to avoid infinite recursion when Avro schema contains self-references. e.g. LinkedList std::map symbolic_skip_fn_map; @@ -73,7 +104,6 @@ public: class SchemaRegistry; private: - const ColumnsWithTypeAndName header_columns; std::shared_ptr schema_registry; using SchemaId = uint32_t; std::unordered_map deserializer_cache; diff --git a/tests/queries/0_stateless/01060_avro.reference b/tests/queries/0_stateless/01060_avro.reference index a21e7a3a101..0550967a224 100644 --- a/tests/queries/0_stateless/01060_avro.reference +++ b/tests/queries/0_stateless/01060_avro.reference @@ -19,6 +19,10 @@ = references "a1","c1" "a2","c2" += nested +1,"b1",2.2,2.3,"c3" +2.3,"b1",1,"c3" +not found = compression 1000 1000 diff --git a/tests/queries/0_stateless/01060_avro.sh b/tests/queries/0_stateless/01060_avro.sh index 15e97abfa52..a64b2884731 100755 --- a/tests/queries/0_stateless/01060_avro.sh +++ b/tests/queries/0_stateless/01060_avro.sh @@ -27,6 +27,11 @@ cat $DATA_DIR/logical_types.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --out echo = references cat $DATA_DIR/references.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S "a String, c String" -q 'select * from table' +echo = nested +cat $DATA_DIR/nested.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S 'a Int64, "b.a" String, "b.b" Double, "b.c" Double, c String' -q 'select * from table' +cat $DATA_DIR/nested.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S '"b.c" Double, "b.a" String, a Int64, c String' -q 'select * from table' +cat $DATA_DIR/nested.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S '"b" Double' -q 'select * from table' 2>&1 | grep -i 'not found' -o + echo = compression cat $DATA_DIR/simple.null.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S 'a Int64' -q 'select count() from table' cat $DATA_DIR/simple.deflate.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S 'a Int64' -q 'select count() from table' @@ -68,4 +73,4 @@ ${CLICKHOUSE_LOCAL} -q "select toInt64(number) as a from numbers(0) format Avro ${CLICKHOUSE_LOCAL} -q "select toInt64(number) as a from numbers(1000) format Avro" | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S "$S4" -q 'select count() from table' # type supported via conversion -${CLICKHOUSE_LOCAL} -q "select toInt16(123) as a format Avro" | wc -c \ No newline at end of file +${CLICKHOUSE_LOCAL} -q "select toInt16(123) as a format Avro" | wc -c | tr -d ' ' \ No newline at end of file diff --git a/tests/queries/0_stateless/data_avro/generate_avro.sh b/tests/queries/0_stateless/data_avro/generate_avro.sh index 6ec26efc049..b6ec75ad4dd 100755 --- a/tests/queries/0_stateless/data_avro/generate_avro.sh +++ b/tests/queries/0_stateless/data_avro/generate_avro.sh @@ -8,6 +8,7 @@ avro-tools fromjson --schema-file complex.avsc complex.json > complex.avro avro-tools fromjson --schema-file logical_types.avsc logical_types.json > logical_types.avro avro-tools fromjson --schema-file empty.avsc empty.json > empty.avro avro-tools fromjson --schema-file references.avsc references.json > references.avro +avro-tools fromjson --schema-file nested.avsc nested.json > nested.avro #compression avro-tools fromjson --codec null --schema-file simple.avsc simple.json > simple.null.avro diff --git a/tests/queries/0_stateless/data_avro/nested.avro b/tests/queries/0_stateless/data_avro/nested.avro new file mode 100644 index 0000000000000000000000000000000000000000..1415c45d32824a6acb7c7970d2d0ce6ba117a5e1 GIT binary patch literal 341 zcmeZI%3@>@ODrqO*DFrWNX*jgl4A<4r`j;$2|heC3GN@_AkUTIDaLx`Prul0)F1@8ijv%Yrm#53tJu_PJJ Tnh64o4ryS(D!_4( literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_avro/nested.avsc b/tests/queries/0_stateless/data_avro/nested.avsc new file mode 100644 index 00000000000..966dc6defb3 --- /dev/null +++ b/tests/queries/0_stateless/data_avro/nested.avsc @@ -0,0 +1,17 @@ +{ + "type": "record", + "name": "main", + "fields": [ + {"name": "a", "type": "long"}, + {"name": "b", "type": { + "type": "record", + "name": "sub1", + "fields": [ + {"name": "a", "type": "string"}, + {"name": "b", "type": "double"}, + {"name": "c", "type": "double"} + ] + }}, + {"name": "c", "type": "string"} + ] + } \ No newline at end of file diff --git a/tests/queries/0_stateless/data_avro/nested.json b/tests/queries/0_stateless/data_avro/nested.json new file mode 100644 index 00000000000..63a7bc40e4b --- /dev/null +++ b/tests/queries/0_stateless/data_avro/nested.json @@ -0,0 +1 @@ +{"a":1, "b": {"a":"b1", "b": 2.2, "c": 2.3}, "c": "c3"} From 673ddc9d77c12556d228be7cfc0fb4fe381d6bd4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Apr 2020 20:23:06 +0300 Subject: [PATCH 383/743] Drop superfluous locking for atomic in DirectoryMonitor --- src/Storages/Distributed/DirectoryMonitor.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 01bf0798a63..d21c2eb0d4c 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -100,10 +100,7 @@ StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor() { if (!quit) { - { - quit = true; - std::lock_guard lock{mutex}; - } + quit = true; cond.notify_one(); thread.join(); } @@ -122,10 +119,7 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() { if (!quit) { - { - quit = true; - std::lock_guard lock{mutex}; - } + quit = true; cond.notify_one(); thread.join(); } From 5ffd8bdd75b907facb4ca563b3462241af00eae7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Apr 2020 20:38:34 +0300 Subject: [PATCH 384/743] Cleanup 01040_distributed_directory_monitor_batch_inserts --- ..._distributed_directory_monitor_batch_inserts.sql | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.sql b/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.sql index ffc33ce6949..dbec319ab76 100644 --- a/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.sql +++ b/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.sql @@ -2,8 +2,11 @@ SET distributed_directory_monitor_batch_inserts=1; SET distributed_directory_monitor_sleep_time_ms=10; SET distributed_directory_monitor_max_sleep_time_ms=100; -CREATE TABLE test (key UInt64) ENGINE=TinyLog(); -CREATE TABLE dist_test AS test Engine=Distributed(test_cluster_two_shards, currentDatabase(), test, key); -INSERT INTO dist_test SELECT toUInt64(number) FROM numbers(2); -SYSTEM FLUSH DISTRIBUTED dist_test; -SELECT * FROM dist_test; +DROP TABLE IF EXISTS test_01040; +DROP TABLE IF EXISTS dist_test_01040; + +CREATE TABLE test_01040 (key UInt64) ENGINE=TinyLog(); +CREATE TABLE dist_test_01040 AS test_01040 Engine=Distributed(test_cluster_two_shards, currentDatabase(), test_01040, key); +INSERT INTO dist_test_01040 SELECT toUInt64(number) FROM numbers(2); +SYSTEM FLUSH DISTRIBUTED dist_test_01040; +SELECT * FROM dist_test_01040; From 8f3b14ab221f377483f3d153ba8c5bb6c8a7511f Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sat, 18 Apr 2020 20:28:51 -0400 Subject: [PATCH 385/743] Fixing 00964_live_view_watch_events_heartbeat.py test to try to make it consistent. --- .../00964_live_view_watch_events_heartbeat.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index 9d0fb7e941a..4418c22872a 100755 --- a/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -23,7 +23,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.lv') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.mt') + client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) client1.send('SET live_view_heartbeat_interval=1') client1.expect(prompt) @@ -31,14 +31,16 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE LIVE VIEW test.lv WITH TIMEOUT AS SELECT sum(a) FROM test.mt') client1.expect(prompt) - client1.send('WATCH test.lv EVENTS') + client1.send('WATCH test.lv EVENTS FORMAT CSV') + client1.expect('Progress: 1.00 rows.*\)') client2.send('INSERT INTO test.mt VALUES (1)') - client1.expect('1.*' + end_of_block) + client2.expect(prompt) + client1.expect('Progress: 2.00 rows.*\)') client2.send('INSERT INTO test.mt VALUES (2),(3)') - client1.expect('[23].*' + end_of_block) - client1.expect('Progress: [23]\.00 rows.*\)') + client2.expect(prompt) + client1.expect('Progress: 3.00 rows.*\)') # wait for heartbeat - client1.expect('Progress: [23]\.00 rows.*\)') + client1.expect('Progress: 3.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') match = client1.expect('(%s)|([#\$] )' % prompt) From 68f2e934439766f235361abd4339164c8b8dd7d0 Mon Sep 17 00:00:00 2001 From: Andrew Onyshchuk Date: Sat, 18 Apr 2020 22:54:22 -0500 Subject: [PATCH 386/743] Fix style --- src/Processors/Formats/Impl/AvroRowInputFormat.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index 49a34261cfc..6245d704e74 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -52,7 +52,7 @@ private: void execute(MutableColumns & columns, avro::Decoder & decoder) const { - switch(type) + switch (type) { case Deserialize: deserialize_fn(*columns[target_column_idx], decoder); From 201d5d5b25718f2dd1ecabf9828be3ad635f05bd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 15 Apr 2020 09:35:23 +0300 Subject: [PATCH 387/743] Update comment for background_schedule_pool_size Include info about: - kafka streaming - dns cache updates --- 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 73fc4578ce5..9cd33962f6b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -81,7 +81,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, background_buffer_flush_schedule_pool_size, 16, "Number of threads performing background flush for tables with Buffer engine. Only has meaning at server startup.", 0) \ M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ M(SettingUInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \ - M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.", 0) \ + M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, kafka streaming, dns cache updates. Only has meaning at server startup.", 0) \ \ M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \ M(SettingMilliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \ From 5d11118cc9edcd98a4cff73acc6901c521f5827e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Apr 2020 21:12:08 +0300 Subject: [PATCH 388/743] Use thread pool (background_distributed_schedule_pool_size) for distributed sends After #8756 the problem with 1 thread for each (distributed table, disk) for distributed sends became even worse (since there can be multiple disks), so use predefined thread pool for this tasks, that can be controlled with background_distributed_schedule_pool_size knob. --- src/Core/Settings.h | 1 + src/Interpreters/Context.cpp | 10 ++++ src/Interpreters/Context.h | 1 + src/Storages/Distributed/DirectoryMonitor.cpp | 47 +++++++++++++------ src/Storages/Distributed/DirectoryMonitor.h | 12 +++-- .../DistributedBlockOutputStream.cpp | 7 ++- src/Storages/StorageDistributed.cpp | 11 +++-- src/Storages/StorageDistributed.h | 2 +- 8 files changed, 67 insertions(+), 24 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9cd33962f6b..cffdd4a66e4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -82,6 +82,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ M(SettingUInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \ M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, kafka streaming, dns cache updates. Only has meaning at server startup.", 0) \ + M(SettingUInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \ \ M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \ M(SettingMilliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5d8a0e53276..647c3fb8020 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -321,6 +321,7 @@ struct ContextShared std::optional background_pool; /// The thread pool for the background work performed by the tables. std::optional background_move_pool; /// The thread pool for the background moves performed by the tables. std::optional schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) + std::optional distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker; /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. @@ -418,6 +419,7 @@ struct ContextShared background_pool.reset(); background_move_pool.reset(); schedule_pool.reset(); + distributed_schedule_pool.reset(); ddl_worker.reset(); /// Stop trace collector if any @@ -1348,6 +1350,14 @@ BackgroundSchedulePool & Context::getSchedulePool() return *shared->schedule_pool; } +BackgroundSchedulePool & Context::getDistributedSchedulePool() +{ + auto lock = getLock(); + if (!shared->distributed_schedule_pool) + shared->distributed_schedule_pool.emplace(settings.background_distributed_schedule_pool_size); + return *shared->distributed_schedule_pool; +} + void Context::setDDLWorker(std::unique_ptr ddl_worker) { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 11937d4fc89..1f81cdbc58b 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -475,6 +475,7 @@ public: BackgroundProcessingPool & getBackgroundPool(); BackgroundProcessingPool & getBackgroundMovePool(); BackgroundSchedulePool & getSchedulePool(); + BackgroundSchedulePool & getDistributedSchedulePool(); void setDDLWorker(std::unique_ptr ddl_worker); DDLWorker & getDDLWorker() const; diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index d21c2eb0d4c..e937d5e8a90 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include @@ -78,7 +77,7 @@ namespace StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( - StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_) + StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool_) /// It's important to initialize members before `thread` to avoid race. : storage(storage_) , pool(std::move(pool_)) @@ -92,7 +91,10 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , max_sleep_time{storage.global_context.getSettingsRef().distributed_directory_monitor_max_sleep_time_ms.totalMilliseconds()} , log{&Logger::get(getLoggerName())} , monitor_blocker(monitor_blocker_) + , bg_pool(bg_pool_) { + task_handle = bg_pool.createTask(getLoggerName() + "/Bg", [this]{ run(); }); + task_handle->activateAndSchedule(); } @@ -102,7 +104,7 @@ StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor() { quit = true; cond.notify_one(); - thread.join(); + task_handle->deactivate(); } } @@ -121,7 +123,7 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() { quit = true; cond.notify_one(); - thread.join(); + task_handle->deactivate(); } Poco::File(path).remove(true); @@ -130,16 +132,11 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() void StorageDistributedDirectoryMonitor::run() { - setThreadName("DistrDirMonitor"); - std::unique_lock lock{mutex}; - const auto quit_requested = [this] { return quit.load(std::memory_order_relaxed); }; - - while (!quit_requested()) + while (!quit) { - auto do_sleep = true; - + bool do_sleep = true; if (!monitor_blocker.isCancelled()) { try @@ -161,15 +158,25 @@ void StorageDistributedDirectoryMonitor::run() LOG_DEBUG(log, "Skipping send data over distributed table."); } - if (do_sleep) - cond.wait_for(lock, sleep_time, quit_requested); - const auto now = std::chrono::system_clock::now(); if (now - last_decrease_time > decrease_error_count_period) { error_count /= 2; last_decrease_time = now; } + + if (do_sleep) + break; + } + + if (!quit) + { + /// If there is no error, then it will be scheduled by the DistributedBlockOutputStream, + /// so this is just in case, hence it is distributed_directory_monitor_max_sleep_time_ms + if (error_count) + task_handle->scheduleAfter(sleep_time.count()); + else + task_handle->scheduleAfter(max_sleep_time.count()); } } @@ -580,6 +587,13 @@ BlockInputStreamPtr StorageDistributedDirectoryMonitor::createStreamFromFile(con return std::make_shared(file_name); } +bool StorageDistributedDirectoryMonitor::scheduleAfter(size_t ms) +{ + if (quit) + return false; + return task_handle->scheduleAfter(ms); +} + void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map & files) { std::unordered_set file_indices_to_skip; @@ -708,8 +722,13 @@ std::string StorageDistributedDirectoryMonitor::getLoggerName() const void StorageDistributedDirectoryMonitor::updatePath(const std::string & new_path) { std::lock_guard lock{mutex}; + + task_handle->deactivate(); + path = new_path; current_batch_file_path = path + "current_batch.txt"; + + task_handle->activateAndSchedule(); } } diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index 475a3bc7bc6..61d51e5acfd 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -1,10 +1,9 @@ #pragma once #include -#include +#include #include -#include #include #include #include @@ -20,7 +19,7 @@ class StorageDistributedDirectoryMonitor { public: StorageDistributedDirectoryMonitor( - StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_); + StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool_); ~StorageDistributedDirectoryMonitor(); @@ -33,6 +32,9 @@ public: void shutdownAndDropAllData(); static BlockInputStreamPtr createStreamFromFile(const String & file_name); + + /// For scheduling via DistributedBlockOutputStream + bool scheduleAfter(size_t ms); private: void run(); bool processFiles(); @@ -67,7 +69,9 @@ private: std::condition_variable cond; Logger * log; ActionBlocker & monitor_blocker; - ThreadFromGlobalPool thread{&StorageDistributedDirectoryMonitor::run, this}; + + BackgroundSchedulePool & bg_pool; + BackgroundSchedulePoolTaskHolder task_handle; /// Read insert query and insert settings for backward compatible. static void readHeader(ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Logger * log); diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index b0695ccad1b..3a341f9f43c 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -589,8 +589,8 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: const std::string path(disk + data_path + dir_name + '/'); /// ensure shard subdirectory creation and notify storage - if (Poco::File(path).createDirectory()) - storage.requireDirectoryMonitor(disk, dir_name); + Poco::File(path).createDirectory(); + auto & directory_monitor = storage.requireDirectoryMonitor(disk, dir_name); const auto & file_name = toString(storage.file_names_increment.get()) + ".bin"; const auto & block_file_path = path + file_name; @@ -632,6 +632,9 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: stream.writePrefix(); stream.write(block); stream.writeSuffix(); + + auto sleep_ms = context.getSettingsRef().distributed_directory_monitor_sleep_time_ms; + directory_monitor.scheduleAfter(sleep_ms.totalMilliseconds()); } if (link(first_file_tmp_path.data(), block_file_path.data())) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b453b73c4cb..14e7eea5c96 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -577,15 +577,20 @@ void StorageDistributed::createDirectoryMonitors(const std::string & disk) } -void StorageDistributed::requireDirectoryMonitor(const std::string & disk, const std::string & name) +StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(const std::string & disk, const std::string & name) { const std::string path(disk + relative_data_path + name); const std::string key(disk + name); std::lock_guard lock(cluster_nodes_mutex); auto & node_data = cluster_nodes_data[key]; - node_data.conneciton_pool = StorageDistributedDirectoryMonitor::createPool(name, *this); - node_data.directory_monitor = std::make_unique(*this, path, node_data.conneciton_pool, monitors_blocker); + if (!node_data.directory_monitor) + { + node_data.conneciton_pool = StorageDistributedDirectoryMonitor::createPool(name, *this); + node_data.directory_monitor = std::make_unique( + *this, path, node_data.conneciton_pool, monitors_blocker, global_context.getDistributedSchedulePool()); + } + return *node_data.directory_monitor; } size_t StorageDistributed::getShardCount() const diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 81c6b54a63e..2c5e321fc5f 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -109,7 +109,7 @@ public: /// create directory monitors for each existing subdirectory void createDirectoryMonitors(const std::string & disk); /// ensure directory monitor thread and connectoin pool creation by disk and subdirectory name - void requireDirectoryMonitor(const std::string & disk, const std::string & name); + StorageDistributedDirectoryMonitor & requireDirectoryMonitor(const std::string & disk, const std::string & name); void flushClusterNodesAllData(); From 6f76f2791d17ebec63494cf3728340b4b0e88aea Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 17 Apr 2020 10:40:12 +0300 Subject: [PATCH 389/743] Cover distributed_group_by_no_merge on GROUP BY injective function of sharding key --- ..._GROUP_BY_injective_sharding_key.reference | 50 +++++++++++++++++ ..._merge_GROUP_BY_injective_sharding_key.sql | 55 +++++++++++++++++++ 2 files changed, 105 insertions(+) create mode 100644 tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference create mode 100644 tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql diff --git a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference new file mode 100644 index 00000000000..5af7c8d8473 --- /dev/null +++ b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference @@ -0,0 +1,50 @@ +- +0 +1 +0 +1 +optimize_skip_unused_shards +0 +1 +0 +1 +GROUP BY number +1 0 +1 1 +1 0 +1 1 +GROUP BY number, 1 +1 0 +1 1 +1 0 +1 1 +GROUP BY 1 +4 0 +GROUP BY number ORDER BY number DESC +2 1 +2 0 +GROUP BY toString(number) +1 0 +1 1 +1 0 +1 1 +GROUP BY number%2 +2 0 +2 1 +countDistinct +2 +countDistinct GROUP BY number +1 +1 +1 +1 +DISTINCT +0 +1 +HAVING +LIMIT +2 0 +2 1 +LIMIT BY +2 0 +2 1 diff --git a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql new file mode 100644 index 00000000000..221871bd3e1 --- /dev/null +++ b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql @@ -0,0 +1,55 @@ +drop table if exists dist_01247; +drop table if exists data_01247; + +create table data_01247 as system.numbers engine=Memory(); +insert into data_01247 select * from system.numbers limit 2; +create table dist_01247 as data_01247 engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01247, number); +-- since data is not inserted via distributed it will have duplicates +-- (and this is how we ensure that this optimization will work) + +set max_distributed_connections=1; + +select '-'; +select * from dist_01247; + +select 'optimize_skip_unused_shards'; +set optimize_skip_unused_shards=1; +select * from dist_01247; + +select 'GROUP BY number'; +select count(), * from dist_01247 group by number; + +-- dumb, but should work, since "GROUP BY 1" optimized out +select 'GROUP BY number, 1'; +select count(), * from dist_01247 group by number, 1; +select 'GROUP BY 1'; +select count(), min(number) from dist_01247 group by 1; + +select 'GROUP BY number ORDER BY number DESC'; +select count(), * from dist_01247 group by number order by number desc; + +select 'GROUP BY toString(number)'; +select count(), * from dist_01247 group by toString(number); + +select 'GROUP BY number%2'; +select count(), any(number) from dist_01247 group by number%2; + +select 'countDistinct'; +select count(DISTINCT number) from dist_01247; + +select 'countDistinct GROUP BY number'; +select count(DISTINCT number) from dist_01247 group by number; + +select 'DISTINCT'; +select DISTINCT number from dist_01247; + +select 'HAVING'; +select count() cnt, * from dist_01247 group by number having cnt < 0; + +select 'LIMIT'; +select count(), * from dist_01247 group by number limit 1; +select count(), * from dist_01247 group by number limit 1 offset 1; + +select 'LIMIT BY'; +select count(), * from dist_01247 group by number limit 0 by number; +select count(), * from dist_01247 group by number limit 1 by number; From de4a72326426c52e396fce9847f8fca31afdaa90 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 17 Apr 2020 00:54:43 +0300 Subject: [PATCH 390/743] Auto distributed_group_by_no_merge on GROUP BY injective function of sharding key --- src/Storages/StorageDistributed.cpp | 50 ++++++++++++++++++++++++++++- src/Storages/StorageDistributed.h | 2 ++ 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b453b73c4cb..a245cf66449 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -73,6 +74,7 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int TOO_MANY_ROWS; extern const int UNABLE_TO_SKIP_UNUSED_SHARDS; + extern const int LOGICAL_ERROR; } namespace ActionLocks @@ -378,8 +380,54 @@ StoragePtr StorageDistributed::createWithOwnCluster( } -QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context &context, QueryProcessingStage::Enum to_stage, const ASTPtr &query_ptr) const +bool StorageDistributed::canForceGroupByNoMerge(const Context &context, const ASTPtr & query_ptr) const { + const auto & settings = context.getSettingsRef(); + + if (settings.distributed_group_by_no_merge) + return true; + if (!settings.optimize_skip_unused_shards) + return false; + if (!has_sharding_key) + return false; + + const auto & select = query_ptr->as(); + + if (select.orderBy()) + return false; + if (select.distinct) + return false; + + // This can use distributed_group_by_no_merge but in this case limit stage + // should be done later (which is not the case right now). + if (select.limitBy() || select.limitLength()) + return false; + + const ASTPtr group_by = select.groupBy(); + if (!group_by) + return false; + + // injective functions are optimized out in optimizeGroupBy() + // hence all we need to check is that column in GROUP BY matches sharding expression + auto & group_exprs = group_by->children; + if (!group_exprs.size()) + throw Exception("No ASTExpressionList in GROUP BY", ErrorCodes::LOGICAL_ERROR); + + auto id = group_exprs[0]->as(); + if (!id) + return false; + if (!sharding_key_expr->getSampleBlock().has(id->name)) + return false; + + LOG_DEBUG(log, "Force distributed_group_by_no_merge for GROUP BY " << backQuote(serializeAST(*group_by, true)) << " (injective)"); + return true; +} + +QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context &context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const +{ + if (canForceGroupByNoMerge(context, query_ptr)) + return QueryProcessingStage::Complete; + auto cluster = getOptimizedCluster(context, query_ptr); return getQueryProcessingStageImpl(context, to_stage, cluster); } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 81c6b54a63e..05b8b9fb55c 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -67,6 +67,8 @@ public: bool isRemote() const override { return true; } + /// Return true if distributed_group_by_no_merge may be applied. + bool canForceGroupByNoMerge(const Context &, const ASTPtr &) const; QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override; Pipes read( From e969f66acab6100abbee81c3ea30c21d79fc94c6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 19 Apr 2020 18:41:35 +0300 Subject: [PATCH 391/743] Try fix test. --- src/Interpreters/Context.cpp | 1 + src/Interpreters/Context.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 876229bbaec..98100994900 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -442,6 +442,7 @@ Context::Context(const Context &) = default; Context & Context::operator=(const Context &) = default; SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; +SharedContextHolder::SharedContextHolder() = default; SharedContextHolder::~SharedContextHolder() = default; SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) : shared(std::move(shared_context)) {} diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index eb9157bb326..ee808211d1e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -136,6 +136,7 @@ using IHostContextPtr = std::shared_ptr; struct SharedContextHolder { ~SharedContextHolder(); + SharedContextHolder(); SharedContextHolder(std::unique_ptr shared_context); SharedContextHolder(SharedContextHolder &&) noexcept; From 67b34872075f11a91f1b94b84b2015e85c5a7889 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 19 Apr 2020 18:46:12 +0300 Subject: [PATCH 392/743] Try fix build. --- src/Interpreters/Context.cpp | 1 + src/Interpreters/Context.h | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 98100994900..f9f405ea2cd 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -442,6 +442,7 @@ Context::Context(const Context &) = default; Context & Context::operator=(const Context &) = default; SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; +SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) = default; SharedContextHolder::SharedContextHolder() = default; SharedContextHolder::~SharedContextHolder() = default; SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ee808211d1e..d1b00b21f09 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -140,6 +140,8 @@ struct SharedContextHolder SharedContextHolder(std::unique_ptr shared_context); SharedContextHolder(SharedContextHolder &&) noexcept; + SharedContextHolder & operator=(SharedContextHolder &&); + ContextShared * get() const { return shared.get(); } void reset(); private: From 93d049fe64298973e75cb53b5c988440125d61b2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 19 Apr 2020 18:47:33 +0300 Subject: [PATCH 393/743] Allow auto distributed_group_by_no_merge for DISTINCT of sharding key --- src/Storages/StorageDistributed.cpp | 46 +++++++++++++------ ..._GROUP_BY_injective_sharding_key.reference | 2 + 2 files changed, 35 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a245cf66449..237e283d918 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -383,6 +383,7 @@ StoragePtr StorageDistributed::createWithOwnCluster( bool StorageDistributed::canForceGroupByNoMerge(const Context &context, const ASTPtr & query_ptr) const { const auto & settings = context.getSettingsRef(); + std::string reason; if (settings.distributed_group_by_no_merge) return true; @@ -395,8 +396,20 @@ bool StorageDistributed::canForceGroupByNoMerge(const Context &context, const AS if (select.orderBy()) return false; + if (select.distinct) - return false; + { + for (auto & expr : select.select()->children) + { + auto id = expr->as(); + if (!id) + return false; + if (!sharding_key_expr->getSampleBlock().has(id->name)) + return false; + } + + reason = "DISTINCT " + backQuote(serializeAST(*select.select(), true)); + } // This can use distributed_group_by_no_merge but in this case limit stage // should be done later (which is not the case right now). @@ -405,21 +418,28 @@ bool StorageDistributed::canForceGroupByNoMerge(const Context &context, const AS const ASTPtr group_by = select.groupBy(); if (!group_by) - return false; + { + if (!select.distinct) + return false; + } + else + { + // injective functions are optimized out in optimizeGroupBy() + // hence all we need to check is that column in GROUP BY matches sharding expression + auto & group_exprs = group_by->children; + if (!group_exprs.size()) + throw Exception("No ASTExpressionList in GROUP BY", ErrorCodes::LOGICAL_ERROR); - // injective functions are optimized out in optimizeGroupBy() - // hence all we need to check is that column in GROUP BY matches sharding expression - auto & group_exprs = group_by->children; - if (!group_exprs.size()) - throw Exception("No ASTExpressionList in GROUP BY", ErrorCodes::LOGICAL_ERROR); + auto id = group_exprs[0]->as(); + if (!id) + return false; + if (!sharding_key_expr->getSampleBlock().has(id->name)) + return false; - auto id = group_exprs[0]->as(); - if (!id) - return false; - if (!sharding_key_expr->getSampleBlock().has(id->name)) - return false; + reason = "GROUP BY " + backQuote(serializeAST(*group_by, true)); + } - LOG_DEBUG(log, "Force distributed_group_by_no_merge for GROUP BY " << backQuote(serializeAST(*group_by, true)) << " (injective)"); + LOG_DEBUG(log, "Force distributed_group_by_no_merge for " << reason << " (injective)"); return true; } diff --git a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference index 5af7c8d8473..1807c1b51cc 100644 --- a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference +++ b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference @@ -41,6 +41,8 @@ countDistinct GROUP BY number DISTINCT 0 1 +0 +1 HAVING LIMIT 2 0 From 681034f4a3f6a8d2d90466ea3666c9dbba3cf943 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 19 Apr 2020 20:51:37 +0300 Subject: [PATCH 394/743] Fix 01213_optimize_skip_unused_shards_DISTINCT after distributed_group_by_no_merge optimization --- .../01213_optimize_skip_unused_shards_DISTINCT.reference | 2 ++ .../0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql | 1 + 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.reference b/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.reference index 4ade9cd9c5d..eac8957dd47 100644 --- a/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.reference +++ b/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.reference @@ -6,3 +6,5 @@ optimize_skip_unused_shards optimize_skip_unused_shards lack of WHERE 0 1 +0 +1 diff --git a/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql b/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql index 5b45bea9046..67ba1cda870 100644 --- a/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql +++ b/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql @@ -12,6 +12,7 @@ SELECT DISTINCT id FROM dist_01213 WHERE id = 1 SETTINGS distributed_group_by_no SELECT 'optimize_skip_unused_shards'; SELECT DISTINCT id FROM dist_01213 WHERE id = 1 SETTINGS optimize_skip_unused_shards=1; -- check that querying all shards is ok +-- (there will be duplicates, since the INSERT was done via local table) SELECT 'optimize_skip_unused_shards lack of WHERE'; SELECT DISTINCT id FROM dist_01213 SETTINGS optimize_skip_unused_shards=1; From be1dec9239f1d659b1b2013846ff5078507f7c36 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 19 Apr 2020 21:06:21 +0300 Subject: [PATCH 395/743] Fix distributed_group_by_no_merge optimization for Distributed-over-Distributed --- src/Storages/StorageDistributed.cpp | 7 +++++-- src/Storages/StorageDistributed.h | 4 ++-- ...ge_GROUP_BY_injective_sharding_key.reference | 17 +++++++++++++++++ ...no_merge_GROUP_BY_injective_sharding_key.sql | 7 +++++++ 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 237e283d918..22f51e7a5b3 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -380,13 +380,16 @@ StoragePtr StorageDistributed::createWithOwnCluster( } -bool StorageDistributed::canForceGroupByNoMerge(const Context &context, const ASTPtr & query_ptr) const +bool StorageDistributed::canForceGroupByNoMerge(const Context &context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const { const auto & settings = context.getSettingsRef(); std::string reason; if (settings.distributed_group_by_no_merge) return true; + /// Distributed-over-Distributed (see getQueryProcessingStageImpl()) + if (to_stage == QueryProcessingStage::WithMergeableState) + return false; if (!settings.optimize_skip_unused_shards) return false; if (!has_sharding_key) @@ -445,7 +448,7 @@ bool StorageDistributed::canForceGroupByNoMerge(const Context &context, const AS QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context &context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const { - if (canForceGroupByNoMerge(context, query_ptr)) + if (canForceGroupByNoMerge(context, to_stage, query_ptr)) return QueryProcessingStage::Complete; auto cluster = getOptimizedCluster(context, query_ptr); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 05b8b9fb55c..bbebedba1cb 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -68,8 +68,8 @@ public: bool isRemote() const override { return true; } /// Return true if distributed_group_by_no_merge may be applied. - bool canForceGroupByNoMerge(const Context &, const ASTPtr &) const; - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override; + bool canForceGroupByNoMerge(const Context &, QueryProcessingStage::Enum to_stage, const ASTPtr &) const; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum to_stage, const ASTPtr &) const override; Pipes read( const Names & column_names, diff --git a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference index 1807c1b51cc..b8cf0042ed3 100644 --- a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference +++ b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.reference @@ -13,6 +13,11 @@ GROUP BY number 1 1 1 0 1 1 +GROUP BY number distributed_group_by_no_merge +1 0 +1 1 +1 0 +1 1 GROUP BY number, 1 1 0 1 1 @@ -50,3 +55,15 @@ LIMIT LIMIT BY 2 0 2 1 +GROUP BY (Distributed-over-Distributed) +4 0 +4 1 +GROUP BY (Distributed-over-Distributed) distributed_group_by_no_merge +1 0 +1 1 +1 0 +1 1 +1 0 +1 1 +1 0 +1 1 diff --git a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql index 221871bd3e1..56345e23094 100644 --- a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql +++ b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql @@ -18,6 +18,8 @@ select * from dist_01247; select 'GROUP BY number'; select count(), * from dist_01247 group by number; +select 'GROUP BY number distributed_group_by_no_merge'; +select count(), * from dist_01247 group by number settings distributed_group_by_no_merge=1; -- dumb, but should work, since "GROUP BY 1" optimized out select 'GROUP BY number, 1'; @@ -53,3 +55,8 @@ select count(), * from dist_01247 group by number limit 1 offset 1; select 'LIMIT BY'; select count(), * from dist_01247 group by number limit 0 by number; select count(), * from dist_01247 group by number limit 1 by number; + +select 'GROUP BY (Distributed-over-Distributed)'; +select count(), * from cluster(test_cluster_two_shards, currentDatabase(), dist_01247) group by number; +select 'GROUP BY (Distributed-over-Distributed) distributed_group_by_no_merge'; +select count(), * from cluster(test_cluster_two_shards, currentDatabase(), dist_01247) group by number settings distributed_group_by_no_merge=1; From cf6bbaeb979d9a01a1c181d9c520185a9849aedc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Feb 2020 04:09:49 +0300 Subject: [PATCH 396/743] Flush output format on each block in executeQuery --- src/Interpreters/executeQuery.cpp | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 68bebb83619..63c64a84308 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -689,19 +689,7 @@ void executeQuery( if (set_result_details) set_result_details(context.getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone()); - if (ast->as()) - { - /// For Watch query, flush data if block is empty (to send data to client). - auto flush_callback = [&out](const Block & block) - { - if (block.rows() == 0) - out->flush(); - }; - - copyData(*streams.in, *out, [](){ return false; }, std::move(flush_callback)); - } - else - copyData(*streams.in, *out); + copyData(*streams.in, *out, [](){ return false; }, [&out](const Block &) { out->flush(); }); } if (pipeline.initialized()) From 9dc70545c0c7032546fff1c184bb05e490585abb Mon Sep 17 00:00:00 2001 From: tai Date: Tue, 5 Nov 2019 17:46:17 +0800 Subject: [PATCH 397/743] Add TestCase for feature/ISSUE-5436 --- .../integration/test_custom_http/__init__.py | 0 .../test_custom_http/configs/custom_http.xml | 11 +++++++ .../integration/test_custom_http/test.py | 31 +++++++++++++++++++ 3 files changed, 42 insertions(+) create mode 100644 dbms/tests/integration/test_custom_http/__init__.py create mode 100644 dbms/tests/integration/test_custom_http/configs/custom_http.xml create mode 100644 dbms/tests/integration/test_custom_http/test.py diff --git a/dbms/tests/integration/test_custom_http/__init__.py b/dbms/tests/integration/test_custom_http/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http.xml b/dbms/tests/integration/test_custom_http/configs/custom_http.xml new file mode 100644 index 00000000000..22d570b7bca --- /dev/null +++ b/dbms/tests/integration/test_custom_http/configs/custom_http.xml @@ -0,0 +1,11 @@ + + 8123 + + + /${database}/a/${id}/${table} + INSERT INTO ${database:ASTIdentifier}.${table:ASTIdentifier}(id) VALUES + SELECT * FROM ${database:ASTIdenfier}.t + SELECT * FROM a.${table:ASTIdenfier} WHERE id={id:UInt8} + + + diff --git a/dbms/tests/integration/test_custom_http/test.py b/dbms/tests/integration/test_custom_http/test.py new file mode 100644 index 00000000000..d8f79e64e3d --- /dev/null +++ b/dbms/tests/integration/test_custom_http/test.py @@ -0,0 +1,31 @@ +import pytest +import requests + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', main_configs=['configs/custom_http.xml']) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + node.query(''' +CREATE DATABASE `test`; + +CREATE TABLE `test`.`test_custom_http` (`id` UInt8) Engine=Memory; + ''') + + yield cluster + finally: + cluster.shutdown() + +def test(started_cluster): + node_ip = cluster.get_instance_ip(node) + url = 'http://%s:8123/test/a/1/test_custom_http' % node_ip + data="(1)" + params = {'id':1} + response = requests.post(url, params = params, data = data) + + assert response.text == '\n1\n1\n' From 159ba24fcab37981197f50368122842b44227459 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 1 Nov 2019 18:03:21 +0800 Subject: [PATCH 398/743] ISSUES-5436 support custom http --- .../CustomHTTP/HTTPMatchExecutor.cpp | 259 ++++++++++++++++++ .../CustomHTTP/HTTPMatchExecutor.h | 55 ++++ .../CustomHTTP/HTTPMatchExecutorDefault.h | 46 ++++ .../CustomHTTP/HTTPStreamsWithInput.cpp | 73 +++++ .../CustomHTTP/HTTPStreamsWithInput.h | 29 ++ .../CustomHTTP/HTTPStreamsWithOutput.cpp | 217 +++++++++++++++ .../CustomHTTP/HTTPStreamsWithOutput.h | 48 ++++ programs/server/HTTPHandler.cpp | 22 +- programs/server/HTTPHandler.h | 6 +- src/IO/WriteBufferFromHTTPServerResponse.h | 5 + src/Interpreters/Context.cpp | 6 + src/Interpreters/Context.h | 2 + 12 files changed, 755 insertions(+), 13 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp new file mode 100644 index 00000000000..88890755955 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp @@ -0,0 +1,259 @@ +#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 SYNTAX_ERROR; + extern const int LOGICAL_ERROR; + extern const int REQUIRED_PASSWORD; + extern const int INVALID_SESSION_TIMEOUT; + extern const int UNKNOWN_COMPRESSION_METHOD; +} + + +namespace +{ + duration parseSessionTimeout(const HTMLForm & params, size_t default_session_timeout, size_t max_session_timeout) + { + size_t session_timeout = default_session_timeout; + + if (params.has("session_timeout")) + { + std::string session_timeout_str = params.get("session_timeout"); + + ReadBufferFromString buf(session_timeout_str); + if (!tryReadIntText(session_timeout, buf) || !buf.eof()) + throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); + + if (session_timeout > max_session_timeout) + throw Exception( + "Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) + + ". Maximum session timeout could be modified in configuration file.", ErrorCodes::INVALID_SESSION_TIMEOUT); + } + + return std::chrono::seconds(session_timeout); + } +} + + +void HTTPMatchExecutor::execute(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & params, HTTPStreamsWithOutput & used_output) const +{ + authentication(context, request, params); + + std::shared_ptr session_context; + String session_id = params.get("session_id", ""); + duration session_timeout = parseSessionTimeout(params, 1800, 3600); + + SCOPE_EXIT({ detachSessionContext(session_context, session_id, session_timeout); }); + session_context = attachSessionContext(context, params, session_id, session_timeout); + + initClientInfo(context, request); + used_output.attachRequestAndResponse(context, request, response, params, /* TODO: keep_alive_time_out */ 0); + + HTTPStreamsWithInput used_input(request, params); + collectParamsAndApplySettings(request, params, context); + + Settings & query_settings = context.getSettingsRef(); + used_input.attachSettings(context, query_settings, request); + used_output.attachSettings(context, query_settings, request); + + String execute_query_string = getExecuteQuery(params); + ReadBufferPtr query_in_buffer = std::make_shared(execute_query_string); + + ReadBufferPtr in = query_in_buffer; + if (!needParsePostBody(request, params) || !context.getExternalTables().empty()) + in = std::make_shared(*query_in_buffer, *used_input.in_maybe_internal_compressed); + + executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); }); + + used_output.finalize(); +} + +void HTTPMatchExecutor::initClientInfo(Context & context, HTTPServerRequest & request) const +{ + + ClientInfo & client_info = context.getClientInfo(); + client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + client_info.interface = ClientInfo::Interface::HTTP; + + /// Query sent through HTTP interface is initial. + client_info.initial_user = client_info.current_user; + client_info.initial_query_id = client_info.current_query_id; + client_info.initial_address = client_info.current_address; + + ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + http_method = ClientInfo::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) + http_method = ClientInfo::HTTPMethod::POST; + + client_info.http_method = http_method; + client_info.http_user_agent = request.get("User-Agent", ""); +} + +void HTTPMatchExecutor::authentication(Context & context, HTTPServerRequest & request, HTMLForm & params) const +{ + auto user = request.get("X-ClickHouse-User", ""); + auto password = request.get("X-ClickHouse-Key", ""); + auto quota_key = request.get("X-ClickHouse-Quota", ""); + + if (user.empty() && password.empty() && quota_key.empty()) + { + /// User name and password can be passed using query parameters + /// or using HTTP Basic auth (both methods are insecure). + if (request.hasCredentials()) + { + Poco::Net::HTTPBasicCredentials credentials(request); + + user = credentials.getUsername(); + password = credentials.getPassword(); + } + else + { + user = params.get("user", "default"); + password = params.get("password", ""); + } + + quota_key = params.get("quota_key", ""); + } + else + { + /// It is prohibited to mix different authorization schemes. + if (request.hasCredentials() + || params.has("user") + || params.has("password") + || params.has("quota_key")) + { + throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); + } + } + + std::string query_id = params.get("query_id", ""); + context.setUser(user, password, request.clientAddress(), quota_key); + context.setCurrentQueryId(query_id); +} + +std::shared_ptr HTTPMatchExecutor::attachSessionContext( + Context & context, HTMLForm & params, const String & session_id, const duration & session_timeout) const +{ + if (!session_id.empty()) + { + std::string session_check = params.get("session_check", ""); + auto session_context = context.acquireSession(session_id, session_timeout, session_check == "1"); + + context = *session_context; + context.setSessionContext(*session_context); + return session_context; + } + return {}; +} + +void HTTPMatchExecutor::detachSessionContext(std::shared_ptr & session_context, const String & session_id, const duration & session_timeout) const +{ + if (session_context) + session_context->releaseSession(session_id, session_timeout); +} + +void HTTPMatchExecutor::collectParamsAndApplySettings(HTTPServerRequest & request, HTMLForm & params, Context & context) const +{ + static const NameSet reserved_param_names{ + "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "buffer_size", "wait_end_of_query", + "session_id", "session_timeout", "session_check"}; + + Names reserved_param_suffixes; + + auto param_could_be_skipped = [&] (const String & name) + { + if (reserved_param_names.count(name)) + return true; + + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } + + return false; + }; + + /// Settings can be overridden in the query. + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + + /// In theory if initially readonly = 0, the client can change any setting and then set readonly + /// to some other value. + /// Only readonly queries are allowed for HTTP GET requests. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + { + Settings & settings = context.getSettingsRef(); + + if (settings.readonly == 0) + settings.readonly = 2; + } + + bool has_multipart = startsWith(request.getContentType().data(), "multipart/form-data"); + + if (has_multipart || needParsePostBody(request, params)) + { + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); + + if (has_multipart) + { + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + reserved_param_suffixes.reserve(3); + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + reserved_param_suffixes.emplace_back("_format"); + reserved_param_suffixes.emplace_back("_types"); + reserved_param_suffixes.emplace_back("_structure"); + } + } + + SettingsChanges settings_changes; + for (const auto & [key, value] : params) + { + if (key == "database") + context.setCurrentDatabase(value); + else if (key == "default_format") + context.setDefaultFormat(value); + else if (!param_could_be_skipped(key) && !acceptQueryParam(context, key, value)) + settings_changes.push_back({key, value}); /// All other query parameters are treated as settings. + } + + /// For external data we also want settings + context.checkSettingsConstraints(settings_changes); + context.applySettingsChanges(settings_changes); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h new file mode 100644 index 00000000000..425605aec07 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; +class HTTPMatchExecutor; +struct HTTPStreamsWithInput; +struct HTTPStreamsWithOutput; + +using duration = std::chrono::steady_clock::duration; +using HTTPMatchExecutorPtr = std::shared_ptr; + +class HTTPMatchExecutor +{ +public: + using HTTPServerRequest = Poco::Net::HTTPServerRequest; + using HTTPServerResponse = Poco::Net::HTTPServerResponse; + + bool match(HTTPServerRequest & request, HTMLForm & params) const { return matchImpl(request, params); }; + + void execute(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & params, HTTPStreamsWithOutput & used_output) const; + + virtual ~HTTPMatchExecutor() = default; +protected: + + virtual bool matchImpl(HTTPServerRequest & request, HTMLForm & params) const = 0; + + virtual String getExecuteQuery(HTMLForm & params) const = 0; + + virtual bool needParsePostBody(HTTPServerRequest & request, HTMLForm & params) const = 0; + + virtual bool acceptQueryParam(Context & context, const String & key, const String & value) const = 0; + + void initClientInfo(Context & context, HTTPServerRequest & request) const; + + void authentication(Context & context, HTTPServerRequest & request, HTMLForm & params) const; + + void detachSessionContext(std::shared_ptr & context, const String & session_id, const duration & session_timeout) const; + + std::shared_ptr attachSessionContext(Context & context, HTMLForm & params, const String & session_id, const duration & session_timeout) const; + + void collectParamsAndApplySettings(HTTPServerRequest & request, HTMLForm & params, Context & context) const; + +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h new file mode 100644 index 00000000000..54ea20120f5 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h @@ -0,0 +1,46 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class HTTPMatchExecutorDefault : public HTTPMatchExecutor +{ +protected: + + bool matchImpl(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } + + bool needParsePostBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return false; } + + String getExecuteQuery(HTMLForm & params) const override + { + String execute_query; + for (const auto & [key, value] : params) + { + if (key == "query") + execute_query += value; + } + + return execute_query; + } + + bool acceptQueryParam(Context &context, const String &key, const String &value) const override + { + if (startsWith(key, "param_")) + { + /// Save name and values of substitution in dictionary. + context.setQueryParameter(key.substr(strlen("param_")), value); + return true; + } + + return key == "query"; + } + +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp new file mode 100644 index 00000000000..6ff17217f24 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp @@ -0,0 +1,73 @@ +#include + +#include +#include +#include +#include "HTTPStreamsWithInput.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_COMPRESSION_METHOD; +} + +HTTPStreamsWithInput::HTTPStreamsWithInput(HTTPServerRequest & request, HTMLForm & from) + : in(createRawInBuffer(request)) + , in_maybe_compressed(createCompressedBuffer(request, in)) + , in_maybe_internal_compressed(createInternalCompressedBuffer(from, in_maybe_compressed)) +{ +} + +ReadBufferPtr HTTPStreamsWithInput::createRawInBuffer(HTTPServerRequest & request) const +{ + return std::make_unique(request.stream()); +} + +ReadBufferPtr HTTPStreamsWithInput::createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const +{ + /// Request body can be compressed using algorithm specified in the Content-Encoding header. + String http_compressed_method = request.get("Content-Encoding", ""); + + if (!http_compressed_method.empty()) + { + if (http_compressed_method == "gzip") + return std::make_shared(*raw_buffer, CompressionMethod::Gzip); + else if (http_compressed_method == "deflate") + return std::make_shared(*raw_buffer, CompressionMethod::Zlib); +#if USE_BROTLI + else if (http_compressed_method == "br") + return std::make_shared(*raw_buffer); +#endif + else + throw Exception("Unknown Content-Encoding of HTTP request: " + http_compressed_method, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); + } + + return raw_buffer; +} + +ReadBufferPtr HTTPStreamsWithInput::createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const +{ + /// The data can also be compressed using incompatible internal algorithm. This is indicated by + /// 'decompress' query parameter. + std::unique_ptr in_post_maybe_compressed; + if (params.getParsed("decompress", false)) + return std::make_unique(*http_maybe_encoding_buffer); + + return http_maybe_encoding_buffer; +} + +void HTTPStreamsWithInput::attachSettings(Context & /*context*/, Settings & settings, HTTPServerRequest & /*request*/) +{ + /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, + /// checksums of client data compressed with internal algorithm are not checked. + if (settings.http_native_compression_disable_checksumming_on_decompress) + { + if(CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) + compressed_buffer->disableChecksumming(); + } +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h new file mode 100644 index 00000000000..2c3bd1c2a35 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +using HTTPServerRequest = Poco::Net::HTTPServerRequest; + +struct HTTPStreamsWithInput +{ + std::shared_ptr in; + std::shared_ptr in_maybe_compressed; + std::shared_ptr in_maybe_internal_compressed; + + HTTPStreamsWithInput(HTTPServerRequest & request, HTMLForm & from); + + void attachSettings(Context & context, Settings & settings, HTTPServerRequest & request); + + ReadBufferPtr createRawInBuffer(HTTPServerRequest & request) const; + ReadBufferPtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const; + ReadBufferPtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const; +}; + +} \ No newline at end of file diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp new file mode 100644 index 00000000000..35426315eb3 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp @@ -0,0 +1,217 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + inline void listeningProgress(Context & context, ProgressCallback listener) + { + auto prev = context.getProgressCallback(); + context.setProgressCallback([prev, listener] (const Progress & progress) + { + if (prev) + prev(progress); + + listener(progress); + }); + } + + inline ProgressCallback cancelListener(Context & context, Poco::Net::StreamSocket & socket) + { + /// Assume that at the point this method is called no one is reading data from the socket any more. + /// True for read-only queries. + return [&context, &socket](const Progress &) + { + try + { + char b; + int status = socket.receiveBytes(&b, 1, MSG_DONTWAIT | MSG_PEEK); + if (status == 0) + context.killCurrentQuery(); + } + catch (Poco::TimeoutException &) + { + } + catch (...) + { + context.killCurrentQuery(); + } + }; + } +} + +void HTTPStreamsWithOutput::attachSettings(Context & context, Settings & settings, HTTPServerRequest & request) +{ + /// HTTP response compression is turned on only if the client signalled that they support it + /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. + out->setCompression(out->getCompression() && settings.enable_http_compression); + if (out->getCompression()) + out->setCompressionLevel(settings.http_zlib_compression_level); + + out->setSendProgressInterval(settings.http_headers_progress_interval_ms); + + /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed + /// Origin header. + out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); + + /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. + if (settings.send_progress_in_http_headers) + listeningProgress(context, [this] (const Progress & progress) { out->onProgress(progress); }); + + if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) + { + Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); + + listeningProgress(context, cancelListener(context, socket)); + } +} + +void HTTPStreamsWithOutput::attachRequestAndResponse( + Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) +{ + out = createEndpoint(request, response, keep_alive_timeout); + out_maybe_compressed = createMaybeCompressionEndpoint(form, out); + out_maybe_delayed_and_compressed = createMaybeDelayedAndCompressionEndpoint(context, form, out_maybe_compressed); +} + +std::shared_ptr HTTPStreamsWithOutput::createEndpoint( + HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout) +{ + /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). + String http_response_compression_methods = request.get("Accept-Encoding", ""); + + if (!http_response_compression_methods.empty()) + { + /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. + /// NOTE parsing of the list of methods is slightly incorrect. + if (std::string::npos != http_response_compression_methods.find("gzip")) + return std::make_shared( + request, response, keep_alive_timeout, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + else if (std::string::npos != http_response_compression_methods.find("deflate")) + return std::make_shared( + request, response, keep_alive_timeout, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); +#if USE_BROTLI + else if (http_response_compression_methods == "br") + return std::make_shared( + request, response, keep_alive_timeout, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); +#endif + } + + return std::make_shared( + request, response, keep_alive_timeout, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); +} + +WriteBufferPtr HTTPStreamsWithOutput::createMaybeCompressionEndpoint(HTMLForm & form, std::shared_ptr & endpoint) +{ + /// Client can pass a 'compress' flag in the query string. In this case the query result is + /// compressed using internal algorithm. This is not reflected in HTTP headers. + bool internal_compression = form.getParsed("compress", false); + return internal_compression ? std::make_shared(*endpoint) : WriteBufferPtr(endpoint); +} + +WriteBufferPtr HTTPStreamsWithOutput::createMaybeDelayedAndCompressionEndpoint(Context & context, HTMLForm & form, WriteBufferPtr & endpoint) +{ + /// If it is specified, the whole result will be buffered. + /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. + bool buffer_until_eof = form.getParsed("wait_end_of_query", false); + + /// At least, we should postpone sending of first buffer_size result bytes + size_t buffer_size_total = std::max(form.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); + + size_t buffer_size_memory = (buffer_size_total > DBMS_DEFAULT_BUFFER_SIZE) ? buffer_size_total : 0; + + if (buffer_size_memory > 0 || buffer_until_eof) + { + CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; + CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; + + if (buffer_size_memory > 0) + cascade_buffer1.emplace_back(std::make_shared(buffer_size_memory)); + + if (buffer_until_eof) + { + std::string tmp_path_template = context.getTemporaryPath() + "http_buffers/"; + + auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) + { + return WriteBufferFromTemporaryFile::create(tmp_path_template); + }; + + cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); + } + else + { + auto push_memory_buffer_and_continue = [next_buffer = endpoint] (const WriteBufferPtr & prev_buf) + { + auto prev_memory_buffer = typeid_cast(prev_buf.get()); + if (!prev_memory_buffer) + throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); + + auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); + copyData(*rdbuf , *next_buffer); + + return next_buffer; + }; + + cascade_buffer2.emplace_back(push_memory_buffer_and_continue); + } + + return std::make_shared(std::move(cascade_buffer1), std::move(cascade_buffer2)); + } + + return endpoint; +} + +void HTTPStreamsWithOutput::finalize() const +{ + if (out_maybe_delayed_and_compressed != out_maybe_compressed) + { + /// TODO: set Content-Length if possible + std::vector write_buffers; + std::vector read_buffers; + std::vector read_buffers_raw_ptr; + + auto cascade_buffer = typeid_cast(out_maybe_delayed_and_compressed.get()); + if (!cascade_buffer) + throw Exception("Expected CascadeWriteBuffer", ErrorCodes::LOGICAL_ERROR); + + cascade_buffer->getResultBuffers(write_buffers); + + if (write_buffers.empty()) + throw Exception("At least one buffer is expected to overwrite result into HTTP response", ErrorCodes::LOGICAL_ERROR); + + for (auto & write_buf : write_buffers) + { + IReadableWriteBuffer * write_buf_concrete; + ReadBufferPtr reread_buf; + + if (write_buf + && (write_buf_concrete = dynamic_cast(write_buf.get())) + && (reread_buf = write_buf_concrete->tryGetReadBuffer())) + { + read_buffers.emplace_back(reread_buf); + read_buffers_raw_ptr.emplace_back(reread_buf.get()); + } + } + + ConcatReadBuffer concat_read_buffer(read_buffers_raw_ptr); + copyData(concat_read_buffer, *out_maybe_compressed); + } + + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to + /// the client. + out->finalize(); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h new file mode 100644 index 00000000000..65d87b6744a --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +using HTTPServerRequest = Poco::Net::HTTPServerRequest; +using HTTPServerResponse = Poco::Net::HTTPServerResponse; + + +/* Raw data + * ↓ + * CascadeWriteBuffer out_maybe_delayed_and_compressed (optional) + * ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults) + * CompressedWriteBuffer out_maybe_compressed (optional) + * ↓ + * WriteBufferFromHTTPServerResponse out + */ +struct HTTPStreamsWithOutput +{ + using HTTPResponseBufferPtr = std::shared_ptr; + + HTTPResponseBufferPtr out; + /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. + std::shared_ptr out_maybe_compressed; + /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. + std::shared_ptr out_maybe_delayed_and_compressed; + + void finalize() const; + + WriteBufferPtr createMaybeDelayedAndCompressionEndpoint(Context & context, HTMLForm & form, WriteBufferPtr & endpoint); + + WriteBufferPtr createMaybeCompressionEndpoint(HTMLForm & form, std::shared_ptr & endpoint); + + HTTPResponseBufferPtr createEndpoint(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout); + + void attachSettings(Context & context, Settings & settings, HTTPServerRequest & request); + + void attachRequestAndResponse(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); +}; + +} diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 77236210741..99afb319ed5 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -216,7 +216,7 @@ void HTTPHandler::processQuery( Poco::Net::HTTPServerRequest & request, HTMLForm & params, Poco::Net::HTTPServerResponse & response, - Output & used_output) + HTTPStreamsWithOutput & used_output) { Context context = server.context(); @@ -224,6 +224,12 @@ void HTTPHandler::processQuery( LOG_TRACE(log, "Request URI: " << request.getURI()); + if (context.getSettingsRef().allow_experimental_custom_http) + { + context.getHTTPMatchExecutor()->execute(context, request, response, params, used_output); + return; + } + std::istream & istr = request.stream(); /// Part of the query can be passed in the 'query' parameter and the rest in the request body @@ -605,20 +611,14 @@ void HTTPHandler::processQuery( } ); - if (used_output.hasDelayed()) - { - /// TODO: set Content-Length if possible - pushDelayedResults(used_output); - } - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to /// the client. - used_output.out->finalize(); + used_output.finalize(); } void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - Output & used_output) + HTTPStreamsWithOutput & used_output) { try { @@ -655,7 +655,7 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ else if (used_output.out_maybe_compressed) { /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references - if (used_output.hasDelayed()) + if (used_output.out_maybe_delayed_and_compressed != used_output.out_maybe_compressed) used_output.out_maybe_delayed_and_compressed.reset(); /// Send the error message into already used (and possibly compressed) stream. @@ -691,7 +691,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne setThreadName("HTTPHandler"); ThreadStatus thread_status; - Output used_output; + HTTPStreamsWithOutput used_output; /// In case of exception, send stack trace to client. bool with_stacktrace = false; diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 681a3cce932..603278e11aa 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -7,6 +7,8 @@ #include #include +#include + namespace CurrentMetrics { @@ -68,14 +70,14 @@ private: Poco::Net::HTTPServerRequest & request, HTMLForm & params, Poco::Net::HTTPServerResponse & response, - Output & used_output); + HTTPStreamsWithOutput & used_output); void trySendExceptionToClient( const std::string & s, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - Output & used_output); + HTTPStreamsWithOutput & used_output); static void pushDelayedResults(Output & used_output); }; diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index ffa36c11c5b..c10288fff9f 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -113,6 +113,11 @@ public: compress = enable_compression; } + bool getCompression() + { + return compress; + } + /// Set compression level if the compression is turned on. /// The setting has any effect only if HTTP headers haven't been sent yet. void setCompressionLevel(int level) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 647c3fb8020..03dd311f31b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -44,6 +44,7 @@ #include #include #include +#include #include #include #include @@ -348,6 +349,7 @@ struct ContextShared std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config + mutable std::mutex match_executor_mutex; /// Guards match executor and their config #if USE_EMBEDDED_COMPILER std::shared_ptr compiled_expression_cache; @@ -1543,6 +1545,10 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptrclusters->setCluster(cluster_name, cluster); } +HTTPMatchExecutorPtr Context::getHTTPMatchExecutor() +{ + return std::make_shared(); +} void Context::initializeSystemLogs() { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1f81cdbc58b..03859b03348 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -490,6 +490,8 @@ public: Compiler & getCompiler(); + HTTPMatchExecutorPtr getHTTPMatchExecutor(); + /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From 183eb82b7caa012849e2c09f5ffa36c1e5f91d66 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 5 Nov 2019 02:36:28 +0800 Subject: [PATCH 399/743] ISSUES-5436 support custom http [part 2] --- .../Interpreters/CustomHTTP/CustomExecutor.h | 41 ++ .../CustomHTTP/CustomExecutorDefault.h | 50 ++ .../CustomHTTP/ExtractorClientInfo.h | 38 ++ .../CustomHTTP/ExtractorContextChange.h | 98 ++++ ...eamsWithInput.cpp => HTTPInputStreams.cpp} | 33 +- ...PStreamsWithInput.h => HTTPInputStreams.h} | 6 +- .../CustomHTTP/HTTPMatchExecutor.cpp | 259 ---------- .../CustomHTTP/HTTPMatchExecutor.h | 55 -- .../CustomHTTP/HTTPMatchExecutorDefault.h | 46 -- ...msWithOutput.cpp => HTTPOutputStreams.cpp} | 35 +- ...treamsWithOutput.h => HTTPOutputStreams.h} | 16 +- programs/server/HTTPHandler.cpp | 476 +++--------------- programs/server/HTTPHandler.h | 48 +- src/Common/HTMLForm.h | 6 + src/Interpreters/Context.cpp | 12 +- src/Interpreters/Context.h | 2 +- 16 files changed, 359 insertions(+), 862 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutor.h create mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h create mode 100644 dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h create mode 100644 dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h rename dbms/src/Interpreters/CustomHTTP/{HTTPStreamsWithInput.cpp => HTTPInputStreams.cpp} (72%) rename dbms/src/Interpreters/CustomHTTP/{HTTPStreamsWithInput.h => HTTPInputStreams.h} (78%) delete mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h rename dbms/src/Interpreters/CustomHTTP/{HTTPStreamsWithOutput.cpp => HTTPOutputStreams.cpp} (87%) rename dbms/src/Interpreters/CustomHTTP/{HTTPStreamsWithOutput.h => HTTPOutputStreams.h} (61%) diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h new file mode 100644 index 00000000000..0778256b297 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; +class CustomExecutor; +struct HTTPInputStreams; +struct HTTPOutputStreams; + +using duration = std::chrono::steady_clock::duration; +using HTTPMatchExecutorPtr = std::shared_ptr; + +class CustomExecutor +{ +public: + using HTTPServerRequest = Poco::Net::HTTPServerRequest; + using HTTPServerResponse = Poco::Net::HTTPServerResponse; + + virtual ~CustomExecutor() = default; + + virtual bool isQueryParam(const String & param_name) const = 0; + + virtual bool match(HTTPServerRequest & request, HTMLForm & params) const = 0; + + virtual bool canBeParseRequestBody(HTTPServerRequest & request, HTMLForm & params) = 0; + + using QueryExecutor = std::function; + using QueryExecutors = std::vector; + virtual QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const = 0; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h new file mode 100644 index 00000000000..218cb413222 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class CustomExecutorDefault : public CustomExecutor +{ +public: + bool match(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } + + bool canBeParseRequestBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) override { return false; } + + bool isQueryParam(const String & param_name) const override + { + return param_name == "query" || startsWith(param_name, "param_"); + } + + QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const override + { + ReadBufferPtr in = prepareAndGetQueryInput(context, request, params, input_streams); + + return {[&, shared_in = in](HTTPOutputStreams & output, HTTPServerResponse & response) + { + executeQuery( + *shared_in, *output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); + }}; + } + +private: + ReadBufferPtr prepareAndGetQueryInput(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const + { + for (const auto & [key, value] : params) + { + + } + } +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h b/dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h new file mode 100644 index 00000000000..76840f3d682 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ExtractorClientInfo +{ +public: + ExtractorClientInfo(ClientInfo & info_) : client_info(info_) {} + + void extract(Poco::Net::HTTPServerRequest & request) + { + client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + client_info.interface = ClientInfo::Interface::HTTP; + + /// Query sent through HTTP interface is initial. + client_info.initial_user = client_info.current_user; + client_info.initial_query_id = client_info.current_query_id; + client_info.initial_address = client_info.current_address; + + ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + http_method = ClientInfo::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) + http_method = ClientInfo::HTTPMethod::POST; + + client_info.http_method = http_method; + client_info.http_user_agent = request.get("User-Agent", ""); + } + +private: + ClientInfo & client_info; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h new file mode 100644 index 00000000000..a228f75391f --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h @@ -0,0 +1,98 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class ExtractorContextChange +{ +public: + ExtractorContextChange(Context & context_, const HTTPMatchExecutorPtr & executor_) : context(context_), executor(executor_) {} + + void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) + { + Names reserved_param_suffixes; + static const NameSet reserved_param_names{ + "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "buffer_size", "wait_end_of_query", + "session_id", "session_timeout", "session_check"}; + + auto param_could_be_skipped = [&] (const String & name) + { + if (reserved_param_names.count(name)) + return true; + + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } + + return false; + }; + + /// Settings can be overridden in the query. + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + + /// In theory if initially readonly = 0, the client can change any setting and then set readonly + /// to some other value. + /// Only readonly queries are allowed for HTTP GET requests. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + { + Settings & settings = context.getSettingsRef(); + + if (settings.readonly == 0) + settings.readonly = 2; + } + + bool has_multipart = startsWith(request.getContentType().data(), "multipart/form-data"); + + if (has_multipart || executor->canBeParseRequestBody(request, params)) + { + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); + + if (has_multipart) + { + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + reserved_param_suffixes.reserve(3); + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + reserved_param_suffixes.emplace_back("_format"); + reserved_param_suffixes.emplace_back("_types"); + reserved_param_suffixes.emplace_back("_structure"); + } + } + + SettingsChanges settings_changes; + for (const auto & [key, value] : params) + { + if (key == "database") + context.setCurrentDatabase(value); + else if (key == "default_format") + context.setDefaultFormat(value); + else if (!param_could_be_skipped(key) && !executor->isQueryParam(key)) + settings_changes.push_back({key, value}); /// All other query parameters are treated as settings. + } + + /// For external data we also want settings + context.checkSettingsConstraints(settings_changes); + context.applySettingsChanges(settings_changes); + } + +private: + Context & context; + const HTTPMatchExecutorPtr & executor; + +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp similarity index 72% rename from dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp rename to dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp index 6ff17217f24..e6985621b00 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp @@ -1,9 +1,10 @@ -#include +#include + +#include #include #include #include -#include "HTTPStreamsWithInput.h" namespace DB @@ -14,19 +15,26 @@ namespace ErrorCodes extern const int UNKNOWN_COMPRESSION_METHOD; } -HTTPStreamsWithInput::HTTPStreamsWithInput(HTTPServerRequest & request, HTMLForm & from) +HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from) : in(createRawInBuffer(request)) , in_maybe_compressed(createCompressedBuffer(request, in)) , in_maybe_internal_compressed(createInternalCompressedBuffer(from, in_maybe_compressed)) { + /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, + /// checksums of client data compressed with internal algorithm are not checked. + if (context.getSettingsRef().http_native_compression_disable_checksumming_on_decompress) + { + if(CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) + compressed_buffer->disableChecksumming(); + } } -ReadBufferPtr HTTPStreamsWithInput::createRawInBuffer(HTTPServerRequest & request) const +ReadBufferPtr HTTPInputStreams::createRawInBuffer(HTTPServerRequest & request) const { return std::make_unique(request.stream()); } -ReadBufferPtr HTTPStreamsWithInput::createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const +ReadBufferPtr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const { /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_compressed_method = request.get("Content-Encoding", ""); @@ -38,7 +46,7 @@ ReadBufferPtr HTTPStreamsWithInput::createCompressedBuffer(HTTPServerRequest & r else if (http_compressed_method == "deflate") return std::make_shared(*raw_buffer, CompressionMethod::Zlib); #if USE_BROTLI - else if (http_compressed_method == "br") + else if (http_compressed_method == "br") return std::make_shared(*raw_buffer); #endif else @@ -48,7 +56,7 @@ ReadBufferPtr HTTPStreamsWithInput::createCompressedBuffer(HTTPServerRequest & r return raw_buffer; } -ReadBufferPtr HTTPStreamsWithInput::createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const +ReadBufferPtr HTTPInputStreams::createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const { /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. @@ -59,15 +67,4 @@ ReadBufferPtr HTTPStreamsWithInput::createInternalCompressedBuffer(HTMLForm & pa return http_maybe_encoding_buffer; } -void HTTPStreamsWithInput::attachSettings(Context & /*context*/, Settings & settings, HTTPServerRequest & /*request*/) -{ - /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, - /// checksums of client data compressed with internal algorithm are not checked. - if (settings.http_native_compression_disable_checksumming_on_decompress) - { - if(CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) - compressed_buffer->disableChecksumming(); - } -} - } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h similarity index 78% rename from dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h rename to dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h index 2c3bd1c2a35..88f0438398d 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h @@ -11,15 +11,13 @@ namespace DB using HTTPServerRequest = Poco::Net::HTTPServerRequest; -struct HTTPStreamsWithInput +struct HTTPInputStreams { std::shared_ptr in; std::shared_ptr in_maybe_compressed; std::shared_ptr in_maybe_internal_compressed; - HTTPStreamsWithInput(HTTPServerRequest & request, HTMLForm & from); - - void attachSettings(Context & context, Settings & settings, HTTPServerRequest & request); + HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); ReadBufferPtr createRawInBuffer(HTTPServerRequest & request) const; ReadBufferPtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const; diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp deleted file mode 100644 index 88890755955..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp +++ /dev/null @@ -1,259 +0,0 @@ -#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 SYNTAX_ERROR; - extern const int LOGICAL_ERROR; - extern const int REQUIRED_PASSWORD; - extern const int INVALID_SESSION_TIMEOUT; - extern const int UNKNOWN_COMPRESSION_METHOD; -} - - -namespace -{ - duration parseSessionTimeout(const HTMLForm & params, size_t default_session_timeout, size_t max_session_timeout) - { - size_t session_timeout = default_session_timeout; - - if (params.has("session_timeout")) - { - std::string session_timeout_str = params.get("session_timeout"); - - ReadBufferFromString buf(session_timeout_str); - if (!tryReadIntText(session_timeout, buf) || !buf.eof()) - throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); - - if (session_timeout > max_session_timeout) - throw Exception( - "Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) + - ". Maximum session timeout could be modified in configuration file.", ErrorCodes::INVALID_SESSION_TIMEOUT); - } - - return std::chrono::seconds(session_timeout); - } -} - - -void HTTPMatchExecutor::execute(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & params, HTTPStreamsWithOutput & used_output) const -{ - authentication(context, request, params); - - std::shared_ptr session_context; - String session_id = params.get("session_id", ""); - duration session_timeout = parseSessionTimeout(params, 1800, 3600); - - SCOPE_EXIT({ detachSessionContext(session_context, session_id, session_timeout); }); - session_context = attachSessionContext(context, params, session_id, session_timeout); - - initClientInfo(context, request); - used_output.attachRequestAndResponse(context, request, response, params, /* TODO: keep_alive_time_out */ 0); - - HTTPStreamsWithInput used_input(request, params); - collectParamsAndApplySettings(request, params, context); - - Settings & query_settings = context.getSettingsRef(); - used_input.attachSettings(context, query_settings, request); - used_output.attachSettings(context, query_settings, request); - - String execute_query_string = getExecuteQuery(params); - ReadBufferPtr query_in_buffer = std::make_shared(execute_query_string); - - ReadBufferPtr in = query_in_buffer; - if (!needParsePostBody(request, params) || !context.getExternalTables().empty()) - in = std::make_shared(*query_in_buffer, *used_input.in_maybe_internal_compressed); - - executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); }); - - used_output.finalize(); -} - -void HTTPMatchExecutor::initClientInfo(Context & context, HTTPServerRequest & request) const -{ - - ClientInfo & client_info = context.getClientInfo(); - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.interface = ClientInfo::Interface::HTTP; - - /// Query sent through HTTP interface is initial. - client_info.initial_user = client_info.current_user; - client_info.initial_query_id = client_info.current_query_id; - client_info.initial_address = client_info.current_address; - - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) - http_method = ClientInfo::HTTPMethod::GET; - else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) - http_method = ClientInfo::HTTPMethod::POST; - - client_info.http_method = http_method; - client_info.http_user_agent = request.get("User-Agent", ""); -} - -void HTTPMatchExecutor::authentication(Context & context, HTTPServerRequest & request, HTMLForm & params) const -{ - auto user = request.get("X-ClickHouse-User", ""); - auto password = request.get("X-ClickHouse-Key", ""); - auto quota_key = request.get("X-ClickHouse-Quota", ""); - - if (user.empty() && password.empty() && quota_key.empty()) - { - /// User name and password can be passed using query parameters - /// or using HTTP Basic auth (both methods are insecure). - if (request.hasCredentials()) - { - Poco::Net::HTTPBasicCredentials credentials(request); - - user = credentials.getUsername(); - password = credentials.getPassword(); - } - else - { - user = params.get("user", "default"); - password = params.get("password", ""); - } - - quota_key = params.get("quota_key", ""); - } - else - { - /// It is prohibited to mix different authorization schemes. - if (request.hasCredentials() - || params.has("user") - || params.has("password") - || params.has("quota_key")) - { - throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); - } - } - - std::string query_id = params.get("query_id", ""); - context.setUser(user, password, request.clientAddress(), quota_key); - context.setCurrentQueryId(query_id); -} - -std::shared_ptr HTTPMatchExecutor::attachSessionContext( - Context & context, HTMLForm & params, const String & session_id, const duration & session_timeout) const -{ - if (!session_id.empty()) - { - std::string session_check = params.get("session_check", ""); - auto session_context = context.acquireSession(session_id, session_timeout, session_check == "1"); - - context = *session_context; - context.setSessionContext(*session_context); - return session_context; - } - return {}; -} - -void HTTPMatchExecutor::detachSessionContext(std::shared_ptr & session_context, const String & session_id, const duration & session_timeout) const -{ - if (session_context) - session_context->releaseSession(session_id, session_timeout); -} - -void HTTPMatchExecutor::collectParamsAndApplySettings(HTTPServerRequest & request, HTMLForm & params, Context & context) const -{ - static const NameSet reserved_param_names{ - "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "buffer_size", "wait_end_of_query", - "session_id", "session_timeout", "session_check"}; - - Names reserved_param_suffixes; - - auto param_could_be_skipped = [&] (const String & name) - { - if (reserved_param_names.count(name)) - return true; - - for (const String & suffix : reserved_param_suffixes) - { - if (endsWith(name, suffix)) - return true; - } - - return false; - }; - - /// Settings can be overridden in the query. - /// Some parameters (database, default_format, everything used in the code above) do not - /// belong to the Settings class. - - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. - - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - /// Only readonly queries are allowed for HTTP GET requests. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) - { - Settings & settings = context.getSettingsRef(); - - if (settings.readonly == 0) - settings.readonly = 2; - } - - bool has_multipart = startsWith(request.getContentType().data(), "multipart/form-data"); - - if (has_multipart || needParsePostBody(request, params)) - { - ExternalTablesHandler handler(context, params); - params.load(request, request.stream(), handler); - - if (has_multipart) - { - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.reserve(3); - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); - } - } - - SettingsChanges settings_changes; - for (const auto & [key, value] : params) - { - if (key == "database") - context.setCurrentDatabase(value); - else if (key == "default_format") - context.setDefaultFormat(value); - else if (!param_could_be_skipped(key) && !acceptQueryParam(context, key, value)) - settings_changes.push_back({key, value}); /// All other query parameters are treated as settings. - } - - /// For external data we also want settings - context.checkSettingsConstraints(settings_changes); - context.applySettingsChanges(settings_changes); -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h deleted file mode 100644 index 425605aec07..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h +++ /dev/null @@ -1,55 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class Context; -class HTTPMatchExecutor; -struct HTTPStreamsWithInput; -struct HTTPStreamsWithOutput; - -using duration = std::chrono::steady_clock::duration; -using HTTPMatchExecutorPtr = std::shared_ptr; - -class HTTPMatchExecutor -{ -public: - using HTTPServerRequest = Poco::Net::HTTPServerRequest; - using HTTPServerResponse = Poco::Net::HTTPServerResponse; - - bool match(HTTPServerRequest & request, HTMLForm & params) const { return matchImpl(request, params); }; - - void execute(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & params, HTTPStreamsWithOutput & used_output) const; - - virtual ~HTTPMatchExecutor() = default; -protected: - - virtual bool matchImpl(HTTPServerRequest & request, HTMLForm & params) const = 0; - - virtual String getExecuteQuery(HTMLForm & params) const = 0; - - virtual bool needParsePostBody(HTTPServerRequest & request, HTMLForm & params) const = 0; - - virtual bool acceptQueryParam(Context & context, const String & key, const String & value) const = 0; - - void initClientInfo(Context & context, HTTPServerRequest & request) const; - - void authentication(Context & context, HTTPServerRequest & request, HTMLForm & params) const; - - void detachSessionContext(std::shared_ptr & context, const String & session_id, const duration & session_timeout) const; - - std::shared_ptr attachSessionContext(Context & context, HTMLForm & params, const String & session_id, const duration & session_timeout) const; - - void collectParamsAndApplySettings(HTTPServerRequest & request, HTMLForm & params, Context & context) const; - -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h deleted file mode 100644 index 54ea20120f5..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h +++ /dev/null @@ -1,46 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -class HTTPMatchExecutorDefault : public HTTPMatchExecutor -{ -protected: - - bool matchImpl(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } - - bool needParsePostBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return false; } - - String getExecuteQuery(HTMLForm & params) const override - { - String execute_query; - for (const auto & [key, value] : params) - { - if (key == "query") - execute_query += value; - } - - return execute_query; - } - - bool acceptQueryParam(Context &context, const String &key, const String &value) const override - { - if (startsWith(key, "param_")) - { - /// Save name and values of substitution in dictionary. - context.setQueryParameter(key.substr(strlen("param_")), value); - return true; - } - - return key == "query"; - } - -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp similarity index 87% rename from dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp rename to dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp index 35426315eb3..156aa6ba975 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -8,6 +8,7 @@ #include #include #include +#include "HTTPOutputStreams.h" namespace DB @@ -51,8 +52,14 @@ namespace } } -void HTTPStreamsWithOutput::attachSettings(Context & context, Settings & settings, HTTPServerRequest & request) +HTTPOutputStreams::HTTPOutputStreams( + Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) + : out(createResponseOut(request, response, keep_alive_timeout)) + , out_maybe_compressed(createMaybeCompressionOut(form, out)) + , out_maybe_delayed_and_compressed(createMaybeDelayedAndCompressionOut(context, form, out_maybe_compressed)) { + Settings & settings = context.getSettingsRef(); + /// HTTP response compression is turned on only if the client signalled that they support it /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. out->setCompression(out->getCompression() && settings.enable_http_compression); @@ -77,16 +84,8 @@ void HTTPStreamsWithOutput::attachSettings(Context & context, Settings & setting } } -void HTTPStreamsWithOutput::attachRequestAndResponse( - Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) -{ - out = createEndpoint(request, response, keep_alive_timeout); - out_maybe_compressed = createMaybeCompressionEndpoint(form, out); - out_maybe_delayed_and_compressed = createMaybeDelayedAndCompressionEndpoint(context, form, out_maybe_compressed); -} - -std::shared_ptr HTTPStreamsWithOutput::createEndpoint( - HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout) +std::shared_ptr HTTPOutputStreams::createResponseOut( + HTTPServerRequest &request, HTTPServerResponse &response, size_t keep_alive_timeout) { /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -112,15 +111,15 @@ std::shared_ptr HTTPStreamsWithOutput::create request, response, keep_alive_timeout, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); } -WriteBufferPtr HTTPStreamsWithOutput::createMaybeCompressionEndpoint(HTMLForm & form, std::shared_ptr & endpoint) +WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(HTMLForm & form, std::shared_ptr & out_) { /// Client can pass a 'compress' flag in the query string. In this case the query result is /// compressed using internal algorithm. This is not reflected in HTTP headers. bool internal_compression = form.getParsed("compress", false); - return internal_compression ? std::make_shared(*endpoint) : WriteBufferPtr(endpoint); + return internal_compression ? std::make_shared(*out_) : WriteBufferPtr(out_); } -WriteBufferPtr HTTPStreamsWithOutput::createMaybeDelayedAndCompressionEndpoint(Context & context, HTMLForm & form, WriteBufferPtr & endpoint) +WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_) { /// If it is specified, the whole result will be buffered. /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. @@ -152,7 +151,7 @@ WriteBufferPtr HTTPStreamsWithOutput::createMaybeDelayedAndCompressionEndpoint(C } else { - auto push_memory_buffer_and_continue = [next_buffer = endpoint] (const WriteBufferPtr & prev_buf) + auto push_memory_buffer_and_continue = [next_buffer = out_] (const WriteBufferPtr & prev_buf) { auto prev_memory_buffer = typeid_cast(prev_buf.get()); if (!prev_memory_buffer) @@ -170,10 +169,10 @@ WriteBufferPtr HTTPStreamsWithOutput::createMaybeDelayedAndCompressionEndpoint(C return std::make_shared(std::move(cascade_buffer1), std::move(cascade_buffer2)); } - return endpoint; + return out_; } -void HTTPStreamsWithOutput::finalize() const +void HTTPOutputStreams::finalize() const { if (out_maybe_delayed_and_compressed != out_maybe_compressed) { diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h similarity index 61% rename from dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h rename to dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h index 65d87b6744a..b8edaad0b37 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h @@ -22,7 +22,7 @@ using HTTPServerResponse = Poco::Net::HTTPServerResponse; * ↓ * WriteBufferFromHTTPServerResponse out */ -struct HTTPStreamsWithOutput +struct HTTPOutputStreams { using HTTPResponseBufferPtr = std::shared_ptr; @@ -32,17 +32,17 @@ struct HTTPStreamsWithOutput /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. std::shared_ptr out_maybe_delayed_and_compressed; + HTTPOutputStreams() = default; + + HTTPOutputStreams(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); + void finalize() const; - WriteBufferPtr createMaybeDelayedAndCompressionEndpoint(Context & context, HTMLForm & form, WriteBufferPtr & endpoint); + WriteBufferPtr createMaybeDelayedAndCompressionOut(Context &context, HTMLForm &form, WriteBufferPtr &out_); - WriteBufferPtr createMaybeCompressionEndpoint(HTMLForm & form, std::shared_ptr & endpoint); + WriteBufferPtr createMaybeCompressionOut(HTMLForm & form, std::shared_ptr & out_); - HTTPResponseBufferPtr createEndpoint(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout); - - void attachSettings(Context & context, Settings & settings, HTTPServerRequest & request); - - void attachRequestAndResponse(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); + HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout); }; } diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 99afb319ed5..fb018979938 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -170,80 +170,40 @@ static std::chrono::steady_clock::duration parseSessionTimeout( } -void HTTPHandler::pushDelayedResults(Output & used_output) -{ - std::vector write_buffers; - std::vector read_buffers; - std::vector read_buffers_raw_ptr; - - auto cascade_buffer = typeid_cast(used_output.out_maybe_delayed_and_compressed.get()); - if (!cascade_buffer) - throw Exception("Expected CascadeWriteBuffer", ErrorCodes::LOGICAL_ERROR); - - cascade_buffer->getResultBuffers(write_buffers); - - if (write_buffers.empty()) - throw Exception("At least one buffer is expected to overwrite result into HTTP response", ErrorCodes::LOGICAL_ERROR); - - for (auto & write_buf : write_buffers) - { - IReadableWriteBuffer * write_buf_concrete; - ReadBufferPtr reread_buf; - - if (write_buf - && (write_buf_concrete = dynamic_cast(write_buf.get())) - && (reread_buf = write_buf_concrete->tryGetReadBuffer())) - { - read_buffers.emplace_back(reread_buf); - read_buffers_raw_ptr.emplace_back(reread_buf.get()); - } - } - - ConcatReadBuffer concat_read_buffer(read_buffers_raw_ptr); - copyData(concat_read_buffer, *used_output.out_maybe_compressed); -} - - HTTPHandler::HTTPHandler(IServer & server_) - : server(server_) - , log(&Logger::get("HTTPHandler")) + : server(server_), log(&Logger::get("HTTPHandler")) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } -void HTTPHandler::processQuery( - Poco::Net::HTTPServerRequest & request, - HTMLForm & params, - Poco::Net::HTTPServerResponse & response, - HTTPStreamsWithOutput & used_output) +HTTPHandler::SessionContextHolder::~SessionContextHolder() { - Context context = server.context(); + if (session_context) + session_context->releaseSession(session_id, session_timeout); +} - CurrentThread::QueryScope query_scope(context); - LOG_TRACE(log, "Request URI: " << request.getURI()); +HTTPHandler::SessionContextHolder::SessionContextHolder(IServer & accepted_server, HTMLForm & params) +{ + session_id = params.get("session_id", ""); + context = std::make_unique(accepted_server.context()); - if (context.getSettingsRef().allow_experimental_custom_http) + if (!session_id.empty()) { - context.getHTTPMatchExecutor()->execute(context, request, response, params, used_output); - return; + session_timeout = parseSessionTimeout(accepted_server.config(), params); + session_context = context->acquireSession(session_id, session_timeout, params.check("session_check", "1")); + + context = std::make_unique(*session_context); + context->setSessionContext(*session_context); } +} - std::istream & istr = request.stream(); - - /// Part of the query can be passed in the 'query' parameter and the rest in the request body - /// (http method need not necessarily be POST). In this case the entire query consists of the - /// contents of the 'query' parameter, a line break and the request body. - std::string query_param = params.get("query", ""); - if (!query_param.empty()) - query_param += '\n'; - - /// The user and password can be passed by headers (similar to X-Auth-*), - /// which is used by load balancers to pass authentication information. - std::string user = request.get("X-ClickHouse-User", ""); - std::string password = request.get("X-ClickHouse-Key", ""); - std::string quota_key = request.get("X-ClickHouse-Quota", ""); +void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & request, HTMLForm & params) +{ + auto user = request.get("X-ClickHouse-User", ""); + auto password = request.get("X-ClickHouse-Key", ""); + auto quota_key = request.get("X-ClickHouse-Quota", ""); if (user.empty() && password.empty() && quota_key.empty()) { @@ -277,348 +237,35 @@ void HTTPHandler::processQuery( } std::string query_id = params.get("query_id", ""); - context.setUser(user, password, request.clientAddress(), quota_key); - context.setCurrentQueryId(query_id); - - /// The user could specify session identifier and session timeout. - /// It allows to modify settings, create temporary tables and reuse them in subsequent requests. - - std::shared_ptr session; - String session_id; - std::chrono::steady_clock::duration session_timeout; - bool session_is_set = params.has("session_id"); - const auto & config = server.config(); - - if (session_is_set) - { - session_id = params.get("session_id"); - session_timeout = parseSessionTimeout(config, params); - std::string session_check = params.get("session_check", ""); - - session = context.acquireNamedSession(session_id, session_timeout, session_check == "1"); - - context = session->context; - context.setSessionContext(session->context); - } - - SCOPE_EXIT({ - if (session) - session->release(); - }); - - /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). - String http_response_compression_methods = request.get("Accept-Encoding", ""); - CompressionMethod http_response_compression_method = CompressionMethod::None; - - if (!http_response_compression_methods.empty()) - { - /// If client supports brotli - it's preferred. - /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. - /// NOTE parsing of the list of methods is slightly incorrect. - - if (std::string::npos != http_response_compression_methods.find("br")) - http_response_compression_method = CompressionMethod::Brotli; - else if (std::string::npos != http_response_compression_methods.find("gzip")) - http_response_compression_method = CompressionMethod::Gzip; - else if (std::string::npos != http_response_compression_methods.find("deflate")) - http_response_compression_method = CompressionMethod::Zlib; - } - - bool client_supports_http_compression = http_response_compression_method != CompressionMethod::None; - - /// Client can pass a 'compress' flag in the query string. In this case the query result is - /// compressed using internal algorithm. This is not reflected in HTTP headers. - bool internal_compression = params.getParsed("compress", false); - - /// At least, we should postpone sending of first buffer_size result bytes - size_t buffer_size_total = std::max( - params.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); - - /// If it is specified, the whole result will be buffered. - /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. - bool buffer_until_eof = params.getParsed("wait_end_of_query", false); - - size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE; - size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; - - unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); - - used_output.out = std::make_shared( - request, response, keep_alive_timeout, client_supports_http_compression, http_response_compression_method); - - if (internal_compression) - used_output.out_maybe_compressed = std::make_shared(*used_output.out); - else - used_output.out_maybe_compressed = used_output.out; - - if (buffer_size_memory > 0 || buffer_until_eof) - { - CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; - CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; - - if (buffer_size_memory > 0) - cascade_buffer1.emplace_back(std::make_shared(buffer_size_memory)); - - if (buffer_until_eof) - { - const std::string tmp_path(context.getTemporaryVolume()->getNextDisk()->getPath()); - const std::string tmp_path_template(tmp_path + "http_buffers/"); - - auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) - { - return WriteBufferFromTemporaryFile::create(tmp_path_template); - }; - - cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); - } - else - { - auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed] (const WriteBufferPtr & prev_buf) - { - auto prev_memory_buffer = typeid_cast(prev_buf.get()); - if (!prev_memory_buffer) - throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); - - auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); - copyData(*rdbuf , *next_buffer); - - return next_buffer; - }; - - cascade_buffer2.emplace_back(push_memory_buffer_and_continue); - } - - used_output.out_maybe_delayed_and_compressed = std::make_shared( - std::move(cascade_buffer1), std::move(cascade_buffer2)); - } - else - { - used_output.out_maybe_delayed_and_compressed = used_output.out_maybe_compressed; - } - - std::unique_ptr in_param = std::make_unique(query_param); - - std::unique_ptr in_post_raw = std::make_unique(istr); - - /// Request body can be compressed using algorithm specified in the Content-Encoding header. - String http_request_compression_method_str = request.get("Content-Encoding", ""); - std::unique_ptr in_post = wrapReadBufferWithCompressionMethod( - std::make_unique(istr), chooseCompressionMethod({}, http_request_compression_method_str)); - - /// The data can also be compressed using incompatible internal algorithm. This is indicated by - /// 'decompress' query parameter. - std::unique_ptr in_post_maybe_compressed; - bool in_post_compressed = false; - if (params.getParsed("decompress", false)) - { - in_post_maybe_compressed = std::make_unique(*in_post); - in_post_compressed = true; - } - else - in_post_maybe_compressed = std::move(in_post); - - std::unique_ptr in; - - static const NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check"}; - - Names reserved_param_suffixes; - - auto param_could_be_skipped = [&] (const String & name) - { - if (reserved_param_names.count(name)) - return true; - - for (const String & suffix : reserved_param_suffixes) - { - if (endsWith(name, suffix)) - return true; - } - - return false; - }; - - /// Settings can be overridden in the query. - /// Some parameters (database, default_format, everything used in the code above) do not - /// belong to the Settings class. - - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. - - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - const auto & settings = context.getSettingsRef(); - - /// Only readonly queries are allowed for HTTP GET requests. - if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) - { - if (settings.readonly == 0) - context.setSetting("readonly", 2); - } - - bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); - - if (has_external_data) - { - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.reserve(3); - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); - } - - SettingsChanges settings_changes; - for (const auto & [key, value] : params) - { - if (key == "database") - { - context.setCurrentDatabase(value); - } - else if (key == "default_format") - { - context.setDefaultFormat(value); - } - else if (param_could_be_skipped(key)) - { - } - else if (startsWith(key, "param_")) - { - /// Save name and values of substitution in dictionary. - const String parameter_name = key.substr(strlen("param_")); - context.setQueryParameter(parameter_name, value); - } - else - { - /// All other query parameters are treated as settings. - settings_changes.push_back({key, value}); - } - } - - /// For external data we also want settings - context.checkSettingsConstraints(settings_changes); - context.applySettingsChanges(settings_changes); - - /// Used in case of POST request with form-data, but it isn't expected to be deleted after that scope. - std::string full_query; - - /// Support for "external data for query processing". - if (has_external_data) - { - ExternalTablesHandler handler(context, params); - params.load(request, istr, handler); - - /// Params are of both form params POST and uri (GET params) - for (const auto & it : params) - if (it.first == "query") - full_query += it.second; - - in = std::make_unique(full_query); - } - else - in = std::make_unique(*in_param, *in_post_maybe_compressed); - - - /// HTTP response compression is turned on only if the client signalled that they support it - /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. - used_output.out->setCompression(client_supports_http_compression && settings.enable_http_compression); - if (client_supports_http_compression) - used_output.out->setCompressionLevel(settings.http_zlib_compression_level); - - used_output.out->setSendProgressInterval(settings.http_headers_progress_interval_ms); - - /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, - /// checksums of client data compressed with internal algorithm are not checked. - if (in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress) - static_cast(*in_post_maybe_compressed).disableChecksumming(); - - /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed - /// Origin header. - used_output.out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); - - ClientInfo & client_info = context.getClientInfo(); - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.interface = ClientInfo::Interface::HTTP; - - /// Query sent through HTTP interface is initial. - client_info.initial_user = client_info.current_user; - client_info.initial_query_id = client_info.current_query_id; - client_info.initial_address = client_info.current_address; - - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) - http_method = ClientInfo::HTTPMethod::GET; - else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) - http_method = ClientInfo::HTTPMethod::POST; - - client_info.http_method = http_method; - client_info.http_user_agent = request.get("User-Agent", ""); - - auto append_callback = [&context] (ProgressCallback callback) - { - auto prev = context.getProgressCallback(); - - context.setProgressCallback([prev, callback] (const Progress & progress) - { - if (prev) - prev(progress); - - callback(progress); - }); - }; - - /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. - if (settings.send_progress_in_http_headers) - append_callback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); - - if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) - { - Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); - - append_callback([&context, &socket](const Progress &) - { - /// Assume that at the point this method is called no one is reading data from the socket any more. - /// True for read-only queries. - try - { - char b; - //FIXME looks like MSG_DONTWAIT is useless because of POCO_BROKEN_TIMEOUTS - int status = socket.receiveBytes(&b, 1, MSG_DONTWAIT | MSG_PEEK); - if (status == 0) - context.killCurrentQuery(); - } - catch (Poco::TimeoutException &) - { - } - catch (...) - { - context.killCurrentQuery(); - } - }); - } - - customizeContext(context); - - executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone) - { - response.setContentType(content_type); - response.add("X-ClickHouse-Query-Id", current_query_id); - response.add("X-ClickHouse-Format", format); - response.add("X-ClickHouse-Timezone", timezone); - } - ); - - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to - /// the client. - used_output.finalize(); + context->setUser(user, password, request.clientAddress(), quota_key); + context->setCurrentQueryId(query_id); } -void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_code, +void HTTPHandler::processQuery(HTTPRequest & request, HTMLForm & params, HTTPResponse & response, SessionContextHolder & holder) +{ + const auto & [name, custom_executor] = holder.context->getCustomExecutor(request/*, params*/); + + LOG_TRACE(log, "Using " << name << " to execute URI: " << request.getURI()); + + ExtractorClientInfo{holder.context->getClientInfo()}.extract(request); + ExtractorContextChange{*holder.context.get(), custom_executor}.extract(request, params); + + auto & config = server.config(); + HTTPInputStreams input_streams{*holder.context, request, params}; + HTTPOutputStreams output_streams(*holder.context, request, response, params, config.getUInt("keep_alive_timeout", 10)); + + const auto & query_executors = custom_executor->getQueryExecutor(*holder.context, request, params, input_streams); + for (const auto & query_executor : query_executors) + query_executor(output_streams, response); + + output_streams.finalize(); /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + + LOG_INFO(log, "Done processing query"); +} + +void HTTPHandler::trySendExceptionToClient(const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTTPStreamsWithOutput & used_output) + HTTPOutputStreams & used_output) { try { @@ -650,7 +297,7 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ if (!response.sent() && !used_output.out_maybe_compressed) { /// If nothing was sent yet and we don't even know if we must compress the response. - response.send() << s << std::endl; + response.send() << message << std::endl; } else if (used_output.out_maybe_compressed) { @@ -671,7 +318,7 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ used_output.out->position() = used_output.out->buffer().begin(); } - writeString(s, *used_output.out_maybe_compressed); + writeString(message, *used_output.out_maybe_compressed); writeChar('\n', *used_output.out_maybe_compressed); used_output.out_maybe_compressed->next(); @@ -685,21 +332,21 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ } } - void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) { setThreadName("HTTPHandler"); ThreadStatus thread_status; - HTTPStreamsWithOutput used_output; + HTTPOutputStreams used_output; /// In case of exception, send stack trace to client. bool with_stacktrace = false; try { - response.setContentType("text/plain; charset=UTF-8"); + response.set("Content-Type", "text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); + /// For keep-alive to work. if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); @@ -708,14 +355,16 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne with_stacktrace = params.getParsed("stacktrace", false); /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && - !request.hasContentLength()) - { + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); - } - processQuery(request, params, response, used_output); - LOG_INFO(log, "Done processing query"); + { + SessionContextHolder holder{server, params}; + CurrentThread::QueryScope query_scope(*holder.context); + + holder.authentication(request, params); + processQuery(request, params, response, holder); + } } catch (...) { @@ -724,12 +373,11 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne /** If exception is received from remote server, then stack trace is embedded in message. * If exception is thrown on local server, then stack trace is in separate field. */ - std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); int exception_code = getCurrentExceptionCode(); + std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - trySendExceptionToClient(exception_message, exception_code, request, response, used_output); + trySendExceptionToClient(exception_message, exception_code, request, response, HTTPOutputStreams{}); } } - } diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 603278e11aa..10e1f63071e 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -7,7 +7,7 @@ #include #include -#include +#include namespace CurrentMetrics @@ -30,31 +30,22 @@ public: void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - /// This method is called right before the query execution. - virtual void customizeContext(DB::Context& /* context */) {} - private: - struct Output + using HTTPRequest = Poco::Net::HTTPServerRequest; + using HTTPResponse = Poco::Net::HTTPServerResponse; + + struct SessionContextHolder { - /* Raw data - * ↓ - * CascadeWriteBuffer out_maybe_delayed_and_compressed (optional) - * ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults) - * CompressedWriteBuffer out_maybe_compressed (optional) - * ↓ - * WriteBufferFromHTTPServerResponse out - */ + ~SessionContextHolder(); - std::shared_ptr out; - /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. - std::shared_ptr out_maybe_compressed; - /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. - std::shared_ptr out_maybe_delayed_and_compressed; + SessionContextHolder(IServer & accepted_server, HTMLForm & params); - inline bool hasDelayed() const - { - return out_maybe_delayed_and_compressed != out_maybe_compressed; - } + void authentication(HTTPServerRequest & request, HTMLForm & params); + + String session_id; + std::unique_ptr context = nullptr; + std::shared_ptr session_context = nullptr; + std::chrono::steady_clock::duration session_timeout; }; IServer & server; @@ -66,20 +57,11 @@ private: CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; /// Also initializes 'used_output'. - void processQuery( - Poco::Net::HTTPServerRequest & request, - HTMLForm & params, - Poco::Net::HTTPServerResponse & response, - HTTPStreamsWithOutput & used_output); + void processQuery(HTTPRequest & request, HTMLForm & params, HTTPResponse & response, SessionContextHolder & holder); void trySendExceptionToClient( - const std::string & s, - int exception_code, - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response, - HTTPStreamsWithOutput & used_output); + const std::string & message, int exception_code, HTTPRequest & request, HTTPResponse & response, HTTPOutputStreams & used_output); - static void pushDelayedResults(Output & used_output); }; } diff --git a/src/Common/HTMLForm.h b/src/Common/HTMLForm.h index 2490d613160..fa6f31bf5d2 100644 --- a/src/Common/HTMLForm.h +++ b/src/Common/HTMLForm.h @@ -26,6 +26,12 @@ struct HTMLForm : public Poco::Net::HTMLForm readUrl(istr); } + template + bool check(const std::string & key, T check_value) + { + const auto & value = getParsed(key, T()); + return value == check_value; + } template T getParsed(const std::string & key, T default_value) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 03dd311f31b..d5a26966a47 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -44,7 +44,7 @@ #include #include #include -#include +#include #include #include #include @@ -1545,11 +1545,6 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptrclusters->setCluster(cluster_name, cluster); } -HTTPMatchExecutorPtr Context::getHTTPMatchExecutor() -{ - return std::make_shared(); -} - void Context::initializeSystemLogs() { auto lock = getLock(); @@ -2049,6 +2044,11 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } +std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & /*request*/) +{ + return std::pair("Default", std::shared_ptr()); +} + StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where) const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 03859b03348..3cc348260b8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -490,7 +490,7 @@ public: Compiler & getCompiler(); - HTTPMatchExecutorPtr getHTTPMatchExecutor(); + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request/*, HTMLForm & params*/); /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From 60abff3341484dcaf5e858bb12f4935751927de7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 6 Nov 2019 15:02:10 +0800 Subject: [PATCH 400/743] ISSUES-5436 support custom http [part 3] --- .../Interpreters/CustomHTTP/CustomExecutor.h | 6 +- .../CustomHTTP/CustomExecutorDefault.h | 32 +++++-- .../CustomHTTP/ExtractorContextChange.h | 4 +- .../CustomHTTP/HTTPOutputStreams.cpp | 49 +++++++---- .../CustomHTTP/HTTPOutputStreams.h | 18 ++-- programs/server/HTTPHandler.cpp | 88 ++++++------------- programs/server/HTTPHandler.h | 8 +- src/IO/WriteBufferFromHTTPServerResponse.cpp | 8 +- src/IO/WriteBufferFromHTTPServerResponse.h | 4 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 2 +- 11 files changed, 114 insertions(+), 109 deletions(-) diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index 0778256b297..b2014886d93 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -12,13 +12,9 @@ namespace DB { class Context; -class CustomExecutor; struct HTTPInputStreams; struct HTTPOutputStreams; -using duration = std::chrono::steady_clock::duration; -using HTTPMatchExecutorPtr = std::shared_ptr; - class CustomExecutor { public: @@ -38,4 +34,6 @@ public: virtual QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const = 0; }; +using CustomExecutorPtr = std::shared_ptr; + } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h index 218cb413222..9bde5279a5e 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h @@ -6,6 +6,7 @@ #include #include #include +#include "HTTPInputStreams.h" namespace DB @@ -25,12 +26,20 @@ public: QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const override { - ReadBufferPtr in = prepareAndGetQueryInput(context, request, params, input_streams); - - return {[&, shared_in = in](HTTPOutputStreams & output, HTTPServerResponse & response) + return {[&](HTTPOutputStreams & output, HTTPServerResponse & response) { + const auto & execute_query = prepareQuery(context, params); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + ReadBufferPtr temp_query_buf; + if (!startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + executeQuery( - *shared_in, *output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + *execute_query_buf, *output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, [&response] (const String & content_type) { response.setContentType(content_type); }, [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } ); @@ -38,12 +47,21 @@ public: } private: - ReadBufferPtr prepareAndGetQueryInput(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const + String prepareQuery(Context & context, HTMLForm & params) const { - for (const auto & [key, value] : params) - { + const static size_t prefix_size = strlen("param_"); + std::stringstream query_stream; + for (const auto & param : params) + { + if (param.first == "query") + query_stream << param.second; + else if (startsWith(param.first, "param_")) + context.setQueryParameter(param.first.substr(prefix_size), param.second); } + + query_stream << "\n"; + return query_stream.str(); } }; diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h index a228f75391f..7b63df33783 100644 --- a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h +++ b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h @@ -12,7 +12,7 @@ namespace DB class ExtractorContextChange { public: - ExtractorContextChange(Context & context_, const HTTPMatchExecutorPtr & executor_) : context(context_), executor(executor_) {} + ExtractorContextChange(Context & context_, const CustomExecutorPtr & executor_) : context(context_), executor(executor_) {} void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) { @@ -91,7 +91,7 @@ public: private: Context & context; - const HTTPMatchExecutorPtr & executor; + const CustomExecutorPtr & executor; }; diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp index 156aa6ba975..113523494cf 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp @@ -52,10 +52,17 @@ namespace } } +HTTPOutputStreams::HTTPOutputStreams(HTTPServerRequest & request, HTTPServerResponse & response, bool internal_compress, size_t keep_alive_timeout) + : out(createResponseOut(request, response, keep_alive_timeout)) + , out_maybe_compressed(createMaybeCompressionOut(internal_compress, out)) + , out_maybe_delayed_and_compressed(out_maybe_compressed) +{ +} + HTTPOutputStreams::HTTPOutputStreams( Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) : out(createResponseOut(request, response, keep_alive_timeout)) - , out_maybe_compressed(createMaybeCompressionOut(form, out)) + , out_maybe_compressed(createMaybeCompressionOut(form.getParsed("compress", false), out)) , out_maybe_delayed_and_compressed(createMaybeDelayedAndCompressionOut(context, form, out_maybe_compressed)) { Settings & settings = context.getSettingsRef(); @@ -68,8 +75,7 @@ HTTPOutputStreams::HTTPOutputStreams( out->setSendProgressInterval(settings.http_headers_progress_interval_ms); - /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed - /// Origin header. + /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed Origin header. out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. @@ -84,8 +90,7 @@ HTTPOutputStreams::HTTPOutputStreams( } } -std::shared_ptr HTTPOutputStreams::createResponseOut( - HTTPServerRequest &request, HTTPServerResponse &response, size_t keep_alive_timeout) +HTTPResponseBufferPtr HTTPOutputStreams::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive) { /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -96,27 +101,26 @@ std::shared_ptr HTTPOutputStreams::createResp /// NOTE parsing of the list of methods is slightly incorrect. if (std::string::npos != http_response_compression_methods.find("gzip")) return std::make_shared( - request, response, keep_alive_timeout, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); else if (std::string::npos != http_response_compression_methods.find("deflate")) return std::make_shared( - request, response, keep_alive_timeout, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); + request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); #if USE_BROTLI else if (http_response_compression_methods == "br") return std::make_shared( - request, response, keep_alive_timeout, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); + request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); #endif } - return std::make_shared( - request, response, keep_alive_timeout, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); } -WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(HTMLForm & form, std::shared_ptr & out_) +WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(bool compression, HTTPResponseBufferPtr & out_) { /// Client can pass a 'compress' flag in the query string. In this case the query result is /// compressed using internal algorithm. This is not reflected in HTTP headers. - bool internal_compression = form.getParsed("compress", false); - return internal_compression ? std::make_shared(*out_) : WriteBufferPtr(out_); +// bool internal_compression = form.getParsed("compress", false); + return compression ? std::make_shared(*out_) : WriteBufferPtr(out_); } WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_) @@ -172,6 +176,20 @@ WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & return out_; } +HTTPOutputStreams::~HTTPOutputStreams() +{ + /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references + if (out_maybe_delayed_and_compressed != out_maybe_compressed) + out_maybe_delayed_and_compressed.reset(); + + /// If buffer has data, and that data wasn't sent yet, then no need to send that data + if (out->count() == out->offset()) + { + out_maybe_compressed->position() = out_maybe_compressed->buffer().begin(); + out->position() = out->buffer().begin(); + } +} + void HTTPOutputStreams::finalize() const { if (out_maybe_delayed_and_compressed != out_maybe_compressed) @@ -208,8 +226,9 @@ void HTTPOutputStreams::finalize() const copyData(concat_read_buffer, *out_maybe_compressed); } - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to - /// the client. + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + out_maybe_compressed->next(); + out->next(); out->finalize(); } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h index b8edaad0b37..614acb6fcf3 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h @@ -12,7 +12,7 @@ namespace DB using HTTPServerRequest = Poco::Net::HTTPServerRequest; using HTTPServerResponse = Poco::Net::HTTPServerResponse; - +using HTTPResponseBufferPtr = std::shared_ptr; /* Raw data * ↓ @@ -24,25 +24,27 @@ using HTTPServerResponse = Poco::Net::HTTPServerResponse; */ struct HTTPOutputStreams { - using HTTPResponseBufferPtr = std::shared_ptr; - HTTPResponseBufferPtr out; /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. std::shared_ptr out_maybe_compressed; /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. std::shared_ptr out_maybe_delayed_and_compressed; - HTTPOutputStreams() = default; - - HTTPOutputStreams(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); + ~HTTPOutputStreams(); void finalize() const; WriteBufferPtr createMaybeDelayedAndCompressionOut(Context &context, HTMLForm &form, WriteBufferPtr &out_); - WriteBufferPtr createMaybeCompressionOut(HTMLForm & form, std::shared_ptr & out_); + WriteBufferPtr createMaybeCompressionOut(bool compression, std::shared_ptr & out_); - HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout); + HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive); + + HTTPOutputStreams(HTTPServerRequest & request, HTTPServerResponse & response, bool internal_compress, size_t keep_alive_timeout); + + HTTPOutputStreams(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); }; +using HTTPOutputStreamsPtr = std::unique_ptr; + } diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index fb018979938..b8bc3c9820c 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -192,7 +192,7 @@ HTTPHandler::SessionContextHolder::SessionContextHolder(IServer & accepted_serve if (!session_id.empty()) { session_timeout = parseSessionTimeout(accepted_server.config(), params); - session_context = context->acquireSession(session_id, session_timeout, params.check("session_check", "1")); + session_context = context->acquireSession(session_id, session_timeout, params.check("session_check", "1")); context = std::make_unique(*session_context); context->setSessionContext(*session_context); @@ -241,31 +241,26 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque context->setCurrentQueryId(query_id); } -void HTTPHandler::processQuery(HTTPRequest & request, HTMLForm & params, HTTPResponse & response, SessionContextHolder & holder) +void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response) { - const auto & [name, custom_executor] = holder.context->getCustomExecutor(request/*, params*/); + const auto & name_with_custom_executor = context.getCustomExecutor(request/*, params*/); + LOG_TRACE(log, "Using " << name_with_custom_executor.first << " to execute URI: " << request.getURI()); - LOG_TRACE(log, "Using " << name << " to execute URI: " << request.getURI()); + ExtractorClientInfo{context.getClientInfo()}.extract(request); + ExtractorContextChange{context, name_with_custom_executor.second}.extract(request, params); - ExtractorClientInfo{holder.context->getClientInfo()}.extract(request); - ExtractorContextChange{*holder.context.get(), custom_executor}.extract(request, params); + HTTPInputStreams input_streams{context, request, params}; + HTTPOutputStreams output_streams = HTTPOutputStreams(context, request, response, params, getKeepAliveTimeout()); - auto & config = server.config(); - HTTPInputStreams input_streams{*holder.context, request, params}; - HTTPOutputStreams output_streams(*holder.context, request, response, params, config.getUInt("keep_alive_timeout", 10)); - - const auto & query_executors = custom_executor->getQueryExecutor(*holder.context, request, params, input_streams); + const auto & query_executors = name_with_custom_executor.second->getQueryExecutor(context, request, params, input_streams); for (const auto & query_executor : query_executors) query_executor(output_streams, response); output_streams.finalize(); /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. - - LOG_INFO(log, "Done processing query"); } -void HTTPHandler::trySendExceptionToClient(const std::string & message, int exception_code, - Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTTPOutputStreams & used_output) +void HTTPHandler::trySendExceptionToClient( + const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, bool compression) { try { @@ -273,57 +268,25 @@ void HTTPHandler::trySendExceptionToClient(const std::string & message, int exce /// If HTTP method is POST and Keep-Alive is turned on, we should read the whole request body /// to avoid reading part of the current request body in the next request. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST - && response.getKeepAlive() - && !request.stream().eof() - && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) - { + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() + && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) request.stream().ignore(std::numeric_limits::max()); - } - bool auth_fail = exception_code == ErrorCodes::UNKNOWN_USER || - exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::REQUIRED_PASSWORD; - - if (auth_fail) + if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD || exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) { response.requireAuthentication("ClickHouse server HTTP API"); + response.send() << message << std::endl; } else { response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); - } + HTTPOutputStreams output_streams(request, response, compression, getKeepAliveTimeout()); - if (!response.sent() && !used_output.out_maybe_compressed) - { - /// If nothing was sent yet and we don't even know if we must compress the response. - response.send() << message << std::endl; - } - else if (used_output.out_maybe_compressed) - { - /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references - if (used_output.out_maybe_delayed_and_compressed != used_output.out_maybe_compressed) - used_output.out_maybe_delayed_and_compressed.reset(); + writeString(message, *output_streams.out_maybe_compressed); + writeChar('\n', *output_streams.out_maybe_compressed); - /// Send the error message into already used (and possibly compressed) stream. - /// Note that the error message will possibly be sent after some data. - /// Also HTTP code 200 could have already been sent. - - /// If buffer has data, and that data wasn't sent yet, then no need to send that data - bool data_sent = used_output.out->count() != used_output.out->offset(); - - if (!data_sent) - { - used_output.out_maybe_compressed->position() = used_output.out_maybe_compressed->buffer().begin(); - used_output.out->position() = used_output.out->buffer().begin(); - } - - writeString(message, *used_output.out_maybe_compressed); - writeChar('\n', *used_output.out_maybe_compressed); - - used_output.out_maybe_compressed->next(); - used_output.out->next(); - used_output.out->finalize(); + output_streams.finalize(); } } catch (...) @@ -337,10 +300,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne setThreadName("HTTPHandler"); ThreadStatus thread_status; - HTTPOutputStreams used_output; - /// In case of exception, send stack trace to client. - bool with_stacktrace = false; + bool with_stacktrace = false, internal_compression = false; try { @@ -353,6 +314,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne HTMLForm params(request); with_stacktrace = params.getParsed("stacktrace", false); + internal_compression = params.getParsed("compress", false); /// Workaround. Poco does not detect 411 Length Required case. if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) @@ -363,7 +325,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne CurrentThread::QueryScope query_scope(*holder.context); holder.authentication(request, params); - processQuery(request, params, response, holder); + processQuery(*holder.context, request, params, response); + LOG_INFO(log, "Done processing query"); } } catch (...) @@ -375,8 +338,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne */ int exception_code = getCurrentExceptionCode(); std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - - trySendExceptionToClient(exception_message, exception_code, request, response, HTTPOutputStreams{}); + trySendExceptionToClient(exception_message, exception_code, request, response, internal_compression); } } diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 10e1f63071e..47d63af3740 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -56,11 +56,11 @@ private: CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; - /// Also initializes 'used_output'. - void processQuery(HTTPRequest & request, HTMLForm & params, HTTPResponse & response, SessionContextHolder & holder); + size_t getKeepAliveTimeout() { return server.config().getUInt("keep_alive_timeout", 10); } - void trySendExceptionToClient( - const std::string & message, int exception_code, HTTPRequest & request, HTTPResponse & response, HTTPOutputStreams & used_output); + void processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response); + + void trySendExceptionToClient(const std::string & message, int exception_code, HTTPRequest & request, HTTPResponse & response, bool compression); }; diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index 0f30f1352e3..252a32a5d7c 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -153,13 +153,17 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerResponse & response_, unsigned keep_alive_timeout_, bool compress_, - CompressionMethod compression_method_) - : BufferWithOwnMemory(DBMS_DEFAULT_BUFFER_SIZE) + CompressionMethod compression_method_, + size_t size, + bool finish_send_headers_) + : BufferWithOwnMemory(size) , request(request_) , response(response_) , keep_alive_timeout(keep_alive_timeout_) , compress(compress_) , compression_method(compression_method_) + , headers_started_sending(finish_send_headers_) + , headers_finished_sending(finish_send_headers_) { } diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index c10288fff9f..638a1f69dee 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -95,7 +95,9 @@ public: Poco::Net::HTTPServerResponse & response_, unsigned keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. - CompressionMethod compression_method_ = CompressionMethod::None); + CompressionMethod compression_method_ = CompressionMethod::Gzip, + size_t size = DBMS_DEFAULT_BUFFER_SIZE, + bool finish_send_headers_ = false); /// Writes progess in repeating HTTP headers. void onProgress(const Progress & progress); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d5a26966a47..06b34cccd35 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2044,9 +2044,9 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } -std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & /*request*/) +std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & /*request*/) { - return std::pair("Default", std::shared_ptr()); + return std::pair("Default", std::make_shared()); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3cc348260b8..6e0b78250e4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -490,7 +490,7 @@ public: Compiler & getCompiler(); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request/*, HTMLForm & params*/); + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request/*, HTMLForm & params*/); /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From fd00757178323ee948cd660209c93cd7f7611355 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 6 Nov 2019 19:01:02 +0800 Subject: [PATCH 401/743] ISSUES-5436 support custom http [part 4] --- .../CustomHTTP/CustomExecutor.cpp | 121 ++++++++++++++++++ .../Interpreters/CustomHTTP/CustomExecutor.h | 77 +++++++++-- .../CustomHTTP/CustomExecutorDefault.h | 50 ++++---- .../CustomHTTP/HTTPInputStreams.cpp | 2 +- .../CustomHTTP/HTTPInputStreams.h | 2 +- programs/server/HTTPHandler.cpp | 9 +- programs/server/Server.cpp | 1 + src/Interpreters/Context.cpp | 31 ++++- src/Interpreters/Context.h | 4 +- 9 files changed, 249 insertions(+), 48 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp new file mode 100644 index 00000000000..7767082555e --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -0,0 +1,121 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + +bool CustomExecutor::match(HTTPRequest & request, HTMLForm & params) const +{ + for (const auto & matcher : matchers) + { + if (!matcher->match(request, params)) + return false; + } + + return true; +} + +bool CustomExecutor::isQueryParam(const String & param_name) const +{ + for (const auto & query_executor : query_executors) + { + if (!query_executor->isQueryParam(param_name)) + return false; + } + + return true; +} + +bool CustomExecutor::canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const +{ + for (const auto & query_executor : query_executors) + { + if (!query_executor->canBeParseRequestBody(request, params)) + return false; + } + + return true; +} + +void CustomExecutor::executeQuery( + Context & context, HTTPRequest & request, HTTPResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) +{ + for (const auto & query_executor : query_executors) + query_executor->executeQueryImpl(context, request, response, params, input_streams, output_streams); + + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + output_streams.finalize(); +} + +CustomExecutor::CustomExecutor( + const std::vector & matchers_, const std::vector & query_executors_) + : matchers(matchers_), query_executors(query_executors_) +{ +} + +CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +{ + updateCustomExecutors(config, settings, config_prefix); +} + +void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +{ + Configuration::Keys custom_executors_keys; + config.keys(config_prefix, custom_executors_keys); + + std::unordered_map new_custom_executors; + + for (const auto & custom_executor_key : custom_executors_keys) + { + if (custom_executor_key == "Default") + throw Exception("CustomExecutor cannot be 'Default'.", ErrorCodes::SYNTAX_ERROR); + else if (custom_executor_key.find('.') != String::npos) + throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); + + new_custom_executors[custom_executor_key] = createCustomExecutor(config, settings, config_prefix + "." + custom_executor_key); + } + + new_custom_executors["Default"] = CustomExecutorDefault::createDefaultCustomExecutor(); + + std::unique_lock lock(rwlock); + custom_executors = new_custom_executors; +} + +CustomExecutorPtr CustomExecutors::createCustomExecutor(const CustomExecutors::Configuration & config, const Settings & /*settings*/, const String & config_prefix) +{ + Configuration::Keys matchers_or_query_executors_type; + config.keys(config_prefix, matchers_or_query_executors_type); + + for (const auto & matcher_or_query_executor_type : matchers_or_query_executors_type) + { + if (matcher_or_query_executor_type.find('.') != String::npos) + throw Exception( + "CustomMatcher or CustomQueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", + ErrorCodes::SYNTAX_ERROR); + +// throw Exception("", ErrorCodes::NOT_IMPLEMENTED); +// new_custom_executors[matcher_or_query_executor_type] = createCustomExecutor(config, settings, config_prefix + "." + matcher_or_query_executor_type); + } + return DB::CustomExecutorPtr(); +} + +std::pair CustomExecutors::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const +{ + std::shared_lock lock(rwlock); + + for (const auto & custom_executor : custom_executors) + if (custom_executor.second->match(request, params)) + return custom_executor; + + throw Exception("LOGICAL_ERROR not found custom executor.", ErrorCodes::LOGICAL_ERROR); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index b2014886d93..f6584e249be 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -1,39 +1,92 @@ #pragma once +#include + #include +#include +#include #include #include #include #include #include -#include namespace DB { class Context; +class CustomExecutor; struct HTTPInputStreams; struct HTTPOutputStreams; +using HTTPRequest = Poco::Net::HTTPServerRequest; +using HTTPResponse = Poco::Net::HTTPServerResponse; +using CustomExecutorPtr = std::shared_ptr; + +class CustomExecutors +{ +public: + using Configuration = Poco::Util::AbstractConfiguration; + CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix = "CustomHTTP"); + + CustomExecutors(const CustomExecutors &) = delete; + CustomExecutors & operator=(const CustomExecutors &) = delete; + + void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); + + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const; +private: + mutable std::shared_mutex rwlock; + std::unordered_map custom_executors; + + CustomExecutorPtr createCustomExecutor(const Configuration & config, const Settings & settings, const String & config_prefix); +}; + class CustomExecutor { public: - using HTTPServerRequest = Poco::Net::HTTPServerRequest; - using HTTPServerResponse = Poco::Net::HTTPServerResponse; + bool isQueryParam(const String & param_name) const; - virtual ~CustomExecutor() = default; + bool match(HTTPRequest & request, HTMLForm & params) const; - virtual bool isQueryParam(const String & param_name) const = 0; + bool canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const; - virtual bool match(HTTPServerRequest & request, HTMLForm & params) const = 0; + void executeQuery( + Context & context, HTTPRequest & request, HTTPResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams + ); - virtual bool canBeParseRequestBody(HTTPServerRequest & request, HTMLForm & params) = 0; +public: + class CustomMatcher + { + public: + virtual ~CustomMatcher() = default; - using QueryExecutor = std::function; - using QueryExecutors = std::vector; - virtual QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const = 0; + virtual bool match(HTTPRequest & request, HTMLForm & params) const = 0; + }; + + class CustomQueryExecutor + { + public: + virtual ~CustomQueryExecutor() = default; + + virtual bool isQueryParam(const String &) const = 0; + virtual bool canBeParseRequestBody(HTTPRequest &, HTMLForm &) const = 0; + + virtual void executeQueryImpl( + Context & context, HTTPRequest & request, HTTPResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; + }; + +public: + using CustomMatcherPtr = std::shared_ptr; + using CustomQueryExecutorPtr = std::shared_ptr; + + CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); + +private: + std::vector matchers; + std::vector query_executors; }; -using CustomExecutorPtr = std::shared_ptr; - } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h index 9bde5279a5e..20b0fdb4eb5 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h @@ -5,45 +5,51 @@ #include #include #include +#include #include -#include "HTTPInputStreams.h" namespace DB { -class CustomExecutorDefault : public CustomExecutor +class CustomExecutorDefault : public CustomExecutor::CustomMatcher, public CustomExecutor::CustomQueryExecutor { public: bool match(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } - bool canBeParseRequestBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) override { return false; } + bool canBeParseRequestBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return false; } - bool isQueryParam(const String & param_name) const override + bool isQueryParam(const String & param_name) const override { return param_name == "query" || startsWith(param_name, "param_"); } + + void executeQueryImpl( + Context & context, HTTPRequest & request, HTTPResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override { - return param_name == "query" || startsWith(param_name, "param_"); + const auto & execute_query = prepareQuery(context, params); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + ReadBufferPtr temp_query_buf; + if (!startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); } - QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const override + static CustomExecutorPtr createDefaultCustomExecutor() { - return {[&](HTTPOutputStreams & output, HTTPServerResponse & response) - { - const auto & execute_query = prepareQuery(context, params); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + const auto & default_custom_executor = std::make_shared(); - ReadBufferPtr temp_query_buf; - if (!startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } + std::vector custom_matchers{default_custom_executor}; + std::vector custom_query_executors{default_custom_executor}; - executeQuery( - *execute_query_buf, *output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); - }}; + return std::make_shared(custom_matchers, custom_query_executors); } private: diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp index e6985621b00..7c441891b33 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp @@ -24,7 +24,7 @@ HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & reques /// checksums of client data compressed with internal algorithm are not checked. if (context.getSettingsRef().http_native_compression_disable_checksumming_on_decompress) { - if(CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) + if (CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) compressed_buffer->disableChecksumming(); } } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h index 88f0438398d..389d4312362 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h @@ -24,4 +24,4 @@ struct HTTPInputStreams ReadBufferPtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const; }; -} \ No newline at end of file +} diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index b8bc3c9820c..8f460b50db7 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -243,7 +243,7 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response) { - const auto & name_with_custom_executor = context.getCustomExecutor(request/*, params*/); + const auto & name_with_custom_executor = context.getCustomExecutor(request, params); LOG_TRACE(log, "Using " << name_with_custom_executor.first << " to execute URI: " << request.getURI()); ExtractorClientInfo{context.getClientInfo()}.extract(request); @@ -251,12 +251,7 @@ void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLFor HTTPInputStreams input_streams{context, request, params}; HTTPOutputStreams output_streams = HTTPOutputStreams(context, request, response, params, getKeepAliveTimeout()); - - const auto & query_executors = name_with_custom_executor.second->getQueryExecutor(context, request, params, input_streams); - for (const auto & query_executor : query_executors) - query_executor(output_streams, response); - - output_streams.finalize(); /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + name_with_custom_executor.second->executeQuery(context, request, response, params, input_streams, output_streams); } void HTTPHandler::trySendExceptionToClient( diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3490ff6a445..710506151c4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -466,6 +466,7 @@ int Server::main(const std::vector & /*args*/) //setTextLog(global_context->getTextLog()); //buildLoggers(*config, logger()); global_context->setClustersConfig(config); + global_context->setCustomExecutorConfig(config); global_context->setMacros(std::make_unique(*config, "macros")); /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 06b34cccd35..39c7953b29a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -44,7 +45,7 @@ #include #include #include -#include +#include #include #include #include @@ -349,7 +350,9 @@ struct ContextShared std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config - mutable std::mutex match_executor_mutex; /// Guards match executor and their config + std::unique_ptr custom_executors; + ConfigurationPtr custom_executors_config; + mutable std::mutex custom_executors_mutex; /// Guards custom executors and their config #if USE_EMBEDDED_COMPILER std::shared_ptr compiled_expression_cache; @@ -2044,9 +2047,29 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } -std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & /*request*/) +void Context::setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix) { - return std::pair("Default", std::make_shared()); + std::lock_guard lock(shared->custom_executors_mutex); + + shared->custom_executors_config = config; + + if (!shared->custom_executors) + shared->custom_executors = std::make_unique(*shared->custom_executors_config, settings, config_prefix); + else + shared->custom_executors->updateCustomExecutors(*shared->custom_executors_config, settings, config_prefix); +} + +std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const +{ + std::lock_guard lock(shared->custom_executors_mutex); + + if (!shared->custom_executors) + { + auto & config = shared->custom_executors_config ? *shared->custom_executors_config : getConfigRef(); + shared->custom_executors = std::make_unique(config, settings); + } + + return shared->custom_executors->getCustomExecutor(request, params); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 6e0b78250e4..c73d03179c9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -42,6 +42,7 @@ namespace zkutil class ZooKeeper; } +class HTMLForm; namespace DB { @@ -490,7 +491,8 @@ public: Compiler & getCompiler(); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request/*, HTMLForm & params*/); + void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "CustomHTTP"); + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params) const; /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From 7aef95b0820d446a5ab3df504030fcb0bae7f6c2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 6 Nov 2019 21:40:38 +0800 Subject: [PATCH 402/743] ISSUES-5436 support custom http [part 5] --- .../CustomHTTP/CustomExecutor.cpp | 83 +++++++++++++--- .../Interpreters/CustomHTTP/CustomExecutor.h | 57 ++++------- .../CustomHTTP/CustomExecutorMatchers.h | 97 +++++++++++++++++++ ...ecutorDefault.h => CustomQueryExecutors.h} | 37 ++++--- programs/server/HTTPHandler.cpp | 2 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 4 +- 7 files changed, 211 insertions(+), 73 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h rename dbms/src/Interpreters/CustomHTTP/{CustomExecutorDefault.h => CustomQueryExecutors.h} (67%) diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp index 7767082555e..c37da93bcaf 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include "CustomExecutor.h" namespace DB @@ -11,11 +11,11 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -bool CustomExecutor::match(HTTPRequest & request, HTMLForm & params) const +bool CustomExecutor::match(Context & context, HTTPRequest & request, HTMLForm & params) const { for (const auto & matcher : matchers) { - if (!matcher->match(request, params)) + if (!matcher->match(context, request, params)) return false; } @@ -56,14 +56,17 @@ void CustomExecutor::executeQuery( } CustomExecutor::CustomExecutor( - const std::vector & matchers_, const std::vector & query_executors_) + const std::vector & matchers_, const std::vector & query_executors_) : matchers(matchers_), query_executors(query_executors_) { } -CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +static CustomExecutorPtr createDefaultCustomExecutor() { - updateCustomExecutors(config, settings, config_prefix); + std::vector custom_matchers{std::make_shared()}; + std::vector custom_query_executors{std::make_shared()}; + + return std::make_shared(custom_matchers, custom_query_executors); } void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) @@ -71,7 +74,7 @@ void CustomExecutors::updateCustomExecutors(const Configuration & config, const Configuration::Keys custom_executors_keys; config.keys(config_prefix, custom_executors_keys); - std::unordered_map new_custom_executors; + std::vector> new_custom_executors; for (const auto & custom_executor_key : custom_executors_keys) { @@ -80,20 +83,48 @@ void CustomExecutors::updateCustomExecutors(const Configuration & config, const else if (custom_executor_key.find('.') != String::npos) throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); - new_custom_executors[custom_executor_key] = createCustomExecutor(config, settings, config_prefix + "." + custom_executor_key); + new_custom_executors.push_back( + std::make_pair(custom_executor_key, createCustomExecutor(config, config_prefix + "." + custom_executor_key))); } - new_custom_executors["Default"] = CustomExecutorDefault::createDefaultCustomExecutor(); + new_custom_executors.push_back(std::make_pair("Default", createDefaultCustomExecutor())); std::unique_lock lock(rwlock); custom_executors = new_custom_executors; } -CustomExecutorPtr CustomExecutors::createCustomExecutor(const CustomExecutors::Configuration & config, const Settings & /*settings*/, const String & config_prefix) +void CustomExecutors::registerQueryExecutor(const String & query_executor_name, const CustomExecutors::QueryExecutorCreator & creator) +{ + const auto & matcher_creator_it = custom_matcher_creators.find(query_executor_name); + const auto & query_executor_creator_it = query_executor_creators.find(query_executor_name); + + if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) + throw Exception("LOGICAL_ERROR CustomQueryExecutor name must be unique between the CustomQueryExecutor and CustomExecutorMatcher.", + ErrorCodes::LOGICAL_ERROR); + + query_executor_creators[query_executor_name] = creator; +} + +void CustomExecutors::registerCustomMatcher(const String & matcher_name, const CustomExecutors::CustomMatcherCreator & creator) +{ + const auto & matcher_creator_it = custom_matcher_creators.find(matcher_name); + const auto & query_executor_creator_it = query_executor_creators.find(matcher_name); + + if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) + throw Exception("LOGICAL_ERROR CustomExecutorMatcher name must be unique between the CustomQueryExecutor and CustomExecutorMatcher.", + ErrorCodes::LOGICAL_ERROR); + + custom_matcher_creators[matcher_name] = creator; +} + +CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & config, const String & config_prefix) { Configuration::Keys matchers_or_query_executors_type; config.keys(config_prefix, matchers_or_query_executors_type); + std::vector custom_query_executors; + std::vector custom_executor_matchers; + for (const auto & matcher_or_query_executor_type : matchers_or_query_executors_type) { if (matcher_or_query_executor_type.find('.') != String::npos) @@ -101,21 +132,43 @@ CustomExecutorPtr CustomExecutors::createCustomExecutor(const CustomExecutors::C "CustomMatcher or CustomQueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", ErrorCodes::SYNTAX_ERROR); -// throw Exception("", ErrorCodes::NOT_IMPLEMENTED); -// new_custom_executors[matcher_or_query_executor_type] = createCustomExecutor(config, settings, config_prefix + "." + matcher_or_query_executor_type); + const auto & matcher_creator_it = custom_matcher_creators.find(matcher_or_query_executor_type); + const auto & query_executor_creator_it = query_executor_creators.find(matcher_or_query_executor_type); + + if (matcher_creator_it == custom_matcher_creators.end() && query_executor_creator_it == query_executor_creators.end()) + throw Exception("CustomMatcher or CustomQueryExecutor '" + matcher_or_query_executor_type + "' is not implemented.", + ErrorCodes::NOT_IMPLEMENTED); + + if (matcher_creator_it != custom_matcher_creators.end()) + custom_executor_matchers.push_back(matcher_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); + + if (query_executor_creator_it != query_executor_creators.end()) + custom_query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); } - return DB::CustomExecutorPtr(); + + for (const auto & custom_executor_matcher : custom_executor_matchers) + custom_executor_matcher->checkQueryExecutor(custom_query_executors); + + return std::make_shared(custom_executor_matchers, custom_query_executors); } -std::pair CustomExecutors::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const +std::pair CustomExecutors::getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const { std::shared_lock lock(rwlock); for (const auto & custom_executor : custom_executors) - if (custom_executor.second->match(request, params)) + if (custom_executor.second->match(context, request, params)) return custom_executor; throw Exception("LOGICAL_ERROR not found custom executor.", ErrorCodes::LOGICAL_ERROR); } +CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +{ + registerCustomMatcher("URL", [&](const auto & config, const auto & prefix) + { return std::make_shared(config, prefix); }); + + updateCustomExecutors(config, settings, config_prefix); +} + } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index f6584e249be..5b5ccc53850 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -10,14 +10,15 @@ #include #include #include +#include +#include +#include +#include +#include namespace DB { - -class Context; class CustomExecutor; -struct HTTPInputStreams; -struct HTTPOutputStreams; using HTTPRequest = Poco::Net::HTTPServerRequest; using HTTPResponse = Poco::Net::HTTPServerResponse; @@ -32,14 +33,22 @@ public: CustomExecutors(const CustomExecutors &) = delete; CustomExecutors & operator=(const CustomExecutors &) = delete; + using QueryExecutorCreator = std::function; + void registerQueryExecutor(const String & query_executor_name, const QueryExecutorCreator & creator); + + using CustomMatcherCreator = const std::function; + void registerCustomMatcher(const String & matcher_name, const CustomMatcherCreator & creator); + void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const; + std::pair getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const; private: mutable std::shared_mutex rwlock; - std::unordered_map custom_executors; + std::vector> custom_executors; + std::unordered_map query_executor_creators; + std::unordered_map custom_matcher_creators; - CustomExecutorPtr createCustomExecutor(const Configuration & config, const Settings & settings, const String & config_prefix); + CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix); }; class CustomExecutor @@ -47,45 +56,19 @@ class CustomExecutor public: bool isQueryParam(const String & param_name) const; - bool match(HTTPRequest & request, HTMLForm & params) const; - bool canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const; + bool match(Context & context, HTTPRequest & request, HTMLForm & params) const; + void executeQuery( Context & context, HTTPRequest & request, HTTPResponse & response, HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams ); -public: - class CustomMatcher - { - public: - virtual ~CustomMatcher() = default; - - virtual bool match(HTTPRequest & request, HTMLForm & params) const = 0; - }; - - class CustomQueryExecutor - { - public: - virtual ~CustomQueryExecutor() = default; - - virtual bool isQueryParam(const String &) const = 0; - virtual bool canBeParseRequestBody(HTTPRequest &, HTMLForm &) const = 0; - - virtual void executeQueryImpl( - Context & context, HTTPRequest & request, HTTPResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; - }; - -public: - using CustomMatcherPtr = std::shared_ptr; - using CustomQueryExecutorPtr = std::shared_ptr; - - CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); + CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); private: - std::vector matchers; + std::vector matchers; std::vector query_executors; }; diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h new file mode 100644 index 00000000000..79a48f069ca --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h @@ -0,0 +1,97 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +#if USE_RE2_ST +# include +#else +# define re2_st re2 +#endif + +namespace DB +{ + +class CustomExecutorMatcher +{ +public: + virtual ~CustomExecutorMatcher() = default; + + virtual bool checkQueryExecutor(const std::vector & check_executors) const = 0; + + virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; +}; + +using CustomExecutorMatcherPtr = std::shared_ptr; + + +class AlwaysMatchedCustomExecutorMatcher : public CustomExecutorMatcher +{ +public: + bool checkQueryExecutor(const std::vector & /*check_executors*/) const override { return true; } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } +}; + +class HTTPURLCustomExecutorMatcher : public CustomExecutorMatcher +{ +public: + HTTPURLCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & url_config_key) + : url_match_searcher(analyzeURLPatten(configuration.getString(url_config_key, ""), params_name_extract_from_url)) + { + } + + bool checkQueryExecutor(const std::vector & custom_query_executors) const override + { + for (const auto & param_name_from_url : params_name_extract_from_url) + { + bool found_param_name = false; + for (const auto & custom_query_executor : custom_query_executors) + { + if (custom_query_executor->isQueryParam(param_name_from_url)) + { + found_param_name = true; + break; + } + } + + if (!found_param_name) + throw Exception("The param name '" + param_name_from_url + "' is uselessed.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + } + + return true; + } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override + { + const String request_uri = request.getURI(); + re2_st::StringPiece query_params_matches[params_name_extract_from_url.size()]; + +// re2_st::StringPiece input; +// if (url_match_searcher.Match(input, start_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, query_params_matches, num_captures)) +// { +// +// } + return false; + } + +private: + re2_st::RE2 url_match_searcher; + std::vector params_name_extract_from_url; + + String analyzeURLPatten(const String & /*url_patten*/, std::vector & /*matches*/) + { + return ".+"; + /// TODO: first we replace all capture group + /// TODO: second we replace all ${identifier} + } +}; + + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h similarity index 67% rename from dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h rename to dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h index 20b0fdb4eb5..0473148b89c 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h @@ -1,28 +1,43 @@ #pragma once -#include +#include +#include #include #include #include #include #include #include - +#include +#include namespace DB { -class CustomExecutorDefault : public CustomExecutor::CustomMatcher, public CustomExecutor::CustomQueryExecutor +class CustomQueryExecutor { public: - bool match(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } + virtual ~CustomQueryExecutor() = default; - bool canBeParseRequestBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return false; } + virtual bool isQueryParam(const String &) const = 0; + virtual bool canBeParseRequestBody(Poco::Net::HTTPServerRequest &, HTMLForm &) const = 0; + virtual void executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; +}; + +using CustomQueryExecutorPtr = std::shared_ptr; + +class ExtractQueryParamCustomQueryExecutor : public CustomQueryExecutor +{ +public: bool isQueryParam(const String & param_name) const override { return param_name == "query" || startsWith(param_name, "param_"); } + bool canBeParseRequestBody(Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*form*/) const override { return false; } + void executeQueryImpl( - Context & context, HTTPRequest & request, HTTPResponse & response, + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override { const auto & execute_query = prepareQuery(context, params); @@ -42,16 +57,6 @@ public: ); } - static CustomExecutorPtr createDefaultCustomExecutor() - { - const auto & default_custom_executor = std::make_shared(); - - std::vector custom_matchers{default_custom_executor}; - std::vector custom_query_executors{default_custom_executor}; - - return std::make_shared(custom_matchers, custom_query_executors); - } - private: String prepareQuery(Context & context, HTMLForm & params) const { diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 8f460b50db7..64138a6d7d0 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -244,7 +244,7 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response) { const auto & name_with_custom_executor = context.getCustomExecutor(request, params); - LOG_TRACE(log, "Using " << name_with_custom_executor.first << " to execute URI: " << request.getURI()); + LOG_TRACE(log, "Using '" << name_with_custom_executor.first << "' CustomExecutor to execute URI: " << request.getURI()); ExtractorClientInfo{context.getClientInfo()}.extract(request); ExtractorContextChange{context, name_with_custom_executor.second}.extract(request, params); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 39c7953b29a..b479a7bc46d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2059,7 +2059,7 @@ void Context::setCustomExecutorConfig(const ConfigurationPtr & config, const Str shared->custom_executors->updateCustomExecutors(*shared->custom_executors_config, settings, config_prefix); } -std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const +std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) { std::lock_guard lock(shared->custom_executors_mutex); @@ -2069,7 +2069,7 @@ std::pair Context::getCustomExecutor(Poco::Net::HTTPS shared->custom_executors = std::make_unique(config, settings); } - return shared->custom_executors->getCustomExecutor(request, params); + return shared->custom_executors->getCustomExecutor(*this, request, params); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c73d03179c9..2d3ce2f9962 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -42,7 +42,7 @@ namespace zkutil class ZooKeeper; } -class HTMLForm; +struct HTMLForm; namespace DB { @@ -492,7 +492,7 @@ public: Compiler & getCompiler(); void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "CustomHTTP"); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params) const; + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params); /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From 6b716e511bab5983475e101dd975c2ff4d478b7d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 7 Nov 2019 11:51:11 +0800 Subject: [PATCH 403/743] ISSUES-5436 support custom http [part 6] --- .../CustomHTTP/CustomExecutor.cpp | 36 +++++- .../Interpreters/CustomHTTP/CustomExecutor.h | 7 +- .../CustomHTTP/CustomExecutorMatchers.h | 91 +++++++++------ .../CustomHTTP/CustomQueryExecutors.h | 52 +++++++++ dbms/src/Interpreters/QueryParameterVisitor.h | 35 ++++++ programs/server/HTTPHandler.cpp | 10 +- programs/server/config.xml | 10 +- src/IO/WriteBufferFromHTTPServerResponse.cpp | 107 +++++++++++------- src/IO/WriteBufferFromHTTPServerResponse.h | 2 + src/Interpreters/Context.h | 2 +- .../ReplaceQueryParameterVisitor.cpp | 6 +- 11 files changed, 271 insertions(+), 87 deletions(-) create mode 100644 dbms/src/Interpreters/QueryParameterVisitor.h diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp index c37da93bcaf..981d243ef35 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -9,6 +9,8 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; + extern const int DUPLICATE_CUSTOM_EXECUTOR; + extern const int TOO_MANY_INPUT_CUSTOM_EXECUTOR; } bool CustomExecutor::match(Context & context, HTTPRequest & request, HTMLForm & params) const @@ -69,7 +71,7 @@ static CustomExecutorPtr createDefaultCustomExecutor() return std::make_shared(custom_matchers, custom_query_executors); } -void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & /*settings*/, const String & config_prefix) { Configuration::Keys custom_executors_keys; config.keys(config_prefix, custom_executors_keys); @@ -83,6 +85,11 @@ void CustomExecutors::updateCustomExecutors(const Configuration & config, const else if (custom_executor_key.find('.') != String::npos) throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); + const auto & exists_executor = [&](auto & ele) { return ele.first == custom_executor_key; }; + if (std::count_if(new_custom_executors.begin(), new_custom_executors.end(), exists_executor)) + throw Exception("CustomExecutor name '" + custom_executor_key + "' already exists in system.", + ErrorCodes::DUPLICATE_CUSTOM_EXECUTOR); + new_custom_executors.push_back( std::make_pair(custom_executor_key, createCustomExecutor(config, config_prefix + "." + custom_executor_key))); } @@ -146,12 +153,23 @@ CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & co custom_query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); } - for (const auto & custom_executor_matcher : custom_executor_matchers) - custom_executor_matcher->checkQueryExecutor(custom_query_executors); - + checkCustomMatchersAndQueryExecutors(custom_executor_matchers, custom_query_executors); return std::make_shared(custom_executor_matchers, custom_query_executors); } +void CustomExecutors::checkCustomMatchersAndQueryExecutors( + std::vector & matchers, std::vector & query_executors) +{ + const auto & sum_func = [&](auto & ele) { return !ele->canBeParseRequestBody(); }; + const auto & need_post_data_count = std::count_if(query_executors.begin(), query_executors.end(), sum_func); + + if (need_post_data_count > 1) + throw Exception("The CustomExecutor can only contain one insert query.", ErrorCodes::TOO_MANY_INPUT_CUSTOM_EXECUTOR); + + for (const auto & matcher : matchers) + matcher->checkQueryExecutors(query_executors); +} + std::pair CustomExecutors::getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const { std::shared_lock lock(rwlock); @@ -165,8 +183,14 @@ std::pair CustomExecutors::getCustomExecutor(Context CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) { - registerCustomMatcher("URL", [&](const auto & config, const auto & prefix) - { return std::make_shared(config, prefix); }); + registerCustomMatcher("URL", [&](const auto & matcher_config, const auto & prefix) + { return std::make_shared(matcher_config, prefix); }); + + registerCustomMatcher("method", [&](const auto & matcher_config, const auto & prefix) + { return std::make_shared(matcher_config, prefix); }); + + registerQueryExecutor("query", [&](const auto & matcher_config, const auto & prefix) + { return std::make_shared(matcher_config, prefix); }); updateCustomExecutors(config, settings, config_prefix); } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index 5b5ccc53850..94d1fe28923 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -28,7 +28,7 @@ class CustomExecutors { public: using Configuration = Poco::Util::AbstractConfiguration; - CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix = "CustomHTTP"); + CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix = "custom_http"); CustomExecutors(const CustomExecutors &) = delete; CustomExecutors & operator=(const CustomExecutors &) = delete; @@ -36,7 +36,7 @@ public: using QueryExecutorCreator = std::function; void registerQueryExecutor(const String & query_executor_name, const QueryExecutorCreator & creator); - using CustomMatcherCreator = const std::function; + using CustomMatcherCreator = std::function; void registerCustomMatcher(const String & matcher_name, const CustomMatcherCreator & creator); void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); @@ -49,6 +49,9 @@ private: std::unordered_map custom_matcher_creators; CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix); + + void checkCustomMatchersAndQueryExecutors(std::vector & matchers, std::vector & query_executors); + }; class CustomExecutor diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h index 79a48f069ca..825b3c0b268 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h @@ -22,7 +22,7 @@ class CustomExecutorMatcher public: virtual ~CustomExecutorMatcher() = default; - virtual bool checkQueryExecutor(const std::vector & check_executors) const = 0; + virtual bool checkQueryExecutors(const std::vector &check_executors) const = 0; virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; }; @@ -33,63 +33,88 @@ using CustomExecutorMatcherPtr = std::shared_ptr; class AlwaysMatchedCustomExecutorMatcher : public CustomExecutorMatcher { public: - bool checkQueryExecutor(const std::vector & /*check_executors*/) const override { return true; } + bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } }; +class HTTPMethodCustomExecutorMatcher : public CustomExecutorMatcher +{ +public: + + HTTPMethodCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & method_config_key) + { + match_method = Poco::toLower(configuration.getString(method_config_key)); + } + + bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override + { + return Poco::toLower(request.getMethod()) == match_method; + } + +private: + String match_method; +}; + class HTTPURLCustomExecutorMatcher : public CustomExecutorMatcher { public: HTTPURLCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & url_config_key) - : url_match_searcher(analyzeURLPatten(configuration.getString(url_config_key, ""), params_name_extract_from_url)) { + regex_matcher = std::make_unique(configuration.getString(url_config_key)); } - bool checkQueryExecutor(const std::vector & custom_query_executors) const override + bool checkQueryExecutors(const std::vector & custom_query_executors) const override { - for (const auto & param_name_from_url : params_name_extract_from_url) - { - bool found_param_name = false; - for (const auto & custom_query_executor : custom_query_executors) - { - if (custom_query_executor->isQueryParam(param_name_from_url)) - { - found_param_name = true; - break; - } - } - - if (!found_param_name) - throw Exception("The param name '" + param_name_from_url + "' is uselessed.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - } + for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) + if (!checkQueryExecutors(named_capturing_group.first, custom_query_executors)) + throw Exception("The param name '" + named_capturing_group.first + "' is uselessed.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return true; } - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override + bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override { const String request_uri = request.getURI(); - re2_st::StringPiece query_params_matches[params_name_extract_from_url.size()]; + int num_captures = regex_matcher->NumberOfCapturingGroups() + 1; -// re2_st::StringPiece input; -// if (url_match_searcher.Match(input, start_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, query_params_matches, num_captures)) -// { -// -// } + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(request_uri.data(), request_uri.size()); + if (regex_matcher->Match(input, 0, request_uri.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + { + const auto & full_match = matches[0]; + const char * url_end = request_uri.data() + request_uri.size(); + const char * not_matched_begin = request_uri.data() + full_match.size(); + + if (not_matched_begin != url_end && *not_matched_begin == '/') + ++not_matched_begin; + + if (not_matched_begin == url_end || *not_matched_begin == '?') + { + for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) + { + const auto & capturing_value = matches[named_capturing_group.second]; + context.setQueryParameter(named_capturing_group.first, String(capturing_value.data(), capturing_value.size())); + } + + return true; + } + } return false; } private: - re2_st::RE2 url_match_searcher; - std::vector params_name_extract_from_url; + std::unique_ptr regex_matcher; - String analyzeURLPatten(const String & /*url_patten*/, std::vector & /*matches*/) + bool checkQueryExecutors(const String & param_name, const std::vector & custom_query_executors) const { - return ".+"; - /// TODO: first we replace all capture group - /// TODO: second we replace all ${identifier} + for (const auto & custom_query_executor : custom_query_executors) + if (custom_query_executor->isQueryParam(param_name)) + return true; + + return false; } }; diff --git a/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h index 0473148b89c..f986db6d41d 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h @@ -6,10 +6,14 @@ #include #include #include +#include #include #include #include #include +#include +#include +#include namespace DB { @@ -29,6 +33,54 @@ public: using CustomQueryExecutorPtr = std::shared_ptr; +class ConstQueryCustomQueryExecutor : public CustomQueryExecutor +{ +public: + ConstQueryCustomQueryExecutor(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) + { + execute_query = configuration.getString(config_key, ""); + + const char * query_begin = execute_query.data(); + const char * query_end = execute_query.data() + execute_query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); + + QueryParameterVisitor{query_params_name}.visit(extract_query_ast); + can_be_parse_request_body = !extract_query_ast->as(); + } + + bool isQueryParam(const String & param_name) const override { return query_params_name.count(param_name); } + + bool canBeParseRequestBody(Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*form*/) const override { return can_be_parse_request_body; } + + void executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & /*request*/, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & /*input_streams*/, const HTTPOutputStreams & output_streams) const override + { + prepareQueryParams(context, params); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); + } + +private: + String execute_query; + NameSet query_params_name; + bool can_be_parse_request_body{false}; + + void prepareQueryParams(Context & context, HTMLForm & params) const + { + for (const auto & param : params) + if (isQueryParam(param.first)) + context.setQueryParameter(param.first, param.second); + } +}; + class ExtractQueryParamCustomQueryExecutor : public CustomQueryExecutor { public: diff --git a/dbms/src/Interpreters/QueryParameterVisitor.h b/dbms/src/Interpreters/QueryParameterVisitor.h new file mode 100644 index 00000000000..d765aa00bea --- /dev/null +++ b/dbms/src/Interpreters/QueryParameterVisitor.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class QueryParameterVisitor +{ +public: + QueryParameterVisitor(NameSet & parameters_name) : query_parameters(parameters_name) {} + + void visit(const ASTPtr & ast) + { + for (const auto & child : ast->children) + { + if (const auto & query_parameter = child->as()) + visitQueryParameter(*query_parameter); + else + visit(child); + } + } + +private: + NameSet & query_parameters; + + void visitQueryParameter(const ASTQueryParameter & query_parameter) + { + query_parameters.insert(query_parameter.name); + } +}; + +} diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 64138a6d7d0..6ceaf7d7d3f 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -267,10 +267,14 @@ void HTTPHandler::trySendExceptionToClient( && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) request.stream().ignore(std::numeric_limits::max()); - if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::REQUIRED_PASSWORD || exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD + || exception_code == ErrorCodes::REQUIRED_PASSWORD || exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) { - response.requireAuthentication("ClickHouse server HTTP API"); + if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) + response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); + else + response.requireAuthentication("ClickHouse server HTTP API"); + response.send() << message << std::endl; } else diff --git a/programs/server/config.xml b/programs/server/config.xml index fb2f9be6e24..5111e6102c2 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -512,7 +512,6 @@ --> /var/lib/clickhouse/format_schemas/ - + + diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index 252a32a5d7c..c7b3abb008b 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -31,11 +31,6 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() response.set("Access-Control-Allow-Origin", "*"); setResponseDefaultHeaders(response, keep_alive_timeout); - -#if defined(POCO_CLICKHOUSE_PATCH) - if (request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) - std::tie(response_header_ostr, response_body_ostr) = response.beginSend(); -#endif } } @@ -96,46 +91,79 @@ void WriteBufferFromHTTPServerResponse::finishSendHeaders() } +void WriteBufferFromHTTPServerResponse::choiceSendEncode() +{ + if (!out && request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) + { + if (!compress) + { +#if defined(POCO_CLICKHOUSE_PATCH) + *response_header_ostr << "Content-Encoding: gzip\r\n"; +#else + response.set("Content-Encoding", "gzip"); + response_body_ostr = &(response.send()); +#endif + out_raw = std::make_unique(*response_body_ostr); + deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); + out = &*deflating_buf; + } + else if (compression_method == CompressionMethod::Zlib) + { +#if defined(POCO_CLICKHOUSE_PATCH) + *response_header_ostr << "Content-Encoding: deflate\r\n"; +#else + /// Newline autosent by response.send() + response_body_ostr = &(response.send()); +#endif + out_raw = std::make_unique(*response_body_ostr); + deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); + out = &*deflating_buf; + } +#if USE_BROTLI + else if (compression_method == CompressionMethod::Brotli) + { +#if defined(POCO_CLICKHOUSE_PATCH) + std::tie(response_header_ostr, response_body_ostr) = response.beginSend(); + if (headers_started_sending && !headers_finished_sending) + *response_header_ostr << "Content-Encoding: " << encoding_type << "\r\n"; +#else + response.set("Content-Encoding", content_encoding_name); +#endif + + /// Newline autosent by response.send() + /// This may result in an extra empty line in the response body + response_body_ostr = &(response.send()); +#endif + }; + + out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); + out = &*out_raw; + } +#if USE_BROTLI + else if (compression_method == CompressionMethod::Brotli) + { + set_encoding_type("br"); + out_raw.emplace(*response_body_ostr); + brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin()); + out = &*brotli_buf; + } +#endif + else + throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse", + ErrorCodes::LOGICAL_ERROR); + /// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy. + } + } +} + + void WriteBufferFromHTTPServerResponse::nextImpl() { { std::lock_guard lock(mutex); startSendHeaders(); - - if (!out && request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) - { - if (compress) - { - auto content_encoding_name = toContentEncodingName(compression_method); - -#if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: " << content_encoding_name << "\r\n"; -#else - response.set("Content-Encoding", content_encoding_name); -#endif - } - -#if !defined(POCO_CLICKHOUSE_PATCH) - response_body_ostr = &(response.send()); -#endif - - /// We reuse our buffer in "out" to avoid extra allocations and copies. - - if (compress) - out = wrapWriteBufferWithCompressionMethod( - std::make_unique(*response_body_ostr), - compress ? compression_method : CompressionMethod::None, - compression_level, - working_buffer.size(), - working_buffer.begin()); - else - out = std::make_unique( - *response_body_ostr, - working_buffer.size(), - working_buffer.begin()); - } - + choiceSendEncode(); finishSendHeaders(); } @@ -207,7 +235,6 @@ void WriteBufferFromHTTPServerResponse::finalize() } } - WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { try diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index 638a1f69dee..fd57279646f 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -87,6 +87,8 @@ private: /// This method finish headers with \r\n, allowing to start to send body. void finishSendHeaders(); + void choiceSendEncode(); + void nextImpl() override; public: diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2d3ce2f9962..bef448a2389 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -491,7 +491,7 @@ public: Compiler & getCompiler(); - void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "CustomHTTP"); + void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "custom_http"); std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params); /// Call after initialization before using system logs. Call for global context. diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 5c29c722f88..98dc90d9dda 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -55,7 +56,10 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) IColumn & temp_column = *temp_column_ptr; ReadBufferFromString read_buffer{value}; FormatSettings format_settings; - data_type->deserializeAsTextEscaped(temp_column, read_buffer, format_settings); + + skipWhitespaceIfAny(read_buffer); /// Skip white space on both sides + data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings); + skipWhitespaceIfAny(read_buffer); /// Skip white space on both sides if (!read_buffer.eof()) throw Exception("Value " + value + " cannot be parsed as " + type_name + " for query parameter '" + ast_param.name + "'" From 07ed4ba4f4ff44d5985803f720d6965f22524d28 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 7 Nov 2019 16:30:47 +0800 Subject: [PATCH 404/743] ISSUES-5436 add integration test for custom http --- .../test_custom_http/configs/custom_http.xml | 11 ---- .../configs/custom_http_config.xml | 54 +++++++++++++++++++ .../integration/test_custom_http/test.py | 22 +++----- 3 files changed, 60 insertions(+), 27 deletions(-) delete mode 100644 dbms/tests/integration/test_custom_http/configs/custom_http.xml create mode 100644 dbms/tests/integration/test_custom_http/configs/custom_http_config.xml diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http.xml b/dbms/tests/integration/test_custom_http/configs/custom_http.xml deleted file mode 100644 index 22d570b7bca..00000000000 --- a/dbms/tests/integration/test_custom_http/configs/custom_http.xml +++ /dev/null @@ -1,11 +0,0 @@ - - 8123 - - - /${database}/a/${id}/${table} - INSERT INTO ${database:ASTIdentifier}.${table:ASTIdentifier}(id) VALUES - SELECT * FROM ${database:ASTIdenfier}.t - SELECT * FROM a.${table:ASTIdenfier} WHERE id={id:UInt8} - - - diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml new file mode 100644 index 00000000000..f91e196abab --- /dev/null +++ b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml @@ -0,0 +1,54 @@ + + 8123 + + + + /test_for_single_insert + INSERT INTO test.test VALUES + + + + /test_for_single_select + SELECT * FROM test.test ORDER BY id + + + + /test_for_multiple_select + SELECT * FROM test.test ORDER BY id + SELECT * FROM test.test ORDER BY id + + + + /test_for_multiple_select + INSERT INTO test.test VALUES + SELECT 'test_for_hybrid_insert_and_select_with_one_insert first' + SELECT 'test_for_hybrid_insert_and_select_with_one_insert second' + + + + + + /test_for_url_match + SELECT 'Matched test_for_url_match' + + + + PUT + SELECT 'Matched test_for_method_match' + + + + /test_for_multiple_match + GET + SELECT 'Matched test_for_multiple_match' + + + diff --git a/dbms/tests/integration/test_custom_http/test.py b/dbms/tests/integration/test_custom_http/test.py index d8f79e64e3d..805c6163839 100644 --- a/dbms/tests/integration/test_custom_http/test.py +++ b/dbms/tests/integration/test_custom_http/test.py @@ -1,31 +1,21 @@ import pytest -import requests from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', main_configs=['configs/custom_http.xml']) +test_instance = cluster.add_instance('node', main_configs=['configs/custom_http_config.xml']) + @pytest.fixture(scope="module") def start_cluster(): try: cluster.start() - node.query(''' -CREATE DATABASE `test`; - -CREATE TABLE `test`.`test_custom_http` (`id` UInt8) Engine=Memory; - ''') - + test_instance.query('CREATE DATABASE `test`') + test_instance.query('CREATE TABLE `test`.`test` (`id` UInt8) Engine = Memory') yield cluster finally: cluster.shutdown() -def test(started_cluster): - node_ip = cluster.get_instance_ip(node) - url = 'http://%s:8123/test/a/1/test_custom_http' % node_ip - data="(1)" - params = {'id':1} - response = requests.post(url, params = params, data = data) - assert response.text == '\n1\n1\n' +def test_for_single_insert(started_cluster): + assert test_instance.http_query('/test_for_single_insert', data='(1)(2)(3)') == '\n' From 847f7ab4396f4e21ac3c07f13ab9629d01011b33 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 7 Nov 2019 18:24:30 +0800 Subject: [PATCH 405/743] ISSUES-5436 fix build failure & fix test failure --- .../CustomHTTP/AlwaysQueryMatcher.h | 16 +++ .../CustomHTTP/ConstQueryExecutor.cpp | 73 ++++++++++ .../CustomHTTP/ConstQueryExecutor.h | 30 ++++ .../CustomHTTP/CustomExecutor.cpp | 135 +++++++++--------- .../Interpreters/CustomHTTP/CustomExecutor.h | 27 ++-- .../CustomHTTP/CustomExecutorMatchers.h | 122 ---------------- .../CustomHTTP/CustomQueryExecutors.h | 131 ----------------- .../CustomHTTP/DynamicQueryExecutor.cpp | 60 ++++++++ .../CustomHTTP/DynamicQueryExecutor.h | 37 +++++ .../CustomHTTP/ExtractorContextChange.h | 117 +++++++++------ .../CustomHTTP/HTTPInputStreams.cpp | 2 + .../CustomHTTP/HTTPOutputStreams.cpp | 44 ++---- .../CustomHTTP/HTTPOutputStreams.h | 8 +- .../CustomHTTP/MethodQueryMatcher.h | 30 ++++ .../CustomHTTP/QueryExecutorAndMatcher.h | 47 ++++++ .../CustomHTTP/URLQueryMatcher.cpp | 73 ++++++++++ .../Interpreters/CustomHTTP/URLQueryMatcher.h | 32 +++++ .../configs/custom_http_config.xml | 4 +- programs/server/HTTPHandler.cpp | 103 +++++++++---- programs/server/HTTPHandler.h | 14 +- programs/server/config.xml | 11 +- src/IO/WriteBufferFromHTTPServerResponse.cpp | 64 ++++----- src/IO/WriteBufferFromHTTPServerResponse.h | 5 +- .../ReplaceQueryParameterVisitor.cpp | 2 - 24 files changed, 691 insertions(+), 496 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h create mode 100644 dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h create mode 100644 dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h create mode 100644 dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h create mode 100644 dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h create mode 100644 dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h diff --git a/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h new file mode 100644 index 00000000000..56fa001d57c --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h @@ -0,0 +1,16 @@ +#pragma once + +#include + +namespace DB +{ + +class AlwaysQueryMatcher : public QueryMatcher +{ +public: + bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp new file mode 100644 index 00000000000..e6ed579b696 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp @@ -0,0 +1,73 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_CUSTOM_EXECUTOR_PARAM; +} + +void prepareQueryParams(Context & context, HTMLForm & params, const NameSet & query_params_name) +{ + for (const auto & param : params) + if (query_params_name.count(param.first)) + context.setQueryParameter(param.first, param.second); +} + +QueryExecutorConst::QueryExecutorConst(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) +{ + execute_query = configuration.getString(config_key); + + const char * query_begin = execute_query.data(); + const char * query_end = execute_query.data() + execute_query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); + + QueryParameterVisitor{query_params_name}.visit(extract_query_ast); + can_be_parse_request_body = !extract_query_ast->as(); + + const auto & reserved_params_name = ExtractorContextChange::getReservedParamNames(); + for (const auto & prepared_param_name : query_params_name) + { + if (Settings::findIndex(prepared_param_name) != Settings::npos || reserved_params_name.count(prepared_param_name)) + throw Exception( + "Illegal custom executor query param name '" + prepared_param_name + "', Because it's a reserved name or Settings name", + ErrorCodes::ILLEGAL_CUSTOM_EXECUTOR_PARAM); + } +} + +void QueryExecutorConst::executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const +{ + ReadBufferPtr temp_query_buf; + prepareQueryParams(context, params, query_params_name); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + if (!canBeParseRequestBody() && !startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h new file mode 100644 index 00000000000..170d2a959ef --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h @@ -0,0 +1,30 @@ +#pragma once + +#include + +namespace DB +{ + +class QueryExecutorConst : public QueryExecutor +{ +public: + using HTTPServerRequest = Poco::Net::HTTPServerRequest; + using HTTPServerResponse = Poco::Net::HTTPServerResponse; + + bool canBeParseRequestBody() const override { return can_be_parse_request_body; } + + bool isQueryParam(const String & param_name) const override { return query_params_name.count(param_name); } + + QueryExecutorConst(const Poco::Util::AbstractConfiguration & configuration, const String & config_key); + + void executeQueryImpl( + Context & context, HTTPServerRequest & request, HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override; + +private: + String execute_query; + NameSet query_params_name; + bool can_be_parse_request_body{false}; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp index 981d243ef35..d3d0bfc5aeb 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -1,6 +1,10 @@ #include #include -#include "CustomExecutor.h" +#include +#include +#include +#include +#include namespace DB @@ -9,10 +13,17 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; - extern const int DUPLICATE_CUSTOM_EXECUTOR; + extern const int UNKNOW_QUERY_EXECUTOR; extern const int TOO_MANY_INPUT_CUSTOM_EXECUTOR; } +CustomExecutor::CustomExecutor( + const std::vector & matchers_, + const std::vector & query_executors_) + : matchers(matchers_), query_executors(query_executors_) +{ +} + bool CustomExecutor::match(Context & context, HTTPRequest & request, HTMLForm & params) const { for (const auto & matcher : matchers) @@ -35,11 +46,11 @@ bool CustomExecutor::isQueryParam(const String & param_name) const return true; } -bool CustomExecutor::canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const +bool CustomExecutor::canBeParseRequestBody() const { for (const auto & query_executor : query_executors) { - if (!query_executor->canBeParseRequestBody(request, params)) + if (!query_executor->canBeParseRequestBody()) return false; } @@ -57,20 +68,6 @@ void CustomExecutor::executeQuery( output_streams.finalize(); } -CustomExecutor::CustomExecutor( - const std::vector & matchers_, const std::vector & query_executors_) - : matchers(matchers_), query_executors(query_executors_) -{ -} - -static CustomExecutorPtr createDefaultCustomExecutor() -{ - std::vector custom_matchers{std::make_shared()}; - std::vector custom_query_executors{std::make_shared()}; - - return std::make_shared(custom_matchers, custom_query_executors); -} - void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & /*settings*/, const String & config_prefix) { Configuration::Keys custom_executors_keys; @@ -80,91 +77,91 @@ void CustomExecutors::updateCustomExecutors(const Configuration & config, const for (const auto & custom_executor_key : custom_executors_keys) { - if (custom_executor_key == "Default") - throw Exception("CustomExecutor cannot be 'Default'.", ErrorCodes::SYNTAX_ERROR); - else if (custom_executor_key.find('.') != String::npos) + if (custom_executor_key.find('.') != String::npos) throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); - const auto & exists_executor = [&](auto & ele) { return ele.first == custom_executor_key; }; - if (std::count_if(new_custom_executors.begin(), new_custom_executors.end(), exists_executor)) - throw Exception("CustomExecutor name '" + custom_executor_key + "' already exists in system.", - ErrorCodes::DUPLICATE_CUSTOM_EXECUTOR); - - new_custom_executors.push_back( - std::make_pair(custom_executor_key, createCustomExecutor(config, config_prefix + "." + custom_executor_key))); + new_custom_executors.push_back({custom_executor_key, createCustomExecutor(config, config_prefix, custom_executor_key)}); } - new_custom_executors.push_back(std::make_pair("Default", createDefaultCustomExecutor())); - std::unique_lock lock(rwlock); custom_executors = new_custom_executors; } -void CustomExecutors::registerQueryExecutor(const String & query_executor_name, const CustomExecutors::QueryExecutorCreator & creator) -{ - const auto & matcher_creator_it = custom_matcher_creators.find(query_executor_name); - const auto & query_executor_creator_it = query_executor_creators.find(query_executor_name); - - if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) - throw Exception("LOGICAL_ERROR CustomQueryExecutor name must be unique between the CustomQueryExecutor and CustomExecutorMatcher.", - ErrorCodes::LOGICAL_ERROR); - - query_executor_creators[query_executor_name] = creator; -} - void CustomExecutors::registerCustomMatcher(const String & matcher_name, const CustomExecutors::CustomMatcherCreator & creator) { const auto & matcher_creator_it = custom_matcher_creators.find(matcher_name); const auto & query_executor_creator_it = query_executor_creators.find(matcher_name); if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) - throw Exception("LOGICAL_ERROR CustomExecutorMatcher name must be unique between the CustomQueryExecutor and CustomExecutorMatcher.", - ErrorCodes::LOGICAL_ERROR); + throw Exception("LOGICAL_ERROR QueryMatcher name must be unique between the QueryExecutor and QueryMatcher.", + ErrorCodes::LOGICAL_ERROR); custom_matcher_creators[matcher_name] = creator; } -CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & config, const String & config_prefix) +void CustomExecutors::registerQueryExecutor(const String & query_executor_name, const CustomExecutors::QueryExecutorCreator & creator) { - Configuration::Keys matchers_or_query_executors_type; - config.keys(config_prefix, matchers_or_query_executors_type); + const auto & matcher_creator_it = custom_matcher_creators.find(query_executor_name); + const auto & query_executor_creator_it = query_executor_creators.find(query_executor_name); - std::vector custom_query_executors; - std::vector custom_executor_matchers; + if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) + throw Exception("LOGICAL_ERROR QueryExecutor name must be unique between the QueryExecutor and QueryMatcher.", + ErrorCodes::LOGICAL_ERROR); - for (const auto & matcher_or_query_executor_type : matchers_or_query_executors_type) + query_executor_creators[query_executor_name] = creator; +} + +String fixMatcherOrExecutorTypeName(const String & matcher_or_executor_type_name) +{ + auto type_name_end_pos = matcher_or_executor_type_name.find('['); + return type_name_end_pos == String::npos ? matcher_or_executor_type_name : matcher_or_executor_type_name.substr(0, type_name_end_pos); +} + +CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & config, const String & config_prefix, const String & name) +{ + Configuration::Keys matchers_key; + config.keys(config_prefix + "." + name, matchers_key); + + std::vector query_matchers; + std::vector query_executors; + + for (const auto & matcher_key : matchers_key) { + String matcher_or_query_executor_type = fixMatcherOrExecutorTypeName(matcher_key); + if (matcher_or_query_executor_type.find('.') != String::npos) - throw Exception( - "CustomMatcher or CustomQueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", + throw Exception("CustomMatcher or QueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", ErrorCodes::SYNTAX_ERROR); const auto & matcher_creator_it = custom_matcher_creators.find(matcher_or_query_executor_type); const auto & query_executor_creator_it = query_executor_creators.find(matcher_or_query_executor_type); if (matcher_creator_it == custom_matcher_creators.end() && query_executor_creator_it == query_executor_creators.end()) - throw Exception("CustomMatcher or CustomQueryExecutor '" + matcher_or_query_executor_type + "' is not implemented.", - ErrorCodes::NOT_IMPLEMENTED); + throw Exception("CustomMatcher or QueryExecutor '" + matcher_or_query_executor_type + "' is not implemented.", + ErrorCodes::NOT_IMPLEMENTED); if (matcher_creator_it != custom_matcher_creators.end()) - custom_executor_matchers.push_back(matcher_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); + query_matchers.push_back(matcher_creator_it->second(config, config_prefix + "." + name + "." + matcher_key)); if (query_executor_creator_it != query_executor_creators.end()) - custom_query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); + query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + name + "." + matcher_key)); } - checkCustomMatchersAndQueryExecutors(custom_executor_matchers, custom_query_executors); - return std::make_shared(custom_executor_matchers, custom_query_executors); + checkQueryMatchersAndExecutors(name, query_matchers, query_executors); + return std::make_shared(query_matchers, query_executors); } -void CustomExecutors::checkCustomMatchersAndQueryExecutors( - std::vector & matchers, std::vector & query_executors) +void CustomExecutors::checkQueryMatchersAndExecutors( + const String & name, std::vector & matchers, std::vector & query_executors) { - const auto & sum_func = [&](auto & ele) { return !ele->canBeParseRequestBody(); }; + if (matchers.empty() || query_executors.empty()) + throw Exception("The CustomExecutor '" + name + "' must contain a Matcher and a QueryExecutor.", ErrorCodes::SYNTAX_ERROR); + + const auto & sum_func = [&](auto & ele) -> bool { return !ele->canBeParseRequestBody(); }; const auto & need_post_data_count = std::count_if(query_executors.begin(), query_executors.end(), sum_func); if (need_post_data_count > 1) - throw Exception("The CustomExecutor can only contain one insert query.", ErrorCodes::TOO_MANY_INPUT_CUSTOM_EXECUTOR); + throw Exception("The CustomExecutor '" + name + "' can only contain one insert query." + toString(need_post_data_count), ErrorCodes::TOO_MANY_INPUT_CUSTOM_EXECUTOR); for (const auto & matcher : matchers) matcher->checkQueryExecutors(query_executors); @@ -178,19 +175,25 @@ std::pair CustomExecutors::getCustomExecutor(Context if (custom_executor.second->match(context, request, params)) return custom_executor; - throw Exception("LOGICAL_ERROR not found custom executor.", ErrorCodes::LOGICAL_ERROR); + throw Exception("No query executors matched", ErrorCodes::UNKNOW_QUERY_EXECUTOR); } CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) { registerCustomMatcher("URL", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); + { return std::make_shared(matcher_config, prefix); }); registerCustomMatcher("method", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); + { return std::make_shared(matcher_config, prefix); }); + + registerCustomMatcher("always_matched", [&](const auto & /*matcher_config*/, const auto & /*prefix*/) + { return std::make_shared(); }); registerQueryExecutor("query", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); + { return std::make_shared(matcher_config, prefix); }); + + registerQueryExecutor("dynamic_query", [&](const auto & matcher_config, const auto & prefix) + { return std::make_shared(matcher_config, prefix); }); updateCustomExecutors(config, settings, config_prefix); } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index 94d1fe28923..7b37a6afdf5 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -7,14 +7,13 @@ #include #include #include -#include -#include -#include #include #include #include -#include -#include +#include +#include +#include +#include namespace DB { @@ -33,10 +32,10 @@ public: CustomExecutors(const CustomExecutors &) = delete; CustomExecutors & operator=(const CustomExecutors &) = delete; - using QueryExecutorCreator = std::function; + using QueryExecutorCreator = std::function; void registerQueryExecutor(const String & query_executor_name, const QueryExecutorCreator & creator); - using CustomMatcherCreator = std::function; + using CustomMatcherCreator = std::function; void registerCustomMatcher(const String & matcher_name, const CustomMatcherCreator & creator); void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); @@ -48,18 +47,18 @@ private: std::unordered_map query_executor_creators; std::unordered_map custom_matcher_creators; - CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix); + CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix, const String & name); - void checkCustomMatchersAndQueryExecutors(std::vector & matchers, std::vector & query_executors); + void checkQueryMatchersAndExecutors(const String & name, std::vector & matchers, std::vector & query_executors); }; class CustomExecutor { public: - bool isQueryParam(const String & param_name) const; + bool canBeParseRequestBody() const; - bool canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const; + bool isQueryParam(const String & param_name) const; bool match(Context & context, HTTPRequest & request, HTMLForm & params) const; @@ -68,11 +67,11 @@ public: HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams ); - CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); + CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); private: - std::vector matchers; - std::vector query_executors; + std::vector matchers; + std::vector query_executors; }; } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h deleted file mode 100644 index 825b3c0b268..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h +++ /dev/null @@ -1,122 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - - -#if USE_RE2_ST -# include -#else -# define re2_st re2 -#endif - -namespace DB -{ - -class CustomExecutorMatcher -{ -public: - virtual ~CustomExecutorMatcher() = default; - - virtual bool checkQueryExecutors(const std::vector &check_executors) const = 0; - - virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; -}; - -using CustomExecutorMatcherPtr = std::shared_ptr; - - -class AlwaysMatchedCustomExecutorMatcher : public CustomExecutorMatcher -{ -public: - bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } - - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } -}; - -class HTTPMethodCustomExecutorMatcher : public CustomExecutorMatcher -{ -public: - - HTTPMethodCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & method_config_key) - { - match_method = Poco::toLower(configuration.getString(method_config_key)); - } - - bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } - - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override - { - return Poco::toLower(request.getMethod()) == match_method; - } - -private: - String match_method; -}; - -class HTTPURLCustomExecutorMatcher : public CustomExecutorMatcher -{ -public: - HTTPURLCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & url_config_key) - { - regex_matcher = std::make_unique(configuration.getString(url_config_key)); - } - - bool checkQueryExecutors(const std::vector & custom_query_executors) const override - { - for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) - if (!checkQueryExecutors(named_capturing_group.first, custom_query_executors)) - throw Exception("The param name '" + named_capturing_group.first + "' is uselessed.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return true; - } - - bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override - { - const String request_uri = request.getURI(); - int num_captures = regex_matcher->NumberOfCapturingGroups() + 1; - - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(request_uri.data(), request_uri.size()); - if (regex_matcher->Match(input, 0, request_uri.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) - { - const auto & full_match = matches[0]; - const char * url_end = request_uri.data() + request_uri.size(); - const char * not_matched_begin = request_uri.data() + full_match.size(); - - if (not_matched_begin != url_end && *not_matched_begin == '/') - ++not_matched_begin; - - if (not_matched_begin == url_end || *not_matched_begin == '?') - { - for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) - { - const auto & capturing_value = matches[named_capturing_group.second]; - context.setQueryParameter(named_capturing_group.first, String(capturing_value.data(), capturing_value.size())); - } - - return true; - } - } - return false; - } - -private: - std::unique_ptr regex_matcher; - - bool checkQueryExecutors(const String & param_name, const std::vector & custom_query_executors) const - { - for (const auto & custom_query_executor : custom_query_executors) - if (custom_query_executor->isQueryParam(param_name)) - return true; - - return false; - } -}; - - -} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h deleted file mode 100644 index f986db6d41d..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h +++ /dev/null @@ -1,131 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class CustomQueryExecutor -{ -public: - virtual ~CustomQueryExecutor() = default; - - virtual bool isQueryParam(const String &) const = 0; - virtual bool canBeParseRequestBody(Poco::Net::HTTPServerRequest &, HTMLForm &) const = 0; - - virtual void executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; -}; - -using CustomQueryExecutorPtr = std::shared_ptr; - -class ConstQueryCustomQueryExecutor : public CustomQueryExecutor -{ -public: - ConstQueryCustomQueryExecutor(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) - { - execute_query = configuration.getString(config_key, ""); - - const char * query_begin = execute_query.data(); - const char * query_end = execute_query.data() + execute_query.size(); - - ParserQuery parser(query_end, false); - ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); - - QueryParameterVisitor{query_params_name}.visit(extract_query_ast); - can_be_parse_request_body = !extract_query_ast->as(); - } - - bool isQueryParam(const String & param_name) const override { return query_params_name.count(param_name); } - - bool canBeParseRequestBody(Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*form*/) const override { return can_be_parse_request_body; } - - void executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & /*request*/, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & /*input_streams*/, const HTTPOutputStreams & output_streams) const override - { - prepareQueryParams(context, params); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); - } - -private: - String execute_query; - NameSet query_params_name; - bool can_be_parse_request_body{false}; - - void prepareQueryParams(Context & context, HTMLForm & params) const - { - for (const auto & param : params) - if (isQueryParam(param.first)) - context.setQueryParameter(param.first, param.second); - } -}; - -class ExtractQueryParamCustomQueryExecutor : public CustomQueryExecutor -{ -public: - bool isQueryParam(const String & param_name) const override { return param_name == "query" || startsWith(param_name, "param_"); } - - bool canBeParseRequestBody(Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*form*/) const override { return false; } - - void executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override - { - const auto & execute_query = prepareQuery(context, params); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - ReadBufferPtr temp_query_buf; - if (!startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); - } - -private: - String prepareQuery(Context & context, HTMLForm & params) const - { - const static size_t prefix_size = strlen("param_"); - - std::stringstream query_stream; - for (const auto & param : params) - { - if (param.first == "query") - query_stream << param.second; - else if (startsWith(param.first, "param_")) - context.setQueryParameter(param.first.substr(prefix_size), param.second); - } - - query_stream << "\n"; - return query_stream.str(); - } -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp new file mode 100644 index 00000000000..9b4a4006556 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp @@ -0,0 +1,60 @@ +#include + +#include +#include +#include + +namespace DB +{ + +bool QueryExecutorDynamic::isQueryParam(const String & param_name) const +{ + return param_name == dynamic_param_name || startsWith(param_name, "param_"); +} + +QueryExecutorDynamic::QueryExecutorDynamic(const Configuration & configuration, const String & config_key) +{ + dynamic_param_name = configuration.getString(config_key + "." + "param_name", "query"); +} + +void QueryExecutorDynamic::executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const +{ + ReadBufferPtr temp_query_buf; + const auto & execute_query = prepareQuery(context, params); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + if (!startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, + context, [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); +} + +String QueryExecutorDynamic::prepareQuery(Context & context, HTMLForm & params) const +{ + const static size_t prefix_size = strlen("param_"); + + WriteBufferFromOwnString query_buffer; + for (const auto & param : params) + { + if (param.first == dynamic_param_name) + writeString(param.second, query_buffer); + else if (startsWith(param.first, "param_")) + context.setQueryParameter(param.first.substr(prefix_size), param.second); + } + + if (query_buffer.offset()) + writeString("\n", query_buffer); + + return query_buffer.str(); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h new file mode 100644 index 00000000000..a09146246d4 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class QueryExecutorDynamic : public QueryExecutor +{ +public: + using HTTPServerRequest = Poco::Net::HTTPServerRequest; + using HTTPServerResponse = Poco::Net::HTTPServerResponse; + using Configuration = Poco::Util::AbstractConfiguration; + + bool canBeParseRequestBody() const override { return false; } + + bool isQueryParam(const String & param_name) const override; + + QueryExecutorDynamic(const Configuration & configuration, const String & config_key); + + void executeQueryImpl( + Context & context, HTTPServerRequest & request, HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override; + +private: + String dynamic_param_name{"query"}; + + String prepareQuery(Context & context, HTMLForm & params) const; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h index 7b63df33783..56411cb122a 100644 --- a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h +++ b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h @@ -9,44 +9,81 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_HTTP_PARAM; +} + class ExtractorContextChange { public: ExtractorContextChange(Context & context_, const CustomExecutorPtr & executor_) : context(context_), executor(executor_) {} - void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) + static const NameSet & getReservedParamNames() { - Names reserved_param_suffixes; static const NameSet reserved_param_names{ - "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "buffer_size", "wait_end_of_query", - "session_id", "session_timeout", "session_check"}; + "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check" + }; - auto param_could_be_skipped = [&] (const String & name) + return reserved_param_names; + } + + static std::function reservedParamSuffixesFilter(bool reserved) + { + if (!reserved) + return [&](const String &) { return false; }; + + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + return [&](const String & param_name) { - if (reserved_param_names.count(name)) + if (endsWith(param_name, "_format")) + return true; + else if (endsWith(param_name, "_types")) + return true; + else if (endsWith(param_name, "_structure")) return true; - - for (const String & suffix : reserved_param_suffixes) - { - if (endsWith(name, suffix)) - return true; - } return false; }; + } + + void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) + { + bool is_multipart_data = startsWith(request.getContentType().data(), "multipart/form-data"); /// Settings can be overridden in the query. /// Some parameters (database, default_format, everything used in the code above) do not /// belong to the Settings class. + becomeReadonlyIfNeed(request); + changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + if (is_multipart_data || executor->canBeParseRequestBody()) + { + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - /// Only readonly queries are allowed for HTTP GET requests. + /// We use the `Post Request Body Settings` to override the `Qeruy String Param settings` + if (executor->canBeParseRequestBody()) + changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); + } + } + +private: + Context & context; + const CustomExecutorPtr & executor; + + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + + /// In theory if initially readonly = 0, the client can change any setting and then set readonly + /// to some other value. + /// Only readonly queries are allowed for HTTP GET requests. + void becomeReadonlyIfNeed(HTTPRequest & request) + { if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) { Settings & settings = context.getSettingsRef(); @@ -54,45 +91,33 @@ public: if (settings.readonly == 0) settings.readonly = 2; } + } - bool has_multipart = startsWith(request.getContentType().data(), "multipart/form-data"); - - if (has_multipart || executor->canBeParseRequestBody(request, params)) - { - ExternalTablesHandler handler(context, params); - params.load(request, request.stream(), handler); - - if (has_multipart) - { - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.reserve(3); - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); - } - } + void changeSettingsFromParams(HTMLForm & params, const std::function & reserved_param_suffixes) + { SettingsChanges settings_changes; - for (const auto & [key, value] : params) + const auto & reserved_param_names = getReservedParamNames(); + + for (const auto & [name, value] : params) { - if (key == "database") + if (name == "database") context.setCurrentDatabase(value); - else if (key == "default_format") + else if (name == "default_format") context.setDefaultFormat(value); - else if (!param_could_be_skipped(key) && !executor->isQueryParam(key)) - settings_changes.push_back({key, value}); /// All other query parameters are treated as settings. + else if (!reserved_param_names.count(name) && !reserved_param_suffixes(name)) + { + if (Settings::findIndex(name) != Settings::npos) + settings_changes.push_back({name, value}); + else if (!executor->isQueryParam(name)) + throw Exception("Unknown HTTP param name: '" + name + "'", ErrorCodes::UNKNOWN_HTTP_PARAM); + } } /// For external data we also want settings context.checkSettingsConstraints(settings_changes); context.applySettingsChanges(settings_changes); } - -private: - Context & context; - const CustomExecutorPtr & executor; - }; } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp index 7c441891b33..3321b57b38d 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp @@ -1,7 +1,9 @@ +#include #include #include +#include #include #include #include diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp index 113523494cf..8ac682af5e4 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp @@ -52,16 +52,15 @@ namespace } } -HTTPOutputStreams::HTTPOutputStreams(HTTPServerRequest & request, HTTPServerResponse & response, bool internal_compress, size_t keep_alive_timeout) - : out(createResponseOut(request, response, keep_alive_timeout)) +HTTPOutputStreams::HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, bool internal_compress) + : out(raw_out) , out_maybe_compressed(createMaybeCompressionOut(internal_compress, out)) , out_maybe_delayed_and_compressed(out_maybe_compressed) { } -HTTPOutputStreams::HTTPOutputStreams( - Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) - : out(createResponseOut(request, response, keep_alive_timeout)) +HTTPOutputStreams::HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, Context & context, HTTPServerRequest & request, HTMLForm & form) + : out(raw_out) , out_maybe_compressed(createMaybeCompressionOut(form.getParsed("compress", false), out)) , out_maybe_delayed_and_compressed(createMaybeDelayedAndCompressionOut(context, form, out_maybe_compressed)) { @@ -90,36 +89,10 @@ HTTPOutputStreams::HTTPOutputStreams( } } -HTTPResponseBufferPtr HTTPOutputStreams::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive) -{ - /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). - String http_response_compression_methods = request.get("Accept-Encoding", ""); - - if (!http_response_compression_methods.empty()) - { - /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. - /// NOTE parsing of the list of methods is slightly incorrect. - if (std::string::npos != http_response_compression_methods.find("gzip")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); - else if (std::string::npos != http_response_compression_methods.find("deflate")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); -#if USE_BROTLI - else if (http_response_compression_methods == "br") - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); -#endif - } - - return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); -} - WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(bool compression, HTTPResponseBufferPtr & out_) { /// Client can pass a 'compress' flag in the query string. In this case the query result is /// compressed using internal algorithm. This is not reflected in HTTP headers. -// bool internal_compression = form.getParsed("compress", false); return compression ? std::make_shared(*out_) : WriteBufferPtr(out_); } @@ -178,13 +151,18 @@ WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & HTTPOutputStreams::~HTTPOutputStreams() { - /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references + /// This could be a broken HTTP Request + /// Because it does not call finalize or writes some data to output stream after call finalize + /// In this case we need to clean up its broken state to ensure that they are not sent to the client + + /// For delayed stream, we destory CascadeBuffer and without sending any data to client. if (out_maybe_delayed_and_compressed != out_maybe_compressed) out_maybe_delayed_and_compressed.reset(); - /// If buffer has data, and that data wasn't sent yet, then no need to send that data if (out->count() == out->offset()) { + /// If buffer has data and server never sends data to client + /// no need to send that data out_maybe_compressed->position() = out_maybe_compressed->buffer().begin(); out->position() = out->buffer().begin(); } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h index 614acb6fcf3..fba122ea294 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h @@ -34,15 +34,13 @@ struct HTTPOutputStreams void finalize() const; - WriteBufferPtr createMaybeDelayedAndCompressionOut(Context &context, HTMLForm &form, WriteBufferPtr &out_); + WriteBufferPtr createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_); WriteBufferPtr createMaybeCompressionOut(bool compression, std::shared_ptr & out_); - HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive); + HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, bool internal_compress); - HTTPOutputStreams(HTTPServerRequest & request, HTTPServerResponse & response, bool internal_compress, size_t keep_alive_timeout); - - HTTPOutputStreams(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); + HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, Context & context, HTTPServerRequest & request, HTMLForm & form); }; using HTTPOutputStreamsPtr = std::unique_ptr; diff --git a/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h new file mode 100644 index 00000000000..689aff67dc7 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include "QueryExecutorAndMatcher.h" + +namespace DB +{ + +class MethodQueryMatcher : public QueryMatcher +{ +public: + using Config = Poco::Util::AbstractConfiguration; + + MethodQueryMatcher(const Config & configuration, const String & config_key) + : method(Poco::toLower(configuration.getString(config_key))) + { + } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override + { + return Poco::toLower(request.getMethod()) == method; + } + + bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } + +private: + String method; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h b/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h new file mode 100644 index 00000000000..7f32c4c2ddb --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPILE_REGEXP; +} + +class QueryExecutor +{ +public: + virtual ~QueryExecutor() = default; + + virtual bool isQueryParam(const String &) const = 0; + virtual bool canBeParseRequestBody() const = 0; + + virtual void executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; +}; + +using QueryExecutorPtr = std::shared_ptr; + + +class QueryMatcher +{ +public: + virtual ~QueryMatcher() = default; + + virtual bool checkQueryExecutors(const std::vector &check_executors) const = 0; + + virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; +}; + +using QueryMatcherPtr = std::shared_ptr; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp new file mode 100644 index 00000000000..82bb864c926 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp @@ -0,0 +1,73 @@ +#include +#include "URLQueryMatcher.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPILE_REGEXP; + extern const int UNDEFINED_CUSTOM_EXECUTOR_PARAM; +} + +URLQueryMatcher::URLQueryMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) +{ + const auto & regex_str = configuration.getString(config_key); + regex_matcher = std::make_unique(regex_str); + + if (!regex_matcher->ok()) + throw Exception("cannot compile re2: " + regex_str + ", error: " + regex_matcher->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); +} + +bool checkQueryOneQueryParam(const String & param_name, const std::vector & custom_query_executors) +{ + for (const auto & custom_query_executor : custom_query_executors) + if (custom_query_executor->isQueryParam(param_name)) + return true; + + return false; +} + +bool URLQueryMatcher::checkQueryExecutors(const std::vector & custom_query_executors) const +{ + for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) + if (!checkQueryOneQueryParam(named_capturing_group.first, custom_query_executors)) + throw Exception("The param name '" + named_capturing_group.first + "' is not defined in the QueryExecutor.", + ErrorCodes::UNDEFINED_CUSTOM_EXECUTOR_PARAM); + + return true; +} + +bool URLQueryMatcher::match(Context & context, Poco::Net::HTTPServerRequest &request, HTMLForm &) const +{ + const String request_uri = request.getURI(); + int num_captures = regex_matcher->NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(request_uri.data(), request_uri.size()); + if (regex_matcher->Match(input, 0, request_uri.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + { + const auto & full_match = matches[0]; + const char * url_end = request_uri.data() + request_uri.size(); + const char * not_matched_begin = request_uri.data() + full_match.size(); + + if (not_matched_begin != url_end && *not_matched_begin == '/') + ++not_matched_begin; + + if (not_matched_begin == url_end || *not_matched_begin == '?') + { + for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) + { + const auto & capturing_value = matches[named_capturing_group.second]; + context.setQueryParameter(named_capturing_group.first, String(capturing_value.data(), capturing_value.size())); + } + + return true; + } + } + return false; +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h new file mode 100644 index 00000000000..61323db3e53 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + +#include +#include + +#if USE_RE2_ST +# include +#else +# define re2_st re2 +#endif + + +namespace DB +{ + +class URLQueryMatcher : public QueryMatcher +{ +public: + URLQueryMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & config_key); + + bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm &) const override; + + bool checkQueryExecutors(const std::vector & custom_query_executors) const override; + +private: + std::unique_ptr regex_matcher; +}; + +} diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml index f91e196abab..eb56cd6c18c 100644 --- a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml +++ b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml @@ -41,13 +41,13 @@ - PUT + PUT SELECT 'Matched test_for_method_match' - /test_for_multiple_match GET + /test_for_multiple_match SELECT 'Matched test_for_multiple_match' diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 6ceaf7d7d3f..cfbefb94ee4 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -90,6 +90,7 @@ namespace ErrorCodes extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; + extern const int UNKNOW_QUERY_EXECUTOR; } @@ -117,7 +118,8 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti exception_code == ErrorCodes::INCORRECT_DATA || exception_code == ErrorCodes::TYPE_MISMATCH) return HTTPResponse::HTTP_BAD_REQUEST; - else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + else if (exception_code == ErrorCodes::UNKNOW_QUERY_EXECUTOR || + exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_TYPE || @@ -184,18 +186,22 @@ HTTPHandler::SessionContextHolder::~SessionContextHolder() } -HTTPHandler::SessionContextHolder::SessionContextHolder(IServer & accepted_server, HTMLForm & params) +HTTPHandler::SessionContextHolder::SessionContextHolder(Context & query_context_, HTTPRequest & request, HTMLForm & params) + : query_context(query_context_) { - session_id = params.get("session_id", ""); - context = std::make_unique(accepted_server.context()); + authentication(request, params); - if (!session_id.empty()) { - session_timeout = parseSessionTimeout(accepted_server.config(), params); - session_context = context->acquireSession(session_id, session_timeout, params.check("session_check", "1")); + session_id = params.get("session_id", ""); - context = std::make_unique(*session_context); - context->setSessionContext(*session_context); + if (!session_id.empty()) + { + session_timeout = parseSessionTimeout(query_context.getConfigRef(), params); + session_context = query_context.acquireSession(session_id, session_timeout, params.check("session_check", "1")); + + query_context = *session_context; + query_context.setSessionContext(*session_context); + } } } @@ -237,11 +243,11 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque } std::string query_id = params.get("query_id", ""); - context->setUser(user, password, request.clientAddress(), quota_key); - context->setCurrentQueryId(query_id); + query_context.setUser(user, password, request.clientAddress(), quota_key); + query_context.setCurrentQueryId(query_id); } -void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response) +void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response, HTTPResponseBufferPtr & response_out) { const auto & name_with_custom_executor = context.getCustomExecutor(request, params); LOG_TRACE(log, "Using '" << name_with_custom_executor.first << "' CustomExecutor to execute URI: " << request.getURI()); @@ -250,12 +256,13 @@ void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLFor ExtractorContextChange{context, name_with_custom_executor.second}.extract(request, params); HTTPInputStreams input_streams{context, request, params}; - HTTPOutputStreams output_streams = HTTPOutputStreams(context, request, response, params, getKeepAliveTimeout()); + HTTPOutputStreams output_streams = HTTPOutputStreams(response_out, context, request, params); name_with_custom_executor.second->executeQuery(context, request, response, params, input_streams, output_streams); } void HTTPHandler::trySendExceptionToClient( - const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, bool compression) + const std::string & message, int exception_code, HTTPRequest & request, + HTTPResponse & response, HTTPResponseBufferPtr response_out, bool compression) { try { @@ -265,22 +272,25 @@ void HTTPHandler::trySendExceptionToClient( /// to avoid reading part of the current request body in the next request. if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) - request.stream().ignore(std::numeric_limits::max()); - - if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD - || exception_code == ErrorCodes::REQUIRED_PASSWORD || exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) { - if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) - response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); - else - response.requireAuthentication("ClickHouse server HTTP API"); + request.stream().ignore(std::numeric_limits::max()); + } - response.send() << message << std::endl; + if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD) + { + response.requireAuthentication("ClickHouse server HTTP API"); } else { response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); - HTTPOutputStreams output_streams(request, response, compression, getKeepAliveTimeout()); + } + + if (!response_out && !response.sent()) + response.send() << message << std::endl; + else + { + HTTPOutputStreams output_streams(response_out, compression); writeString(message, *output_streams.out_maybe_compressed); writeChar('\n', *output_streams.out_maybe_compressed); @@ -300,10 +310,12 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne ThreadStatus thread_status; /// In case of exception, send stack trace to client. + HTTPResponseBufferPtr response_out; bool with_stacktrace = false, internal_compression = false; try { + response_out = createResponseOut(request, response); response.set("Content-Type", "text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); @@ -316,15 +328,22 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne internal_compression = params.getParsed("compress", false); /// Workaround. Poco does not detect 411 Length Required case. +<<<<<<< HEAD:programs/server/HTTPHandler.cpp if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); +======= + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + && !request.getChunkedTransferEncoding() + && !request.hasContentLength()) + throw Exception("There is neither Transfer-Encoding header nor Content-Length header", ErrorCodes::HTTP_LENGTH_REQUIRED); +>>>>>>> ISSUES-5436 fix build failure & fix test failure:dbms/programs/server/HTTPHandler.cpp { - SessionContextHolder holder{server, params}; - CurrentThread::QueryScope query_scope(*holder.context); + Context query_context = server.context(); + CurrentThread::QueryScope query_scope(query_context); - holder.authentication(request, params); - processQuery(*holder.context, request, params, response); + SessionContextHolder holder{query_context, request, params}; + processQuery(holder.query_context, request, params, response, response_out); LOG_INFO(log, "Done processing query"); } } @@ -337,8 +356,34 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne */ int exception_code = getCurrentExceptionCode(); std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - trySendExceptionToClient(exception_message, exception_code, request, response, internal_compression); + trySendExceptionToClient(exception_message, exception_code, request, response, response_out, internal_compression); } } +HTTPResponseBufferPtr HTTPHandler::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response) +{ + size_t keep_alive = server.config().getUInt("keep_alive_timeout", 10); + /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). + String http_response_compression_methods = request.get("Accept-Encoding", ""); + + if (!http_response_compression_methods.empty()) + { + /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. + /// NOTE parsing of the list of methods is slightly incorrect. + if (std::string::npos != http_response_compression_methods.find("gzip")) + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + else if (std::string::npos != http_response_compression_methods.find("deflate")) + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); +#if USE_BROTLI + else if (http_response_compression_methods == "br") + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); +#endif + } + + return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); +} + } diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 47d63af3740..1faa0efe907 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -38,12 +38,12 @@ private: { ~SessionContextHolder(); - SessionContextHolder(IServer & accepted_server, HTMLForm & params); - void authentication(HTTPServerRequest & request, HTMLForm & params); + SessionContextHolder(Context & query_context_, HTTPRequest & request, HTMLForm & params); + String session_id; - std::unique_ptr context = nullptr; + Context & query_context; std::shared_ptr session_context = nullptr; std::chrono::steady_clock::duration session_timeout; }; @@ -58,9 +58,13 @@ private: size_t getKeepAliveTimeout() { return server.config().getUInt("keep_alive_timeout", 10); } - void processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response); + HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response); - void trySendExceptionToClient(const std::string & message, int exception_code, HTTPRequest & request, HTTPResponse & response, bool compression); + void processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response, HTTPResponseBufferPtr & response_out); + + void trySendExceptionToClient( + const std::string & message, int exception_code, HTTPRequest & request, + HTTPResponse & response, HTTPResponseBufferPtr response_out, bool compression); }; diff --git a/programs/server/config.xml b/programs/server/config.xml index 5111e6102c2..1c04dee6c3a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -525,11 +525,18 @@ --> + + + + + + diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index c7b3abb008b..25c2974056c 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -31,6 +31,11 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() response.set("Access-Control-Allow-Origin", "*"); setResponseDefaultHeaders(response, keep_alive_timeout); + +#if defined(POCO_CLICKHOUSE_PATCH) + if (request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) + std::tie(response_header_ostr, response_body_ostr) = response.beginSend(); +#endif } } @@ -91,12 +96,19 @@ void WriteBufferFromHTTPServerResponse::finishSendHeaders() } -void WriteBufferFromHTTPServerResponse::choiceSendEncode() +void WriteBufferFromHTTPServerResponse::nextImpl() { - if (!out && request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) { - if (!compress) + std::lock_guard lock(mutex); + + startSendHeaders(); + + if (!out && request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) { + if (compress) + { + if (compression_method == CompressionMethod::Gzip) + { #if defined(POCO_CLICKHOUSE_PATCH) *response_header_ostr << "Content-Encoding: gzip\r\n"; #else @@ -112,8 +124,8 @@ void WriteBufferFromHTTPServerResponse::choiceSendEncode() #if defined(POCO_CLICKHOUSE_PATCH) *response_header_ostr << "Content-Encoding: deflate\r\n"; #else - /// Newline autosent by response.send() - response_body_ostr = &(response.send()); + response.set("Content-Encoding", "gzip"); + response_body_ostr = &(response.send()); #endif out_raw = std::make_unique(*response_body_ostr); deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); @@ -123,9 +135,7 @@ void WriteBufferFromHTTPServerResponse::choiceSendEncode() else if (compression_method == CompressionMethod::Brotli) { #if defined(POCO_CLICKHOUSE_PATCH) - std::tie(response_header_ostr, response_body_ostr) = response.beginSend(); - if (headers_started_sending && !headers_finished_sending) - *response_header_ostr << "Content-Encoding: " << encoding_type << "\r\n"; + *response_header_ostr << "Content-Encoding: deflate\r\n"; #else response.set("Content-Encoding", content_encoding_name); #endif @@ -134,37 +144,23 @@ void WriteBufferFromHTTPServerResponse::choiceSendEncode() /// This may result in an extra empty line in the response body response_body_ostr = &(response.send()); #endif - }; out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); out = &*out_raw; } -#if USE_BROTLI - else if (compression_method == CompressionMethod::Brotli) - { - set_encoding_type("br"); - out_raw.emplace(*response_body_ostr); - brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin()); - out = &*brotli_buf; - } -#endif else - throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse", - ErrorCodes::LOGICAL_ERROR); - /// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy. + { +#if !defined(POCO_CLICKHOUSE_PATCH) + response_body_ostr = &(response.send()); +#endif + + out_raw.emplace(*response_body_ostr, working_buffer.size(), working_buffer.begin()); + out = &*out_raw; + } } - } -} - -void WriteBufferFromHTTPServerResponse::nextImpl() -{ - { - std::lock_guard lock(mutex); - - startSendHeaders(); - choiceSendEncode(); finishSendHeaders(); + } if (out) @@ -182,16 +178,13 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( unsigned keep_alive_timeout_, bool compress_, CompressionMethod compression_method_, - size_t size, - bool finish_send_headers_) + size_t size) : BufferWithOwnMemory(size) , request(request_) , response(response_) , keep_alive_timeout(keep_alive_timeout_) , compress(compress_) , compression_method(compression_method_) - , headers_started_sending(finish_send_headers_) - , headers_finished_sending(finish_send_headers_) { } @@ -235,6 +228,7 @@ void WriteBufferFromHTTPServerResponse::finalize() } } + WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { try diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index fd57279646f..528ba8ffe23 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -87,8 +87,6 @@ private: /// This method finish headers with \r\n, allowing to start to send body. void finishSendHeaders(); - void choiceSendEncode(); - void nextImpl() override; public: @@ -98,8 +96,7 @@ public: unsigned keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. CompressionMethod compression_method_ = CompressionMethod::Gzip, - size_t size = DBMS_DEFAULT_BUFFER_SIZE, - bool finish_send_headers_ = false); + size_t size = DBMS_DEFAULT_BUFFER_SIZE); /// Writes progess in repeating HTTP headers. void onProgress(const Progress & progress); diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 98dc90d9dda..e2231e35f03 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -57,9 +57,7 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) ReadBufferFromString read_buffer{value}; FormatSettings format_settings; - skipWhitespaceIfAny(read_buffer); /// Skip white space on both sides data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings); - skipWhitespaceIfAny(read_buffer); /// Skip white space on both sides if (!read_buffer.eof()) throw Exception("Value " + value + " cannot be parsed as " + type_name + " for query parameter '" + ast_param.name + "'" From dafba9d66a4af0f476399fe1bcfc3d1f0db53184 Mon Sep 17 00:00:00 2001 From: tai Date: Fri, 8 Nov 2019 20:37:31 +0800 Subject: [PATCH 406/743] ISSUES-5436 add integration test for custom http --- dbms/programs/server/HTTPHandlerFactory.h | 64 +++ .../CustomHTTP/CustomExecutor.cpp | 2 +- .../common_configs/common_config.xml | 415 ++++++++++++++++++ .../common_configs/common_users.xml | 138 ++++++ .../configs/custom_http_config.xml | 54 --- .../normally_configs/config.xml | 30 ++ .../normally_configs/users.xml | 3 + .../integration/test_custom_http/test.py | 29 +- programs/server/config.xml | 33 +- tests/integration/helpers/cluster.py | 8 +- .../0_stateless/00408_http_keep_alive.sh | 3 - 11 files changed, 702 insertions(+), 77 deletions(-) create mode 100644 dbms/programs/server/HTTPHandlerFactory.h create mode 100644 dbms/tests/integration/test_custom_http/common_configs/common_config.xml create mode 100644 dbms/tests/integration/test_custom_http/common_configs/common_users.xml delete mode 100644 dbms/tests/integration/test_custom_http/configs/custom_http_config.xml create mode 100644 dbms/tests/integration/test_custom_http/normally_configs/config.xml create mode 100644 dbms/tests/integration/test_custom_http/normally_configs/users.xml diff --git a/dbms/programs/server/HTTPHandlerFactory.h b/dbms/programs/server/HTTPHandlerFactory.h new file mode 100644 index 00000000000..73e0a2b7339 --- /dev/null +++ b/dbms/programs/server/HTTPHandlerFactory.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include +#include +#include "IServer.h" +#include "HTTPHandler.h" +#include "InterserverIOHTTPHandler.h" +#include "NotFoundHandler.h" +#include "PingRequestHandler.h" +#include "ReplicasStatusHandler.h" +#include "RootRequestHandler.h" + + +namespace DB +{ + +template +class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +private: + IServer & server; + Logger * log; + std::string name; + +public: + HTTPRequestHandlerFactory(IServer & server_, const std::string & name_) : server(server_), log(&Logger::get(name_)), name(name_) + { + } + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override + { + LOG_TRACE(log, "HTTP Request for " << name << ". " + << "Method: " + << request.getMethod() + << ", Address: " + << request.clientAddress().toString() + << ", User-Agent: " + << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); + + const auto & uri = request.getURI(); + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) + { + if (uri == "/") + return new RootRequestHandler(server); + if (uri == "/ping") + return new PingRequestHandler(server); + else if (startsWith(uri, "/replicas_status")) + return new ReplicasStatusHandler(server.context()); + } + + return new HandlerType(server); + } +}; + +using HTTPHandlerFactory = HTTPRequestHandlerFactory; +using InterserverIOHTTPHandlerFactory = HTTPRequestHandlerFactory; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp index d3d0bfc5aeb..3e984deb8aa 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -175,7 +175,7 @@ std::pair CustomExecutors::getCustomExecutor(Context if (custom_executor.second->match(context, request, params)) return custom_executor; - throw Exception("No query executors matched", ErrorCodes::UNKNOW_QUERY_EXECUTOR); + throw Exception("No CustomExecutor match " + request.getURI(), ErrorCodes::UNKNOW_QUERY_EXECUTOR); } CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) diff --git a/dbms/tests/integration/test_custom_http/common_configs/common_config.xml b/dbms/tests/integration/test_custom_http/common_configs/common_config.xml new file mode 100644 index 00000000000..154ebf6c35e --- /dev/null +++ b/dbms/tests/integration/test_custom_http/common_configs/common_config.xml @@ -0,0 +1,415 @@ + + + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + + 8123 + 9000 + + + + + + + + + /etc/clickhouse-server/server.crt + /etc/clickhouse-server/server.key + + /etc/clickhouse-server/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + + true + true + sslv2,sslv3 + true + + + + RejectCertificateHandler + + + + + + + + + 9009 + + + + + + + + + + + + + + + + + + + + 4096 + 3 + + + 100 + + + + + + 8589934592 + + + 5368709120 + + + + /var/lib/clickhouse/ + + + /var/lib/clickhouse/tmp/ + + + /var/lib/clickhouse/user_files/ + + + users.xml + + + default + + + + + + default + + + + + + + + + false + + + + + + + + localhost + 9000 + + + + + + + localhost + 9000 + + + + + localhost + 9000 + + + + + + + localhost + 9440 + 1 + + + + + + + localhost + 9000 + + + + + localhost + 1 + + + + + + + + + + + + + + + + + 3600 + + + + 3600 + + + 60 + + + + + + + + + + system + query_log
    + + toYYYYMM(event_date) + + 7500 +
    + + + + system + query_thread_log
    + toYYYYMM(event_date) + 7500 +
    + + + + + + + + + + + + + + + *_dictionary.xml + + + + + + + + + + /clickhouse/task_queue/ddl + + + + + + + + + + + + + + + + click_cost + any + + 0 + 3600 + + + 86400 + 60 + + + + max + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + + + /var/lib/clickhouse/format_schemas/ + + + +
    diff --git a/dbms/tests/integration/test_custom_http/common_configs/common_users.xml b/dbms/tests/integration/test_custom_http/common_configs/common_users.xml new file mode 100644 index 00000000000..9755c29d480 --- /dev/null +++ b/dbms/tests/integration/test_custom_http/common_configs/common_users.xml @@ -0,0 +1,138 @@ + + + + + + + + 10000000000 + + + 0 + + + random + + + + + 1 + + + + + + + + + + + + + ::/0 + + + + default + + + default + + + + + + + a = 1 + + + + + a + b < 1 or c - d > 5 + + + + + c = 1 + + + + + + + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml deleted file mode 100644 index eb56cd6c18c..00000000000 --- a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml +++ /dev/null @@ -1,54 +0,0 @@ - - 8123 - - - - /test_for_single_insert - INSERT INTO test.test VALUES - - - - /test_for_single_select - SELECT * FROM test.test ORDER BY id - - - - /test_for_multiple_select - SELECT * FROM test.test ORDER BY id - SELECT * FROM test.test ORDER BY id - - - - /test_for_multiple_select - INSERT INTO test.test VALUES - SELECT 'test_for_hybrid_insert_and_select_with_one_insert first' - SELECT 'test_for_hybrid_insert_and_select_with_one_insert second' - - - - - - /test_for_url_match - SELECT 'Matched test_for_url_match' - - - - PUT - SELECT 'Matched test_for_method_match' - - - - GET - /test_for_multiple_match - SELECT 'Matched test_for_multiple_match' - - - diff --git a/dbms/tests/integration/test_custom_http/normally_configs/config.xml b/dbms/tests/integration/test_custom_http/normally_configs/config.xml new file mode 100644 index 00000000000..6fab88bcf80 --- /dev/null +++ b/dbms/tests/integration/test_custom_http/normally_configs/config.xml @@ -0,0 +1,30 @@ + + + + + + PUT + /test_for_only_insert_queries + INSERT INTO test.test VALUES + + + + POST + /test_for_only_select_queries + SELECT value FROM system.settings WHERE name = 'max_threads' + SELECT value FROM system.settings WHERE name = 'max_alter_threads' + + + + /test_for_hybrid_insert_and_select_queries + INSERT INTO test.test VALUES + SELECT * FROM test.test ORDER BY id + + + + /test_for_throw_exception_when_after_select + SELECT * FROM test.test ORDER BY id + SELECT throwIf(number = 2, 'Throw Exception') FROM numbers(3) + + + diff --git a/dbms/tests/integration/test_custom_http/normally_configs/users.xml b/dbms/tests/integration/test_custom_http/normally_configs/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_custom_http/normally_configs/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_custom_http/test.py b/dbms/tests/integration/test_custom_http/test.py index 805c6163839..08f8d37586c 100644 --- a/dbms/tests/integration/test_custom_http/test.py +++ b/dbms/tests/integration/test_custom_http/test.py @@ -1,21 +1,38 @@ +import os + import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -test_instance = cluster.add_instance('node', main_configs=['configs/custom_http_config.xml']) +SCRIPT_PATH = os.path.dirname(os.path.realpath(__file__)) + + +def add_instance(name, config_dir): + print os.path.join(SCRIPT_PATH, 'common_configs', 'common_users.xml') + print os.path.join(SCRIPT_PATH, 'common_configs', 'common_config.xml') + return cluster.add_instance(name, config_dir=os.path.join(SCRIPT_PATH, config_dir), + main_configs=[os.path.join(SCRIPT_PATH, 'common_configs', 'common_config.xml')], + user_configs=[os.path.join(SCRIPT_PATH, 'common_configs', 'common_users.xml')]) + + +normally_instance = add_instance("normally_node", "normally_configs") @pytest.fixture(scope="module") -def start_cluster(): +def started_cluster(): try: cluster.start() - test_instance.query('CREATE DATABASE `test`') - test_instance.query('CREATE TABLE `test`.`test` (`id` UInt8) Engine = Memory') + normally_instance.query('CREATE DATABASE `test`') + normally_instance.query('CREATE TABLE `test`.`test` (`id` UInt8) Engine = Memory') yield cluster finally: cluster.shutdown() -def test_for_single_insert(started_cluster): - assert test_instance.http_query('/test_for_single_insert', data='(1)(2)(3)') == '\n' +def test_normally_match(started_cluster): + assert normally_instance.http_request('test_for_only_insert_queries', method='PUT', data='(1)(2)(3)') == '' + assert normally_instance.http_request(url='test_for_only_select_queries', + params='max_threads=1', method='POST', data='max_alter_threads=2') == '1\n2\n' + assert normally_instance.http_request('test_for_hybrid_insert_and_select_queries', method='POST', data='(4)') == '1\n2\n3\n4\n' + assert 'Throw Exception' in normally_instance.http_request('test_for_throw_exception_when_after_select') diff --git a/programs/server/config.xml b/programs/server/config.xml index 1c04dee6c3a..706149b66b1 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -525,21 +525,30 @@ --> + - - - + + + + + query + + - diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a9fd572a8b0..efe47da9f46 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -11,6 +11,7 @@ import subprocess import time import urllib import httplib +import requests import xml.dom.minidom import logging import docker @@ -687,7 +688,7 @@ class ClickHouseInstance: def http_code_and_message(): return str(open_result.getcode()) + " " + httplib.responses[open_result.getcode()] + ": " + open_result.read() - + if expect_fail_and_get_error: if open_result.getcode() == 200: raise Exception("ClickHouse HTTP server is expected to fail, but succeeded: " + open_result.read()) @@ -697,6 +698,11 @@ class ClickHouseInstance: raise Exception("ClickHouse HTTP server returned " + http_code_and_message()) return open_result.read() + # Connects to the instance via HTTP interface, sends a query and returns the answer + def http_request(self, url, method='GET', params=None, data=None): + url = "http://" + self.ip_address + ":8123/"+url + return requests.request(method=method, url=url, params=params, data=data).content + # Connects to the instance via HTTP interface, sends a query, expects an error and return the error message def http_query_and_get_error(self, sql, data=None, params=None, user=None, password=None): return self.http_query(sql=sql, data=data, params=params, user=user, password=password, expect_fail_and_get_error=True) diff --git a/tests/queries/0_stateless/00408_http_keep_alive.sh b/tests/queries/0_stateless/00408_http_keep_alive.sh index 79e39d83704..83ac8c55f24 100755 --- a/tests/queries/0_stateless/00408_http_keep_alive.sh +++ b/tests/queries/0_stateless/00408_http_keep_alive.sh @@ -8,6 +8,3 @@ URL="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/ ${CLICKHOUSE_CURL} -vsS ${URL} --data-binary @- <<< "SELECT 1" 2>&1 | perl -lnE 'print if /Keep-Alive/'; ${CLICKHOUSE_CURL} -vsS ${URL} --data-binary @- <<< " error here " 2>&1 | perl -lnE 'print if /Keep-Alive/'; ${CLICKHOUSE_CURL} -vsS ${URL}ping 2>&1 | perl -lnE 'print if /Keep-Alive/'; - -# no keep-alive: -${CLICKHOUSE_CURL} -vsS ${URL}404/not/found/ 2>&1 | perl -lnE 'print if /Keep-Alive/'; From 8105a9bbe0cfc5831150be9f3f94a61348576ba7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 14 Nov 2019 10:20:06 +0800 Subject: [PATCH 407/743] ISSUES-5436 fix review suggestions --- dbms/programs/server/HTTPHandlerFactory.cpp | 140 ++++++++ dbms/programs/server/HTTPHandlerFactory.h | 73 ++-- .../HTTPRequestHandler}/ExtractorClientInfo.h | 0 .../ExtractorContextChange.h | 16 +- .../HTTPExceptionHandler.cpp | 154 ++++++++ .../HTTPRequestHandler/HTTPExceptionHandler.h | 19 + .../HTTPPingRequestHandler.cpp | 45 +++ .../HTTPPingRequestHandler.h | 26 ++ .../HTTPQueryRequestHandler.cpp | 179 ++++++++++ .../HTTPQueryRequestHandler.h | 48 +++ ...TPQueryRequestHandlerMatcherAndCreator.cpp | 330 ++++++++++++++++++ ...HTTPQueryRequestHandlerMatcherAndCreator.h | 92 +++++ .../HTTPReplicasStatusRequestHandler.cpp | 118 +++++++ .../HTTPReplicasStatusRequestHandler.h | 25 ++ .../HTTPRootRequestHandler.cpp | 45 +++ .../HTTPRootRequestHandler.h | 26 ++ .../HTTPSessionContextHolder.cpp | 105 ++++++ .../HTTPSessionContextHolder.h | 24 ++ .../HTTPInputStreams.cpp | 19 +- dbms/src/DataStreams/HTTPInputStreams.h | 29 ++ .../HTTPOutputStreams.cpp | 2 +- .../HTTPOutputStreams.h | 0 .../CustomHTTP/AlwaysQueryMatcher.h | 16 - .../CustomHTTP/ConstQueryExecutor.cpp | 73 ---- .../CustomHTTP/ConstQueryExecutor.h | 30 -- .../CustomHTTP/CustomExecutor.cpp | 201 ----------- .../Interpreters/CustomHTTP/CustomExecutor.h | 77 ---- .../CustomHTTP/DynamicQueryExecutor.cpp | 60 ---- .../CustomHTTP/DynamicQueryExecutor.h | 37 -- .../CustomHTTP/HTTPInputStreams.h | 27 -- .../CustomHTTP/MethodQueryMatcher.h | 30 -- .../CustomHTTP/QueryExecutorAndMatcher.h | 47 --- .../CustomHTTP/URLQueryMatcher.cpp | 73 ---- .../Interpreters/CustomHTTP/URLQueryMatcher.h | 32 -- .../normally_configs/config.xml | 30 -- .../integration/test_custom_http/test.py | 38 -- .../__init__.py | 0 .../common_configs/common_config.xml | 0 .../common_configs/common_users.xml | 0 .../other_tests_configs/config.xml | 18 + .../other_tests_configs}/users.xml | 0 .../test_http_handlers_config/test.py | 87 +++++ .../test_insert_and_select_dynamic/config.xml | 23 ++ .../test_insert_and_select_dynamic/users.xml | 3 + .../config.xml | 26 ++ .../users.xml | 3 + .../config.xml | 31 ++ .../test_param_and_settings_dynamic/users.xml | 3 + .../config.xml | 43 +++ .../users.xml | 3 + programs/server/CMakeLists.txt | 10 +- programs/server/NotFoundHandler.cpp | 18 +- programs/server/NotFoundHandler.h | 9 +- programs/server/Server.cpp | 1 - programs/server/config.xml | 48 +-- src/IO/WriteBufferFromHTTPServerResponse.cpp | 17 +- src/Interpreters/Context.cpp | 31 +- src/Interpreters/Context.h | 5 - .../ReplaceQueryParameterVisitor.cpp | 2 - tests/integration/helpers/cluster.py | 6 +- .../test_https_replication/configs/config.xml | 9 + 61 files changed, 1760 insertions(+), 922 deletions(-) create mode 100644 dbms/programs/server/HTTPHandlerFactory.cpp rename dbms/{src/Interpreters/CustomHTTP => programs/server/HTTPRequestHandler}/ExtractorClientInfo.h (100%) rename dbms/{src/Interpreters/CustomHTTP => programs/server/HTTPRequestHandler}/ExtractorContextChange.h (87%) create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h rename dbms/src/{Interpreters/CustomHTTP => DataStreams}/HTTPInputStreams.cpp (71%) create mode 100644 dbms/src/DataStreams/HTTPInputStreams.h rename dbms/src/{Interpreters/CustomHTTP => DataStreams}/HTTPOutputStreams.cpp (99%) rename dbms/src/{Interpreters/CustomHTTP => DataStreams}/HTTPOutputStreams.h (100%) delete mode 100644 dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h delete mode 100644 dbms/tests/integration/test_custom_http/normally_configs/config.xml delete mode 100644 dbms/tests/integration/test_custom_http/test.py rename dbms/tests/integration/{test_custom_http => test_http_handlers_config}/__init__.py (100%) rename dbms/tests/integration/{test_custom_http => test_http_handlers_config}/common_configs/common_config.xml (100%) rename dbms/tests/integration/{test_custom_http => test_http_handlers_config}/common_configs/common_users.xml (100%) create mode 100644 dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml rename dbms/tests/integration/{test_custom_http/normally_configs => test_http_handlers_config/other_tests_configs}/users.xml (100%) create mode 100644 dbms/tests/integration/test_http_handlers_config/test.py create mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml diff --git a/dbms/programs/server/HTTPHandlerFactory.cpp b/dbms/programs/server/HTTPHandlerFactory.cpp new file mode 100644 index 00000000000..9a2d1e52f09 --- /dev/null +++ b/dbms/programs/server/HTTPHandlerFactory.cpp @@ -0,0 +1,140 @@ +#include "HTTPHandlerFactory.h" + +#include "NotFoundHandler.h" +#include "HTTPRequestHandler/HTTPRootRequestHandler.h" +#include "HTTPRequestHandler/HTTPPingRequestHandler.h" +#include "HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int UNKNOW_HTTP_HANDLER_TYPE; + extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; +} + +InterserverIOHTTPHandlerFactory::InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_) + : server(server_), log(&Logger::get(name_)), name(name_) +{ +} + +Poco::Net::HTTPRequestHandler * InterserverIOHTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +{ + LOG_TRACE(log, "HTTP Request for " << name << ". " + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); + + const auto & uri = request.getURI(); + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) + { + if (uri == "/") + return new HTTPRootRequestHandler(server); + if (uri == "/ping") + return new HTTPPingRequestHandler(server); + else if (startsWith(uri, "/replicas_status")) + return new HTTPReplicasStatusRequestHandler(server.context()); + } + + if (uri.find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + { + return new InterserverIOHTTPHandler(server); + } + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + { + return new NotFoundHandler( + "Use / or /ping for health checks.\n" + "Or /replicas_status for more sophisticated health checks.\n" + "Send queries from your program with POST method or GET /?query=...\n\n" + " Use clickhouse-client:\n\n" + " For interactive data analysis:\n" + " clickhouse-client\n\n" + " For batch query processing:\n" + " clickhouse-client --query='SELECT 1' > result\n" + " clickhouse-client < query > result" + ); + } + + return nullptr; +} + +HTTPHandlerFactory::HTTPHandlerFactory(IServer & server_, const std::string & name_) + : server(server_), log(&Logger::get(name_)), name(name_) +{ + updateHTTPHandlersCreator(server.config()); + + if (handlers_creator.empty()) + throw Exception("The HTTPHandlers does not exist in the config.xml", ErrorCodes::EMPTY_HTTP_HANDLER_IN_CONFIG); +} + +Poco::Net::HTTPRequestHandler * HTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +{ + LOG_TRACE(log, "HTTP Request for " << name << ". " + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); + + for (const auto & [matcher, creator] : handlers_creator) + { + if (matcher(request)) + return creator(); + } + + return new NotFoundHandler(no_handler_description); +} + +HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPredefineQueryHandlerMatcher(IServer &, const String &); + +HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPredefineQueryHandlerCreator(IServer &, const String &); + +void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) +{ + Poco::Util::AbstractConfiguration::Keys http_handlers_item_key; + configuration.keys(key, http_handlers_item_key); + + handlers_creator.reserve(http_handlers_item_key.size()); + for (const auto & http_handler_type_name : http_handlers_item_key) + { + if (http_handler_type_name.find('.') != String::npos) + throw Exception("HTTPHandler type name with dots are not supported: '" + http_handler_type_name + "'", ErrorCodes::SYNTAX_ERROR); + + const auto & handler_key = key + "." + http_handler_type_name; + + if (startsWith(http_handler_type_name, "root_handler")) + handlers_creator.push_back({createRootHandlerMatcher(server, handler_key), createRootHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "ping_handler")) + handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "dynamic_query_handler")) + handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "predefine_query_handler")) + handlers_creator.push_back({createPredefineQueryHandlerMatcher(server, handler_key), createPredefineQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "replicas_status_handler")) + handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); + else if (http_handler_type_name == "no_handler_description") + no_handler_description = configuration.getString(key + ".no_handler_description"); + else + throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOW_HTTP_HANDLER_TYPE); + } +} + +} diff --git a/dbms/programs/server/HTTPHandlerFactory.h b/dbms/programs/server/HTTPHandlerFactory.h index 73e0a2b7339..5add7b619af 100644 --- a/dbms/programs/server/HTTPHandlerFactory.h +++ b/dbms/programs/server/HTTPHandlerFactory.h @@ -1,64 +1,51 @@ #pragma once -#include #include #include +#include +#include #include #include "IServer.h" -#include "HTTPHandler.h" #include "InterserverIOHTTPHandler.h" -#include "NotFoundHandler.h" -#include "PingRequestHandler.h" -#include "ReplicasStatusHandler.h" -#include "RootRequestHandler.h" namespace DB { -template -class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +class InterserverIOHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory { +public: + InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_); + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; + +private: + IServer & server; + Logger * log; + std::string name; +}; + +using HTTPHandlerCreator = std::function; +using HTTPHandlerMatcher = std::function; +using HTTPHandlerMatcherAndCreator = std::pair; +using HTTPHandlersMatcherAndCreator = std::vector; + +class HTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +public: + HTTPHandlerFactory(IServer & server_, const std::string & name_); + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; + + void updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key = "http_handlers"); + private: IServer & server; Logger * log; std::string name; -public: - HTTPRequestHandlerFactory(IServer & server_, const std::string & name_) : server(server_), log(&Logger::get(name_)), name(name_) - { - } - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override - { - LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " - << request.getMethod() - << ", Address: " - << request.clientAddress().toString() - << ", User-Agent: " - << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); - - const auto & uri = request.getURI(); - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) - { - if (uri == "/") - return new RootRequestHandler(server); - if (uri == "/ping") - return new PingRequestHandler(server); - else if (startsWith(uri, "/replicas_status")) - return new ReplicasStatusHandler(server.context()); - } - - return new HandlerType(server); - } + String no_handler_description; + HTTPHandlersMatcherAndCreator handlers_creator; }; -using HTTPHandlerFactory = HTTPRequestHandlerFactory; -using InterserverIOHTTPHandlerFactory = HTTPRequestHandlerFactory; - } diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h b/dbms/programs/server/HTTPRequestHandler/ExtractorClientInfo.h similarity index 100% rename from dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h rename to dbms/programs/server/HTTPRequestHandler/ExtractorClientInfo.h diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h b/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h similarity index 87% rename from dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h rename to dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h index 56411cb122a..0fdfd73b4f6 100644 --- a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h +++ b/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h @@ -1,10 +1,10 @@ #pragma once -#include #include +#include #include #include -#include +#include namespace DB { @@ -17,7 +17,7 @@ namespace ErrorCodes class ExtractorContextChange { public: - ExtractorContextChange(Context & context_, const CustomExecutorPtr & executor_) : context(context_), executor(executor_) {} + ExtractorContextChange(Context & context_, bool settings_may_in_post_) : context(context_), settings_may_in_post(settings_may_in_post_) {} static const NameSet & getReservedParamNames() { @@ -59,20 +59,20 @@ public: becomeReadonlyIfNeed(request); changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); - if (is_multipart_data || executor->canBeParseRequestBody()) + if (is_multipart_data || settings_may_in_post) { ExternalTablesHandler handler(context, params); params.load(request, request.stream(), handler); /// We use the `Post Request Body Settings` to override the `Qeruy String Param settings` - if (executor->canBeParseRequestBody()) + if (settings_may_in_post) changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); } } private: Context & context; - const CustomExecutorPtr & executor; + bool settings_may_in_post; /// 'readonly' setting values mean: /// readonly = 0 - any query is allowed, client can change any setting. @@ -82,7 +82,7 @@ private: /// In theory if initially readonly = 0, the client can change any setting and then set readonly /// to some other value. /// Only readonly queries are allowed for HTTP GET requests. - void becomeReadonlyIfNeed(HTTPRequest & request) + void becomeReadonlyIfNeed(Poco::Net::HTTPServerRequest & request) { if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) { @@ -109,8 +109,6 @@ private: { if (Settings::findIndex(name) != Settings::npos) settings_changes.push_back({name, value}); - else if (!executor->isQueryParam(name)) - throw Exception("Unknown HTTP param name: '" + name + "'", ErrorCodes::UNKNOWN_HTTP_PARAM); } } diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp new file mode 100644 index 00000000000..8bfa351aab2 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp @@ -0,0 +1,154 @@ +#include "HTTPExceptionHandler.h" + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int READONLY; + extern const int UNKNOWN_COMPRESSION_METHOD; + + extern const int CANNOT_PARSE_TEXT; + extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; + extern const int CANNOT_PARSE_QUOTED_STRING; + extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_PARSE_DATETIME; + extern const int CANNOT_PARSE_NUMBER; + extern const int CANNOT_OPEN_FILE; + + extern const int UNKNOWN_ELEMENT_IN_AST; + extern const int UNKNOWN_TYPE_OF_AST_NODE; + extern const int TOO_DEEP_AST; + extern const int TOO_BIG_AST; + extern const int UNEXPECTED_AST_STRUCTURE; + + extern const int SYNTAX_ERROR; + + extern const int INCORRECT_DATA; + extern const int TYPE_MISMATCH; + + extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_FUNCTION; + extern const int UNKNOWN_IDENTIFIER; + extern const int UNKNOWN_TYPE; + extern const int UNKNOWN_STORAGE; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_SETTING; + extern const int UNKNOWN_DIRECTION_OF_SORTING; + extern const int UNKNOWN_AGGREGATE_FUNCTION; + extern const int UNKNOWN_FORMAT; + extern const int UNKNOWN_DATABASE_ENGINE; + extern const int UNKNOWN_TYPE_OF_QUERY; + + extern const int QUERY_IS_TOO_LARGE; + + extern const int NOT_IMPLEMENTED; + extern const int SOCKET_TIMEOUT; + + extern const int UNKNOWN_USER; + extern const int WRONG_PASSWORD; + extern const int REQUIRED_PASSWORD; + + extern const int INVALID_SESSION_TIMEOUT; + extern const int HTTP_LENGTH_REQUIRED; +} + +static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code) +{ + using namespace Poco::Net; + + if (exception_code == ErrorCodes::REQUIRED_PASSWORD) + return HTTPResponse::HTTP_UNAUTHORIZED; + else if (exception_code == ErrorCodes::CANNOT_PARSE_TEXT || + exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || + exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_DATE || + exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || + exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || + + exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || + exception_code == ErrorCodes::TOO_DEEP_AST || + exception_code == ErrorCodes::TOO_BIG_AST || + exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || + + exception_code == ErrorCodes::SYNTAX_ERROR || + + exception_code == ErrorCodes::INCORRECT_DATA || + exception_code == ErrorCodes::TYPE_MISMATCH) + return HTTPResponse::HTTP_BAD_REQUEST; + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + exception_code == ErrorCodes::UNKNOWN_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || + exception_code == ErrorCodes::UNKNOWN_TYPE || + exception_code == ErrorCodes::UNKNOWN_STORAGE || + exception_code == ErrorCodes::UNKNOWN_DATABASE || + exception_code == ErrorCodes::UNKNOWN_SETTING || + exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || + exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_FORMAT || + exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || + + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY) + return HTTPResponse::HTTP_NOT_FOUND; + else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE) + return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE; + else if (exception_code == ErrorCodes::NOT_IMPLEMENTED) + return HTTPResponse::HTTP_NOT_IMPLEMENTED; + else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || + exception_code == ErrorCodes::CANNOT_OPEN_FILE) + return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; + else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) + return HTTPResponse::HTTP_LENGTH_REQUIRED; + + return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR; +} + + +void HTTPExceptionHandler::handle( + const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, + Poco::Net::HTTPServerResponse & response, std::shared_ptr response_out, + bool compression, Poco::Logger * log) +{ + try + { + /// If HTTP method is POST and Keep-Alive is turned on, we should read the whole request body + /// to avoid reading part of the current request body in the next request. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() + && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + { + request.stream().ignore(std::numeric_limits::max()); + } + + if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD) + { + response.requireAuthentication("ClickHouse server HTTP API"); + } + else + { + response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); + } + + if (!response_out && !response.sent()) + response.send() << message << std::endl; + else + { + HTTPOutputStreams output_streams(response_out, compression); + + writeString(message, *output_streams.out_maybe_compressed); + writeChar('\n', *output_streams.out_maybe_compressed); + + output_streams.finalize(); + } + } + catch (...) + { + tryLogCurrentException(log, "Cannot send exception to client"); + } +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h new file mode 100644 index 00000000000..d2a5383ed4a --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class HTTPExceptionHandler +{ +public: + static void handle(const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, + Poco::Net::HTTPServerResponse & response, std::shared_ptr response_out, bool compression, + Poco::Logger * log); +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp new file mode 100644 index 00000000000..292f98ba0eb --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp @@ -0,0 +1,45 @@ +#include "HTTPPingRequestHandler.h" + +#include + +#include + +#include +#include + +namespace DB +{ + +void HTTPPingRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) +{ + try + { + const auto & config = server.config(); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); + + const char * data = "Ok.\n"; + response.sendBuffer(data, strlen(data)); + } + catch (...) + { + tryLogCurrentException("HTTPPingRequestHandler"); + } +} + +HTTPHandlerMatcher createPingHandlerMatcher(IServer & server, const String & key) +{ + const auto & path = server.config().getString(key, "/ping"); + + return [&, path = path](const Poco::Net::HTTPServerRequest & request) + { + return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) && + request.getURI() == path; + }; +} + +HTTPHandlerCreator createPingHandlerCreator(IServer & server, const String &) +{ + return [&]() { return new HTTPPingRequestHandler(server); }; +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h new file mode 100644 index 00000000000..7d524dd638d --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h @@ -0,0 +1,26 @@ +#pragma once + +#include "../IServer.h" +#include "../HTTPHandlerFactory.h" + +#include + + +namespace DB +{ + +/// Response with "Ok.\n". Used for availability checks. +class HTTPPingRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit HTTPPingRequestHandler(const IServer & server_) : server(server_) + { + } + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + const IServer & server; +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp new file mode 100644 index 00000000000..1bb93908638 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp @@ -0,0 +1,179 @@ +#include "HTTPQueryRequestHandler.h" + +#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 +#include +#include +#include + +#include "ExtractorClientInfo.h" +#include "ExtractorContextChange.h" +#include "HTTPQueryRequestHandlerMatcherAndCreator.h" +#include "HTTPSessionContextHolder.h" +#include "HTTPExceptionHandler.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int HTTP_LENGTH_REQUIRED; +} + +template +HTTPQueryRequestHandler::HTTPQueryRequestHandler(const IServer & server_, const QueryParamExtractor & extractor_) + : server(server_), log(&Logger::get("HTTPQueryRequestHandler")), extractor(extractor_) +{ + server_display_name = server.config().getString("display_name", getFQDNOrHostName()); +} + +template +void HTTPQueryRequestHandler::processQuery( + Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, + Poco::Net::HTTPServerResponse & response, HTTPResponseBufferPtr & response_out) +{ + ExtractorClientInfo{context.getClientInfo()}.extract(request); + ExtractorContextChange{context, extractor.loadSettingsFromPost()}.extract(request, params); + + HTTPInputStreams input_streams{context, request, params}; + HTTPOutputStreams output_streams = HTTPOutputStreams(response_out, context, request, params); + + const auto & queries = extractor.extract(context, request, params); + + for (const auto & [execute_query, not_touch_post] : queries) + { + ReadBufferPtr temp_query_buf; + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + if (not_touch_post && !startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, + context, [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); + } + + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + output_streams.finalize(); +} + +template +void HTTPQueryRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) +{ + setThreadName("HTTPHandler"); + ThreadStatus thread_status; + + /// In case of exception, send stack trace to client. + HTTPResponseBufferPtr response_out; + bool with_stacktrace = false, internal_compression = false; + + try + { + response_out = createResponseOut(request, response); + response.set("Content-Type", "text/plain; charset=UTF-8"); + response.set("X-ClickHouse-Server-Display-Name", server_display_name); + + /// For keep-alive to work. + if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) + response.setChunkedTransferEncoding(true); + + HTMLForm params(request); + with_stacktrace = params.getParsed("stacktrace", false); + internal_compression = params.getParsed("compress", false); + + /// Workaround. Poco does not detect 411 Length Required case. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) + throw Exception("There is neither Transfer-Encoding header nor Content-Length header", ErrorCodes::HTTP_LENGTH_REQUIRED); + + { + Context query_context = server.context(); + CurrentThread::QueryScope query_scope(query_context); + + HTTPSessionContextHolder holder{query_context, request, params}; + processQuery(holder.query_context, request, params, response, response_out); + LOG_INFO(log, "Done processing query"); + } + } + catch (...) + { + tryLogCurrentException(log); + + /** If exception is received from remote server, then stack trace is embedded in message. + * If exception is thrown on local server, then stack trace is in separate field. + */ + int exception_code = getCurrentExceptionCode(); + std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); + HTTPExceptionHandler::handle(exception_message, exception_code, request, response, response_out, internal_compression, log); + } +} + +template +HTTPResponseBufferPtr HTTPQueryRequestHandler::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response) +{ + size_t keep_alive = server.config().getUInt("keep_alive_timeout", 10); + /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). + String http_response_compression_methods = request.get("Accept-Encoding", ""); + + if (!http_response_compression_methods.empty()) + { + /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. + /// NOTE parsing of the list of methods is slightly incorrect. + if (std::string::npos != http_response_compression_methods.find("gzip")) + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + else if (std::string::npos != http_response_compression_methods.find("deflate")) + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); +#if USE_BROTLI + else if (http_response_compression_methods == "br") + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); +#endif + } + + return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); +} + + +template class HTTPQueryRequestHandler; +template class HTTPQueryRequestHandler; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h new file mode 100644 index 00000000000..5303b0cda89 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h @@ -0,0 +1,48 @@ +#pragma once + +#include "../IServer.h" + +#include + +#include +#include + +#include + + +namespace CurrentMetrics +{ + extern const Metric HTTPConnection; +} + +namespace Poco { class Logger; } + +namespace DB +{ + +template +class HTTPQueryRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit HTTPQueryRequestHandler(const IServer & server_, const QueryParamExtractor & extractor_); + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + const IServer & server; + Poco::Logger * log; + QueryParamExtractor extractor; + + /// It is the name of the server that will be sent in an http-header X-ClickHouse-Server-Display-Name. + String server_display_name; + + CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; + + HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response); + + void processQuery( + Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, + Poco::Net::HTTPServerResponse & response, HTTPResponseBufferPtr & response_out); +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp new file mode 100644 index 00000000000..321a585d939 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp @@ -0,0 +1,330 @@ +#include "HTTPQueryRequestHandlerMatcherAndCreator.h" + +#include "../HTTPHandlerFactory.h" +#include "HTTPQueryRequestHandler.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int EMPTY_PREDEFINE_QUERY; + extern const int CANNOT_COMPILE_REGEXP; + extern const int UNKNOWN_QUERY_PARAMETER; + extern const int DUPLICATE_CAPTURE_QUERY_PARAM; + extern const int ILLEGAL_HTTP_HANDLER_PARAM_NAME; + extern const int TOO_MANY_INSERT_QUERY_WITH_PREDEFINE_QUERY; +} + +ExtractorDynamicQueryParameters::ExtractorDynamicQueryParameters( + Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) + : url_regex(url_regex_), headers_regex(headers_regex_) +{ + dynamic_param_name = configuration.getString(key + ".query_param_name", "query"); + + NameSet extracted_names; + + if (url_regex) + { + for (const auto & [capturing_name, capturing_index] : url_regex->NamedCapturingGroups()) + { + if (startsWith(capturing_name, "param_")) + { + if (extracted_names.count(capturing_name)) + throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); + + extracted_names.emplace(capturing_name); + extract_from_url[capturing_name] = capturing_index; + } + } + } + + if (!headers_regex.empty()) + { + for (const auto & [header_name, header_regex] : headers_regex) + { + for (const auto & [capturing_name, capturing_index] : header_regex->NamedCapturingGroups()) + { + if (startsWith(capturing_name, "param_")) + { + if (extracted_names.count(capturing_name)) + throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); + + extracted_names.emplace(capturing_name); + extract_from_headers[header_name][capturing_name] = capturing_index; + } + } + } + } +} + +template +void extractParamWithRegex(Context & context, const RegexRule & regex, const std::map & extract_params, const String & value) +{ + int num_captures = regex->NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(value.data(), value.size()); + + if (regex->Match(input, 0, value.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + { + for (const auto & [capturing_name, capturing_index] : extract_params) + { + String param_name = capturing_name; + const auto & capturing_value = matches[capturing_index]; + + if constexpr (remove_prefix_for_param) + { + const static size_t prefix_size = strlen("param_"); + param_name = capturing_name.substr(prefix_size); + } + + context.setQueryParameter(param_name, String(capturing_value.data(), capturing_value.size())); + } + } +} + +ExtractRes ExtractorDynamicQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) +{ + if (!extract_from_url.empty()) + extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); + + + if (!extract_from_headers.empty()) + for (const auto & [header_name, extract_params] : extract_from_headers) + extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name)); + + String extracted_query_from_params; + const static size_t prefix_size = strlen("param_"); + + for (const auto & [param_name, param_value] : params) + { + if (param_name == dynamic_param_name) + extracted_query_from_params += param_value; + else if (startsWith(param_name, "param_")) + context.setQueryParameter(param_name.substr(prefix_size), param_value); + } + + if (!extracted_query_from_params.empty()) + extracted_query_from_params += "\n"; + + return {{extracted_query_from_params, true}}; +} + +ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( + Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) + : url_regex(url_regex_), headers_regex(headers_regex_) +{ + Poco::Util::AbstractConfiguration::Keys queries_key; + configuration.keys(key + ".queries", queries_key); + + if (queries_key.empty()) + throw Exception("There must be at least one predefined query in the predefined HTTPHandler.", ErrorCodes::EMPTY_PREDEFINE_QUERY); + + for (const auto & query_key : queries_key) + { + const auto & predefine_query = configuration.getString(key + ".queries." + query_key); + + const char * query_begin = predefine_query.data(); + const char * query_end = predefine_query.data() + predefine_query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); + QueryParameterVisitor{queries_names}.visit(extract_query_ast); + + bool is_insert_query = extract_query_ast->as(); + + if (has_insert_query && is_insert_query) + throw Exception("Too many insert queries in predefined queries.", ErrorCodes::TOO_MANY_INSERT_QUERY_WITH_PREDEFINE_QUERY); + + has_insert_query |= is_insert_query; + predefine_queries.push_back({predefine_query, is_insert_query}); + } + + const auto & reserved_params_name = ExtractorContextChange::getReservedParamNames(); + for (const auto & predefine_query_name : queries_names) + { + if (Settings::findIndex(predefine_query_name) != Settings::npos || reserved_params_name.count(predefine_query_name)) + throw Exception("Illegal http_handler param name '" + predefine_query_name + + "', Because it's reserved name or Settings name", ErrorCodes::ILLEGAL_HTTP_HANDLER_PARAM_NAME); + } + + NameSet extracted_names; + + if (url_regex) + { + for (const auto & [capturing_name, capturing_index] : url_regex->NamedCapturingGroups()) + { + if (queries_names.count(capturing_name)) + { + if (extracted_names.count(capturing_name)) + throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); + + extracted_names.emplace(capturing_name); + extract_from_url[capturing_name] = capturing_index; + } + } + } + + if (!headers_regex.empty()) + { + for (const auto & [header_name, header_regex] : headers_regex) + { + for (const auto & [capturing_name, capturing_index] : header_regex->NamedCapturingGroups()) + { + if (queries_names.count(capturing_name)) + { + if (extracted_names.count(capturing_name)) + throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); + + extracted_names.emplace(capturing_name); + extract_from_headers[header_name][capturing_name] = capturing_index; + } + } + } + } +} + +ExtractRes ExtractorPredefineQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) +{ + if (!extract_from_url.empty()) + extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); + + if (!extract_from_headers.empty()) + for (const auto & [header_name, extract_params] : extract_from_headers) + extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name)); + + for (const auto & param : params) + if (queries_names.count(param.first)) + context.setQueryParameter(param.first, param.second); + + return predefine_queries; +} + +RegexRule HTTPQueryRequestHandlerMatcherAndCreator::createRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key) +{ + if (!configuration.has(key)) + return {}; + + const auto & regex_str = configuration.getString(key); + const auto & url_regex_rule = std::make_shared(regex_str); + + if (!url_regex_rule->ok()) + throw Exception("cannot compile re2: " + regex_str + " for HTTPHandler url, error: " + url_regex_rule->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); + + return url_regex_rule; +} + +HeadersRegexRule HTTPQueryRequestHandlerMatcherAndCreator::createHeadersRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key) +{ + if (!configuration.has(key)) + return {}; + + Poco::Util::AbstractConfiguration::Keys headers_names; + configuration.keys(key, headers_names); + + HeadersRegexRule headers_regex_rule; + for (const auto & header_name : headers_names) + { + if (headers_regex_rule.count(header_name)) + throw Exception("Duplicate header match declaration '" + header_name + "'", ErrorCodes::LOGICAL_ERROR); + + headers_regex_rule[header_name] = createRegexRule(configuration, key + "." + header_name); + } + + return headers_regex_rule; +} + +size_t findFirstMissingMatchPos(const re2_st::RE2 & regex_rule, const String & match_content) +{ + int num_captures = regex_rule.NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(match_content.data(), match_content.size()); + if (regex_rule.Match(input, 0, match_content.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + return matches[0].size(); + + return size_t(0); +} + +HTTPHandlerMatcher HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher( + const String & method, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) +{ + return [method = Poco::toLower(method), url_rule = url_rule, headers_rule = headers_rule](const Poco::Net::HTTPServerRequest & request) + { + if (!method.empty() && Poco::toLower(request.getMethod()) != method) + return false; + + if (url_rule) + { + Poco::URI uri(request.getURI()); + const auto & request_uri = uri.getPath(); + size_t first_missing_pos = findFirstMissingMatchPos(*url_rule, request_uri); + + const char * url_end = request_uri.data() + request_uri.size(); + const char * first_missing = request_uri.data() + first_missing_pos; + + if (first_missing != url_end && *first_missing == '/') + ++first_missing; + + if (first_missing != url_end && *first_missing != '?') + return false; /// Not full matched + } + + if (!headers_rule.empty()) + { + for (const auto & [header_name, header_rule] : headers_rule) + { + const String & header_value = request.get(header_name); + if (header_value.size() != findFirstMissingMatchPos(*header_rule, header_value)) + return false; + } + } + + return true; + }; +} + +HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer & server, const String & key) +{ + return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, + HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); +} + + +HTTPHandlerMatcher createPredefineQueryHandlerMatcher(IServer & server, const String & key) +{ + return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, + HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); +} + +HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer & server, const String & key) +{ + return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( + server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) + { + const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); + + return [&, query_extract = extract]() + { + return new HTTPQueryRequestHandler(server, *query_extract); + }; + }); +} + +HTTPHandlerCreator createPredefineQueryHandlerCreator(IServer & server, const String & key) +{ + return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( + server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) + { + const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); + + return [&, query_extract = extract]() + { + return new HTTPQueryRequestHandler(server, *query_extract); + }; + }); +} +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h new file mode 100644 index 00000000000..456689d1ff5 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h @@ -0,0 +1,92 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "ExtractorContextChange.h" +#include "../HTTPHandlerFactory.h" + +#if USE_RE2_ST +#include +#else +#define re2_st re2 +#endif + +namespace DB +{ + +using RegexRule = std::shared_ptr; +using HeadersRegexRule = std::map; +using ExtractRes = std::vector>; + +class ExtractorDynamicQueryParameters +{ +public: + ExtractorDynamicQueryParameters( + Poco::Util::AbstractConfiguration & configuration, const String & key, + const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ + ); + + bool loadSettingsFromPost() const { return false; } + + ExtractRes extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params); + +private: + const RegexRule url_regex; + const HeadersRegexRule headers_regex; + + String dynamic_param_name; + std::map extract_from_url; + std::map> extract_from_headers; +}; + +class ExtractorPredefineQueryParameters +{ +public: + ExtractorPredefineQueryParameters( + Poco::Util::AbstractConfiguration & configuration, const String & key, + const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ + ); + + bool loadSettingsFromPost() const { return !has_insert_query; } + + ExtractRes extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params); + +private: + const RegexRule url_regex; + const HeadersRegexRule headers_regex; + + NameSet queries_names; + bool has_insert_query{false}; + ExtractRes predefine_queries; + std::map extract_from_url; + std::map> extract_from_headers; +}; + +class HTTPQueryRequestHandlerMatcherAndCreator +{ +public: + template + static auto invokeWithParsedRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key, const NestedFunction & fun) + { + return fun(configuration.getString(key + ".method", ""), createRegexRule(configuration, key + ".url"), + createHeadersRegexRule(configuration, key + ".headers")); + } + + static HTTPHandlerMatcher createHandlerMatcher(const String & method, const RegexRule & url_rule, const HeadersRegexRule & headers_rule); + +private: + static RegexRule createRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key); + + static HeadersRegexRule createHeadersRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key); +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp new file mode 100644 index 00000000000..ea70abbcc6f --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp @@ -0,0 +1,118 @@ +#include "HTTPReplicasStatusRequestHandler.h" + +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + + +HTTPReplicasStatusRequestHandler::HTTPReplicasStatusRequestHandler(Context & context_) + : context(context_) +{ +} + +void HTTPReplicasStatusRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) +{ + try + { + HTMLForm params(request); + + /// Even if lag is small, output detailed information about the lag. + bool verbose = params.get("verbose", "") == "1"; + + const MergeTreeSettings & settings = context.getMergeTreeSettings(); + + bool ok = true; + std::stringstream message; + + auto databases = context.getDatabases(); + + /// Iterate through all the replicated tables. + for (const auto & db : databases) + { + /// Lazy database can not contain replicated tables + if (db.second->getEngineName() == "Lazy") + continue; + + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + auto & table = iterator->table(); + StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); + + if (!table_replicated) + continue; + + time_t absolute_delay = 0; + time_t relative_delay = 0; + + table_replicated->getReplicaDelays(absolute_delay, relative_delay); + + if ((settings.min_absolute_delay_to_close && absolute_delay >= static_cast(settings.min_absolute_delay_to_close)) + || (settings.min_relative_delay_to_close && relative_delay >= static_cast(settings.min_relative_delay_to_close))) + ok = false; + + message << backQuoteIfNeed(db.first) << "." << backQuoteIfNeed(iterator->name()) + << ":\tAbsolute delay: " << absolute_delay << ". Relative delay: " << relative_delay << ".\n"; + } + } + + const auto & config = context.getConfigRef(); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); + + if (ok && !verbose) + { + const char * data = "Ok.\n"; + response.sendBuffer(data, strlen(data)); + } + else + { + response.send() << message.rdbuf(); + } + } + catch (...) + { + tryLogCurrentException("HTTPReplicasStatusRequestHandler"); + + try + { + response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); + + if (!response.sent()) + { + /// We have not sent anything yet and we don't even know if we need to compress response. + response.send() << getCurrentExceptionMessage(false) << std::endl; + } + } + catch (...) + { + LOG_ERROR((&Logger::get("HTTPReplicasStatusRequestHandler")), "Cannot send exception to client"); + } + } +} + +HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer & server, const String & key) +{ + const auto & prefix = server.config().getString(key, "/replicas_status"); + + return [&, prefix = prefix](const Poco::Net::HTTPServerRequest & request) + { + return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) && + startsWith(request.getURI(), prefix); + }; +} + +HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer & server, const String &) +{ + return [&]() { return new HTTPReplicasStatusRequestHandler(server.context()); }; +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h new file mode 100644 index 00000000000..277e59eb02c --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h @@ -0,0 +1,25 @@ +#pragma once + +#include "../HTTPHandlerFactory.h" +#include + + +namespace DB +{ + +class Context; + +/// Replies "Ok.\n" if all replicas on this server don't lag too much. Otherwise output lag information. +class HTTPReplicasStatusRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit HTTPReplicasStatusRequestHandler(Context & context_); + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + Context & context; +}; + + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp new file mode 100644 index 00000000000..032b51d5b7a --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp @@ -0,0 +1,45 @@ +#include "HTTPRootRequestHandler.h" + +#include + +#include + +#include +#include + +namespace DB +{ + +void HTTPRootRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) +{ + try + { + const auto & config = server.config(); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); + + response.setContentType("text/html; charset=UTF-8"); + + const std::string data = config.getString("http_server_default_response", "Ok.\n"); + response.sendBuffer(data.data(), data.size()); + } + catch (...) + { + tryLogCurrentException("HTTPRootRequestHandler"); + } +} + +HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &) +{ + return [&](const Poco::Net::HTTPServerRequest & request) -> bool + { + return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) + && request.getURI() == "/"; + }; +} + +HTTPHandlerCreator createRootHandlerCreator(IServer & server, const String &) +{ + return [&]() { return new HTTPRootRequestHandler(server); }; +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h new file mode 100644 index 00000000000..136f3292385 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h @@ -0,0 +1,26 @@ +#pragma once + +#include "../IServer.h" +#include "../HTTPHandlerFactory.h" + +#include + + +namespace DB +{ + +/// Response with custom string. Can be used for browser. +class HTTPRootRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit HTTPRootRequestHandler(const IServer & server_) : server(server_) + { + } + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + const IServer & server; +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp new file mode 100644 index 00000000000..840044634aa --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp @@ -0,0 +1,105 @@ +#include "HTTPSessionContextHolder.h" +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int REQUIRED_PASSWORD; + extern const int INVALID_SESSION_TIMEOUT; +} + +static std::chrono::steady_clock::duration parseSessionTimeout( + const Poco::Util::AbstractConfiguration & config, + const HTMLForm & params) +{ + unsigned session_timeout = config.getInt("default_session_timeout", 60); + + if (params.has("session_timeout")) + { + unsigned max_session_timeout = config.getUInt("max_session_timeout", 3600); + std::string session_timeout_str = params.get("session_timeout"); + + ReadBufferFromString buf(session_timeout_str); + if (!tryReadIntText(session_timeout, buf) || !buf.eof()) + throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); + + if (session_timeout > max_session_timeout) + throw Exception("Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) + + ". Maximum session timeout could be modified in configuration file.", + ErrorCodes::INVALID_SESSION_TIMEOUT); + } + + return std::chrono::seconds(session_timeout); +} + +HTTPSessionContextHolder::~HTTPSessionContextHolder() +{ + if (session_context) + session_context->releaseSession(session_id, session_timeout); +} + +void HTTPSessionContextHolder::authentication(Poco::Net::HTTPServerRequest & request, HTMLForm & params) +{ + auto user = request.get("X-ClickHouse-User", ""); + auto password = request.get("X-ClickHouse-Key", ""); + auto quota_key = request.get("X-ClickHouse-Quota", ""); + + if (user.empty() && password.empty() && quota_key.empty()) + { + /// User name and password can be passed using query parameters + /// or using HTTP Basic auth (both methods are insecure). + if (request.hasCredentials()) + { + Poco::Net::HTTPBasicCredentials credentials(request); + + user = credentials.getUsername(); + password = credentials.getPassword(); + } + else + { + user = params.get("user", "default"); + password = params.get("password", ""); + } + + quota_key = params.get("quota_key", ""); + } + else + { + /// It is prohibited to mix different authorization schemes. + if (request.hasCredentials() + || params.has("user") + || params.has("password") + || params.has("quota_key")) + { + throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); + } + } + + std::string query_id = params.get("query_id", ""); + query_context.setUser(user, password, request.clientAddress(), quota_key); + query_context.setCurrentQueryId(query_id); +} + +HTTPSessionContextHolder::HTTPSessionContextHolder(Context & query_context_, Poco::Net::HTTPServerRequest & request, HTMLForm & params) + : query_context(query_context_) +{ + authentication(request, params); + + { + session_id = params.get("session_id", ""); + + if (!session_id.empty()) + { + session_timeout = parseSessionTimeout(query_context.getConfigRef(), params); + session_context = query_context.acquireSession(session_id, session_timeout, params.check("session_check", "1")); + + query_context = *session_context; + query_context.setSessionContext(*session_context); + } + } +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h new file mode 100644 index 00000000000..9ef8b8f7abe --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct HTTPSessionContextHolder +{ + ~HTTPSessionContextHolder(); + + void authentication(Poco::Net::HTTPServerRequest & request, HTMLForm & params); + + HTTPSessionContextHolder(Context & query_context_, Poco::Net::HTTPServerRequest & request, HTMLForm & params); + + String session_id; + Context & query_context; + std::shared_ptr session_context = nullptr; + std::chrono::steady_clock::duration session_timeout; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp b/dbms/src/DataStreams/HTTPInputStreams.cpp similarity index 71% rename from dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp rename to dbms/src/DataStreams/HTTPInputStreams.cpp index 3321b57b38d..3e5d6c01ab3 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp +++ b/dbms/src/DataStreams/HTTPInputStreams.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include @@ -31,12 +31,12 @@ HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & reques } } -ReadBufferPtr HTTPInputStreams::createRawInBuffer(HTTPServerRequest & request) const +std::unique_ptr HTTPInputStreams::createRawInBuffer(HTTPServerRequest & request) const { return std::make_unique(request.stream()); } -ReadBufferPtr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const +std::unique_ptr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & request, std::unique_ptr & raw_buffer) const { /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_compressed_method = request.get("Content-Encoding", ""); @@ -44,21 +44,22 @@ ReadBufferPtr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & reque if (!http_compressed_method.empty()) { if (http_compressed_method == "gzip") - return std::make_shared(*raw_buffer, CompressionMethod::Gzip); + return std::make_unique(std::move(raw_buffer), CompressionMethod::Gzip); else if (http_compressed_method == "deflate") - return std::make_shared(*raw_buffer, CompressionMethod::Zlib); + return std::make_unique(std::move(raw_buffer), CompressionMethod::Zlib); #if USE_BROTLI else if (http_compressed_method == "br") - return std::make_shared(*raw_buffer); + return std::make_unique(std::move(raw_buffer)); #endif else throw Exception("Unknown Content-Encoding of HTTP request: " + http_compressed_method, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); } - return raw_buffer; + return std::move(raw_buffer); } -ReadBufferPtr HTTPInputStreams::createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const +std::unique_ptr HTTPInputStreams::createInternalCompressedBuffer( + HTMLForm & params, std::unique_ptr & http_maybe_encoding_buffer) const { /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. @@ -66,7 +67,7 @@ ReadBufferPtr HTTPInputStreams::createInternalCompressedBuffer(HTMLForm & params if (params.getParsed("decompress", false)) return std::make_unique(*http_maybe_encoding_buffer); - return http_maybe_encoding_buffer; + return std::move(http_maybe_encoding_buffer); } } diff --git a/dbms/src/DataStreams/HTTPInputStreams.h b/dbms/src/DataStreams/HTTPInputStreams.h new file mode 100644 index 00000000000..652f5370a5a --- /dev/null +++ b/dbms/src/DataStreams/HTTPInputStreams.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +using HTTPServerRequest = Poco::Net::HTTPServerRequest; + +struct HTTPInputStreams +{ + using ReadBufferUniquePtr = std::unique_ptr; + + ReadBufferUniquePtr in; + ReadBufferUniquePtr in_maybe_compressed; + ReadBufferUniquePtr in_maybe_internal_compressed; + + HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); + + ReadBufferUniquePtr createRawInBuffer(HTTPServerRequest & request) const; + ReadBufferUniquePtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferUniquePtr & raw_buffer) const; + ReadBufferUniquePtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferUniquePtr & http_maybe_encoding_buffer) const; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp b/dbms/src/DataStreams/HTTPOutputStreams.cpp similarity index 99% rename from dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp rename to dbms/src/DataStreams/HTTPOutputStreams.cpp index 8ac682af5e4..31f4929bef1 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp +++ b/dbms/src/DataStreams/HTTPOutputStreams.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h b/dbms/src/DataStreams/HTTPOutputStreams.h similarity index 100% rename from dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h rename to dbms/src/DataStreams/HTTPOutputStreams.h diff --git a/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h deleted file mode 100644 index 56fa001d57c..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class AlwaysQueryMatcher : public QueryMatcher -{ -public: - bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } - - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp deleted file mode 100644 index e6ed579b696..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_CUSTOM_EXECUTOR_PARAM; -} - -void prepareQueryParams(Context & context, HTMLForm & params, const NameSet & query_params_name) -{ - for (const auto & param : params) - if (query_params_name.count(param.first)) - context.setQueryParameter(param.first, param.second); -} - -QueryExecutorConst::QueryExecutorConst(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) -{ - execute_query = configuration.getString(config_key); - - const char * query_begin = execute_query.data(); - const char * query_end = execute_query.data() + execute_query.size(); - - ParserQuery parser(query_end, false); - ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); - - QueryParameterVisitor{query_params_name}.visit(extract_query_ast); - can_be_parse_request_body = !extract_query_ast->as(); - - const auto & reserved_params_name = ExtractorContextChange::getReservedParamNames(); - for (const auto & prepared_param_name : query_params_name) - { - if (Settings::findIndex(prepared_param_name) != Settings::npos || reserved_params_name.count(prepared_param_name)) - throw Exception( - "Illegal custom executor query param name '" + prepared_param_name + "', Because it's a reserved name or Settings name", - ErrorCodes::ILLEGAL_CUSTOM_EXECUTOR_PARAM); - } -} - -void QueryExecutorConst::executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const -{ - ReadBufferPtr temp_query_buf; - prepareQueryParams(context, params, query_params_name); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - if (!canBeParseRequestBody() && !startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h deleted file mode 100644 index 170d2a959ef..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class QueryExecutorConst : public QueryExecutor -{ -public: - using HTTPServerRequest = Poco::Net::HTTPServerRequest; - using HTTPServerResponse = Poco::Net::HTTPServerResponse; - - bool canBeParseRequestBody() const override { return can_be_parse_request_body; } - - bool isQueryParam(const String & param_name) const override { return query_params_name.count(param_name); } - - QueryExecutorConst(const Poco::Util::AbstractConfiguration & configuration, const String & config_key); - - void executeQueryImpl( - Context & context, HTTPServerRequest & request, HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override; - -private: - String execute_query; - NameSet query_params_name; - bool can_be_parse_request_body{false}; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp deleted file mode 100644 index 3e984deb8aa..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ /dev/null @@ -1,201 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SYNTAX_ERROR; - extern const int UNKNOW_QUERY_EXECUTOR; - extern const int TOO_MANY_INPUT_CUSTOM_EXECUTOR; -} - -CustomExecutor::CustomExecutor( - const std::vector & matchers_, - const std::vector & query_executors_) - : matchers(matchers_), query_executors(query_executors_) -{ -} - -bool CustomExecutor::match(Context & context, HTTPRequest & request, HTMLForm & params) const -{ - for (const auto & matcher : matchers) - { - if (!matcher->match(context, request, params)) - return false; - } - - return true; -} - -bool CustomExecutor::isQueryParam(const String & param_name) const -{ - for (const auto & query_executor : query_executors) - { - if (!query_executor->isQueryParam(param_name)) - return false; - } - - return true; -} - -bool CustomExecutor::canBeParseRequestBody() const -{ - for (const auto & query_executor : query_executors) - { - if (!query_executor->canBeParseRequestBody()) - return false; - } - - return true; -} - -void CustomExecutor::executeQuery( - Context & context, HTTPRequest & request, HTTPResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) -{ - for (const auto & query_executor : query_executors) - query_executor->executeQueryImpl(context, request, response, params, input_streams, output_streams); - - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. - output_streams.finalize(); -} - -void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & /*settings*/, const String & config_prefix) -{ - Configuration::Keys custom_executors_keys; - config.keys(config_prefix, custom_executors_keys); - - std::vector> new_custom_executors; - - for (const auto & custom_executor_key : custom_executors_keys) - { - if (custom_executor_key.find('.') != String::npos) - throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); - - new_custom_executors.push_back({custom_executor_key, createCustomExecutor(config, config_prefix, custom_executor_key)}); - } - - std::unique_lock lock(rwlock); - custom_executors = new_custom_executors; -} - -void CustomExecutors::registerCustomMatcher(const String & matcher_name, const CustomExecutors::CustomMatcherCreator & creator) -{ - const auto & matcher_creator_it = custom_matcher_creators.find(matcher_name); - const auto & query_executor_creator_it = query_executor_creators.find(matcher_name); - - if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) - throw Exception("LOGICAL_ERROR QueryMatcher name must be unique between the QueryExecutor and QueryMatcher.", - ErrorCodes::LOGICAL_ERROR); - - custom_matcher_creators[matcher_name] = creator; -} - -void CustomExecutors::registerQueryExecutor(const String & query_executor_name, const CustomExecutors::QueryExecutorCreator & creator) -{ - const auto & matcher_creator_it = custom_matcher_creators.find(query_executor_name); - const auto & query_executor_creator_it = query_executor_creators.find(query_executor_name); - - if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) - throw Exception("LOGICAL_ERROR QueryExecutor name must be unique between the QueryExecutor and QueryMatcher.", - ErrorCodes::LOGICAL_ERROR); - - query_executor_creators[query_executor_name] = creator; -} - -String fixMatcherOrExecutorTypeName(const String & matcher_or_executor_type_name) -{ - auto type_name_end_pos = matcher_or_executor_type_name.find('['); - return type_name_end_pos == String::npos ? matcher_or_executor_type_name : matcher_or_executor_type_name.substr(0, type_name_end_pos); -} - -CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & config, const String & config_prefix, const String & name) -{ - Configuration::Keys matchers_key; - config.keys(config_prefix + "." + name, matchers_key); - - std::vector query_matchers; - std::vector query_executors; - - for (const auto & matcher_key : matchers_key) - { - String matcher_or_query_executor_type = fixMatcherOrExecutorTypeName(matcher_key); - - if (matcher_or_query_executor_type.find('.') != String::npos) - throw Exception("CustomMatcher or QueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", - ErrorCodes::SYNTAX_ERROR); - - const auto & matcher_creator_it = custom_matcher_creators.find(matcher_or_query_executor_type); - const auto & query_executor_creator_it = query_executor_creators.find(matcher_or_query_executor_type); - - if (matcher_creator_it == custom_matcher_creators.end() && query_executor_creator_it == query_executor_creators.end()) - throw Exception("CustomMatcher or QueryExecutor '" + matcher_or_query_executor_type + "' is not implemented.", - ErrorCodes::NOT_IMPLEMENTED); - - if (matcher_creator_it != custom_matcher_creators.end()) - query_matchers.push_back(matcher_creator_it->second(config, config_prefix + "." + name + "." + matcher_key)); - - if (query_executor_creator_it != query_executor_creators.end()) - query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + name + "." + matcher_key)); - } - - checkQueryMatchersAndExecutors(name, query_matchers, query_executors); - return std::make_shared(query_matchers, query_executors); -} - -void CustomExecutors::checkQueryMatchersAndExecutors( - const String & name, std::vector & matchers, std::vector & query_executors) -{ - if (matchers.empty() || query_executors.empty()) - throw Exception("The CustomExecutor '" + name + "' must contain a Matcher and a QueryExecutor.", ErrorCodes::SYNTAX_ERROR); - - const auto & sum_func = [&](auto & ele) -> bool { return !ele->canBeParseRequestBody(); }; - const auto & need_post_data_count = std::count_if(query_executors.begin(), query_executors.end(), sum_func); - - if (need_post_data_count > 1) - throw Exception("The CustomExecutor '" + name + "' can only contain one insert query." + toString(need_post_data_count), ErrorCodes::TOO_MANY_INPUT_CUSTOM_EXECUTOR); - - for (const auto & matcher : matchers) - matcher->checkQueryExecutors(query_executors); -} - -std::pair CustomExecutors::getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const -{ - std::shared_lock lock(rwlock); - - for (const auto & custom_executor : custom_executors) - if (custom_executor.second->match(context, request, params)) - return custom_executor; - - throw Exception("No CustomExecutor match " + request.getURI(), ErrorCodes::UNKNOW_QUERY_EXECUTOR); -} - -CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) -{ - registerCustomMatcher("URL", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); - - registerCustomMatcher("method", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); - - registerCustomMatcher("always_matched", [&](const auto & /*matcher_config*/, const auto & /*prefix*/) - { return std::make_shared(); }); - - registerQueryExecutor("query", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); - - registerQueryExecutor("dynamic_query", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); - - updateCustomExecutors(config, settings, config_prefix); -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h deleted file mode 100644 index 7b37a6afdf5..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ /dev/null @@ -1,77 +0,0 @@ -#pragma once - -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -class CustomExecutor; - -using HTTPRequest = Poco::Net::HTTPServerRequest; -using HTTPResponse = Poco::Net::HTTPServerResponse; -using CustomExecutorPtr = std::shared_ptr; - -class CustomExecutors -{ -public: - using Configuration = Poco::Util::AbstractConfiguration; - CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix = "custom_http"); - - CustomExecutors(const CustomExecutors &) = delete; - CustomExecutors & operator=(const CustomExecutors &) = delete; - - using QueryExecutorCreator = std::function; - void registerQueryExecutor(const String & query_executor_name, const QueryExecutorCreator & creator); - - using CustomMatcherCreator = std::function; - void registerCustomMatcher(const String & matcher_name, const CustomMatcherCreator & creator); - - void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); - - std::pair getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const; -private: - mutable std::shared_mutex rwlock; - std::vector> custom_executors; - std::unordered_map query_executor_creators; - std::unordered_map custom_matcher_creators; - - CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix, const String & name); - - void checkQueryMatchersAndExecutors(const String & name, std::vector & matchers, std::vector & query_executors); - -}; - -class CustomExecutor -{ -public: - bool canBeParseRequestBody() const; - - bool isQueryParam(const String & param_name) const; - - bool match(Context & context, HTTPRequest & request, HTMLForm & params) const; - - void executeQuery( - Context & context, HTTPRequest & request, HTTPResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams - ); - - CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); - -private: - std::vector matchers; - std::vector query_executors; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp deleted file mode 100644 index 9b4a4006556..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp +++ /dev/null @@ -1,60 +0,0 @@ -#include - -#include -#include -#include - -namespace DB -{ - -bool QueryExecutorDynamic::isQueryParam(const String & param_name) const -{ - return param_name == dynamic_param_name || startsWith(param_name, "param_"); -} - -QueryExecutorDynamic::QueryExecutorDynamic(const Configuration & configuration, const String & config_key) -{ - dynamic_param_name = configuration.getString(config_key + "." + "param_name", "query"); -} - -void QueryExecutorDynamic::executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const -{ - ReadBufferPtr temp_query_buf; - const auto & execute_query = prepareQuery(context, params); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - if (!startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, - context, [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); -} - -String QueryExecutorDynamic::prepareQuery(Context & context, HTMLForm & params) const -{ - const static size_t prefix_size = strlen("param_"); - - WriteBufferFromOwnString query_buffer; - for (const auto & param : params) - { - if (param.first == dynamic_param_name) - writeString(param.second, query_buffer); - else if (startsWith(param.first, "param_")) - context.setQueryParameter(param.first.substr(prefix_size), param.second); - } - - if (query_buffer.offset()) - writeString("\n", query_buffer); - - return query_buffer.str(); -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h deleted file mode 100644 index a09146246d4..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class QueryExecutorDynamic : public QueryExecutor -{ -public: - using HTTPServerRequest = Poco::Net::HTTPServerRequest; - using HTTPServerResponse = Poco::Net::HTTPServerResponse; - using Configuration = Poco::Util::AbstractConfiguration; - - bool canBeParseRequestBody() const override { return false; } - - bool isQueryParam(const String & param_name) const override; - - QueryExecutorDynamic(const Configuration & configuration, const String & config_key); - - void executeQueryImpl( - Context & context, HTTPServerRequest & request, HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override; - -private: - String dynamic_param_name{"query"}; - - String prepareQuery(Context & context, HTMLForm & params) const; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h deleted file mode 100644 index 389d4312362..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -using HTTPServerRequest = Poco::Net::HTTPServerRequest; - -struct HTTPInputStreams -{ - std::shared_ptr in; - std::shared_ptr in_maybe_compressed; - std::shared_ptr in_maybe_internal_compressed; - - HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); - - ReadBufferPtr createRawInBuffer(HTTPServerRequest & request) const; - ReadBufferPtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const; - ReadBufferPtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h deleted file mode 100644 index 689aff67dc7..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include "QueryExecutorAndMatcher.h" - -namespace DB -{ - -class MethodQueryMatcher : public QueryMatcher -{ -public: - using Config = Poco::Util::AbstractConfiguration; - - MethodQueryMatcher(const Config & configuration, const String & config_key) - : method(Poco::toLower(configuration.getString(config_key))) - { - } - - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override - { - return Poco::toLower(request.getMethod()) == method; - } - - bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } - -private: - String method; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h b/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h deleted file mode 100644 index 7f32c4c2ddb..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h +++ /dev/null @@ -1,47 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_COMPILE_REGEXP; -} - -class QueryExecutor -{ -public: - virtual ~QueryExecutor() = default; - - virtual bool isQueryParam(const String &) const = 0; - virtual bool canBeParseRequestBody() const = 0; - - virtual void executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; -}; - -using QueryExecutorPtr = std::shared_ptr; - - -class QueryMatcher -{ -public: - virtual ~QueryMatcher() = default; - - virtual bool checkQueryExecutors(const std::vector &check_executors) const = 0; - - virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; -}; - -using QueryMatcherPtr = std::shared_ptr; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp deleted file mode 100644 index 82bb864c926..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include -#include "URLQueryMatcher.h" - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_COMPILE_REGEXP; - extern const int UNDEFINED_CUSTOM_EXECUTOR_PARAM; -} - -URLQueryMatcher::URLQueryMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) -{ - const auto & regex_str = configuration.getString(config_key); - regex_matcher = std::make_unique(regex_str); - - if (!regex_matcher->ok()) - throw Exception("cannot compile re2: " + regex_str + ", error: " + regex_matcher->error() + - ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); -} - -bool checkQueryOneQueryParam(const String & param_name, const std::vector & custom_query_executors) -{ - for (const auto & custom_query_executor : custom_query_executors) - if (custom_query_executor->isQueryParam(param_name)) - return true; - - return false; -} - -bool URLQueryMatcher::checkQueryExecutors(const std::vector & custom_query_executors) const -{ - for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) - if (!checkQueryOneQueryParam(named_capturing_group.first, custom_query_executors)) - throw Exception("The param name '" + named_capturing_group.first + "' is not defined in the QueryExecutor.", - ErrorCodes::UNDEFINED_CUSTOM_EXECUTOR_PARAM); - - return true; -} - -bool URLQueryMatcher::match(Context & context, Poco::Net::HTTPServerRequest &request, HTMLForm &) const -{ - const String request_uri = request.getURI(); - int num_captures = regex_matcher->NumberOfCapturingGroups() + 1; - - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(request_uri.data(), request_uri.size()); - if (regex_matcher->Match(input, 0, request_uri.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) - { - const auto & full_match = matches[0]; - const char * url_end = request_uri.data() + request_uri.size(); - const char * not_matched_begin = request_uri.data() + full_match.size(); - - if (not_matched_begin != url_end && *not_matched_begin == '/') - ++not_matched_begin; - - if (not_matched_begin == url_end || *not_matched_begin == '?') - { - for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) - { - const auto & capturing_value = matches[named_capturing_group.second]; - context.setQueryParameter(named_capturing_group.first, String(capturing_value.data(), capturing_value.size())); - } - - return true; - } - } - return false; -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h deleted file mode 100644 index 61323db3e53..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h +++ /dev/null @@ -1,32 +0,0 @@ -#pragma once - -#include -#include - -#include -#include - -#if USE_RE2_ST -# include -#else -# define re2_st re2 -#endif - - -namespace DB -{ - -class URLQueryMatcher : public QueryMatcher -{ -public: - URLQueryMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & config_key); - - bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm &) const override; - - bool checkQueryExecutors(const std::vector & custom_query_executors) const override; - -private: - std::unique_ptr regex_matcher; -}; - -} diff --git a/dbms/tests/integration/test_custom_http/normally_configs/config.xml b/dbms/tests/integration/test_custom_http/normally_configs/config.xml deleted file mode 100644 index 6fab88bcf80..00000000000 --- a/dbms/tests/integration/test_custom_http/normally_configs/config.xml +++ /dev/null @@ -1,30 +0,0 @@ - - - - - - PUT - /test_for_only_insert_queries - INSERT INTO test.test VALUES - - - - POST - /test_for_only_select_queries - SELECT value FROM system.settings WHERE name = 'max_threads' - SELECT value FROM system.settings WHERE name = 'max_alter_threads' - - - - /test_for_hybrid_insert_and_select_queries - INSERT INTO test.test VALUES - SELECT * FROM test.test ORDER BY id - - - - /test_for_throw_exception_when_after_select - SELECT * FROM test.test ORDER BY id - SELECT throwIf(number = 2, 'Throw Exception') FROM numbers(3) - - - diff --git a/dbms/tests/integration/test_custom_http/test.py b/dbms/tests/integration/test_custom_http/test.py deleted file mode 100644 index 08f8d37586c..00000000000 --- a/dbms/tests/integration/test_custom_http/test.py +++ /dev/null @@ -1,38 +0,0 @@ -import os - -import pytest - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -SCRIPT_PATH = os.path.dirname(os.path.realpath(__file__)) - - -def add_instance(name, config_dir): - print os.path.join(SCRIPT_PATH, 'common_configs', 'common_users.xml') - print os.path.join(SCRIPT_PATH, 'common_configs', 'common_config.xml') - return cluster.add_instance(name, config_dir=os.path.join(SCRIPT_PATH, config_dir), - main_configs=[os.path.join(SCRIPT_PATH, 'common_configs', 'common_config.xml')], - user_configs=[os.path.join(SCRIPT_PATH, 'common_configs', 'common_users.xml')]) - - -normally_instance = add_instance("normally_node", "normally_configs") - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - normally_instance.query('CREATE DATABASE `test`') - normally_instance.query('CREATE TABLE `test`.`test` (`id` UInt8) Engine = Memory') - yield cluster - finally: - cluster.shutdown() - - -def test_normally_match(started_cluster): - assert normally_instance.http_request('test_for_only_insert_queries', method='PUT', data='(1)(2)(3)') == '' - assert normally_instance.http_request(url='test_for_only_select_queries', - params='max_threads=1', method='POST', data='max_alter_threads=2') == '1\n2\n' - assert normally_instance.http_request('test_for_hybrid_insert_and_select_queries', method='POST', data='(4)') == '1\n2\n3\n4\n' - assert 'Throw Exception' in normally_instance.http_request('test_for_throw_exception_when_after_select') diff --git a/dbms/tests/integration/test_custom_http/__init__.py b/dbms/tests/integration/test_http_handlers_config/__init__.py similarity index 100% rename from dbms/tests/integration/test_custom_http/__init__.py rename to dbms/tests/integration/test_http_handlers_config/__init__.py diff --git a/dbms/tests/integration/test_custom_http/common_configs/common_config.xml b/dbms/tests/integration/test_http_handlers_config/common_configs/common_config.xml similarity index 100% rename from dbms/tests/integration/test_custom_http/common_configs/common_config.xml rename to dbms/tests/integration/test_http_handlers_config/common_configs/common_config.xml diff --git a/dbms/tests/integration/test_custom_http/common_configs/common_users.xml b/dbms/tests/integration/test_http_handlers_config/common_configs/common_users.xml similarity index 100% rename from dbms/tests/integration/test_custom_http/common_configs/common_users.xml rename to dbms/tests/integration/test_http_handlers_config/common_configs/common_users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml new file mode 100644 index 00000000000..f117eaba2f7 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml @@ -0,0 +1,18 @@ + + + + + + /ping_test + + POST + /test_one_handler_with_insert_and_select + + INSERT INTO test.test VALUES(1) + SELECT COUNT() FROM test.test WHERE id = {id:UInt8} + + + + test not found + + diff --git a/dbms/tests/integration/test_custom_http/normally_configs/users.xml b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/users.xml similarity index 100% rename from dbms/tests/integration/test_custom_http/normally_configs/users.xml rename to dbms/tests/integration/test_http_handlers_config/other_tests_configs/users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/test.py b/dbms/tests/integration/test_http_handlers_config/test.py new file mode 100644 index 00000000000..9e332dd8675 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test.py @@ -0,0 +1,87 @@ +import os +import urllib +import contextlib + +from helpers.cluster import ClickHouseCluster + + +class SimpleCluster: + def close(self): + self.cluster.shutdown() + + def __init__(self, cluster, name, config_dir): + self.cluster = cluster + self.instance = self.add_instance(name, config_dir) + cluster.start() + + def add_instance(self, name, config_dir): + script_path = os.path.dirname(os.path.realpath(__file__)) + return self.cluster.add_instance(name, config_dir=os.path.join(script_path, config_dir), + main_configs=[os.path.join(script_path, 'common_configs', 'common_config.xml')], + user_configs=[os.path.join(script_path, 'common_configs', 'common_users.xml')]) + + +def test_dynamic_query_handler_with_insert_and_select(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "dynamic_insert_and_select", "test_insert_and_select_dynamic")) as cluster: + insert_data_query = urllib.quote_plus('INSERT INTO test.test VALUES') + select_data_query = urllib.quote_plus('SELECT * FROM test.test ORDER BY id') + create_database_query = urllib.quote_plus('CREATE DATABASE test') + create_test_table_query = 'CREATE TABLE test.test (id UInt8) Engine = Memory' + assert cluster.instance.http_request('create_test_table?max_threads=1&test_create_query_param=' + create_database_query, method='PUT') == '' + assert cluster.instance.http_request('create_test_table?max_threads=1', method='PUT', data=create_test_table_query) == '' + assert cluster.instance.http_request('insert_data_to_test?max_threads=1&test_insert_query_param=' + insert_data_query + '(1)', method='POST') == '' + assert cluster.instance.http_request('insert_data_to_test?max_threads=1&test_insert_query_param=' + insert_data_query, method='POST', data='(2)') == '' + assert cluster.instance.http_request('insert_data_to_test?max_threads=1', method='POST', data='INSERT INTO test.test VALUES(3)(4)') == '' + assert cluster.instance.http_request('query_data_from_test?max_threads=1&test_select_query_param=' + select_data_query, method='GET') == '1\n2\n3\n4\n' + + +def test_predefine_query_handler_with_insert_and_select(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefine_insert_and_select", "test_insert_and_select_predefine")) as cluster: + assert cluster.instance.http_request('create_test_table?max_threads=1', method='PUT') == '' + assert cluster.instance.http_request('insert_data_to_test?max_threads=1', method='POST', data='(1)(2)(3)(4)') == '' + assert cluster.instance.http_request('query_data_from_test?max_threads=1', method='GET') == '1\n2\n3\n4\n' + + +def test_dynamic_query_handler_with_params_and_settings(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "dynamic_params_and_settings", "test_param_and_settings_dynamic")) as cluster: + settings = 'max_threads=1&max_alter_threads=2' + query_param = 'param_name_1=max_threads¶m_name_2=max_alter_threads' + test_query = 'SELECT value FROM system.settings where name = {name_1:String} OR name = {name_2:String}' + quoted_test_query = urllib.quote_plus(test_query) + assert cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + quoted_test_query + '&' + query_param + '&' + settings, method='POST') == '1\n2\n' + assert cluster.instance.http_request('post_query_params_and_settings?' + query_param + '&' + settings, method='POST', data=test_query) == '1\n2\n' + assert 'Syntax error' in cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + test_query + '&' + settings, method='POST', data=query_param) + assert 'Syntax error' in cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + test_query + '&' + query_param, method='POST', data=settings) + + assert cluster.instance.http_request('get_query_params_and_settings?get_query_param=' + quoted_test_query + '&' + query_param + '&' + settings) == '1\n2\n' + assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?query_param=' + quoted_test_query + '&' + settings) == '1\n2\n' + assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?query_param=' + quoted_test_query + '&' + query_param + '&' + settings) + + assert cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\n2\n' + + +def test_predefine_query_handler_with_params_and_settings(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefine_params_and_settings", "test_param_and_settings_predefine")) as cluster: + settings = 'max_threads=1&max_alter_threads=2' + query_param = 'name_1=max_threads&name_2=max_alter_threads' + assert cluster.instance.http_request('get_query_params_and_settings?' + query_param + '&' + settings, method='GET') == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?' + settings) == '1\nmax_alter_threads\t2\n' + assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?' + query_param + '&' + settings) + + assert cluster.instance.http_request('post_query_params_and_settings?' + query_param, method='POST', data=settings) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('post_query_params_and_settings?' + settings, method='POST', data=query_param) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('post_query_params_and_settings?' + query_param + '&' + settings, method='POST') == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('post_query_params_and_settings', method='POST', data=query_param + '&' + settings) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('test_match_headers?' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\nmax_alter_threads\t2\n' + + +def test_other_configs(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "test_other_configs", "other_tests_configs")) as cluster: + assert cluster.instance.http_request('', method='GET') == 'Ok.\n' + assert cluster.instance.http_request('ping_test', method='GET') == 'Ok.\n' + assert cluster.instance.http_request('404/NOT_FOUND', method='GET') == 'There is no handle /404/NOT_FOUND\n\ntest not found\n' + + cluster.instance.query('CREATE DATABASE test') + cluster.instance.query('CREATE TABLE test.test (id UInt8) Engine = Memory') + assert cluster.instance.http_request('test_one_handler_with_insert_and_select?id=1', method='POST', data='(1)(2)') == '2\n' + assert 'Cannot parse input' in cluster.instance.http_request('test_one_handler_with_insert_and_select', method='POST', data='id=1') diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml new file mode 100644 index 00000000000..a4ec94fe62e --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml @@ -0,0 +1,23 @@ + + + + + + PUT + /create_test_table + test_create_query_param + + + + POST + /insert_data_to_test + test_insert_query_param + + + + GET + /query_data_from_test + test_select_query_param + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml new file mode 100644 index 00000000000..0f0ab81d139 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml @@ -0,0 +1,26 @@ + + + + + + PUT + /create_test_table + + CREATE DATABASE test + CREATE TABLE test.test (id UInt8) Engine = Memory + + + + + POST + /insert_data_to_test + INSERT INTO test.test VALUES + + + + GET + /query_data_from_test + SELECT * FROM test.test ORDER BY id + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml new file mode 100644 index 00000000000..06c7143f3e8 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml @@ -0,0 +1,31 @@ + + + + + + POST + /post_query_params_and_settings + post_query_param + + + + GET + /get_query_params_and_settings + get_query_param + + + + GET + .+)/(?P.+)]]> + query_param + + + + + TEST_HEADER_VALUE + .+)/(?P.+)]]> + + query_param + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml new file mode 100644 index 00000000000..cca4bc04a69 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml @@ -0,0 +1,43 @@ + + + + + + POST + /post_query_params_and_settings + + SELECT value FROM system.settings WHERE name = {name_1:String} + SELECT name, value FROM system.settings WHERE name = {name_2:String} + + + + + GET + /get_query_params_and_settings + + SELECT value FROM system.settings WHERE name = {name_1:String} + SELECT name, value FROM system.settings WHERE name = {name_2:String} + + + + + GET + .+)/(?P.+)]]> + + SELECT value FROM system.settings WHERE name = {name_1:String} + SELECT name, value FROM system.settings WHERE name = {name_2:String} + + + + + + TEST_HEADER_VALUE + .+)/(?P.+)]]> + + + SELECT value FROM system.settings WHERE name = {name_1:String} + SELECT name, value FROM system.settings WHERE name = {name_2:String} + + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index f6e9d1ff702..064cda4847a 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -11,7 +11,15 @@ set(CLICKHOUSE_SERVER_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/RootRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/Server.cpp ${CMAKE_CURRENT_SOURCE_DIR}/TCPHandler.cpp - ) + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPHandlerFactory.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPQueryRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPPingRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPRootRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPSessionContextHolder.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPExceptionHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp +) set(CLICKHOUSE_SERVER_SOURCES ${CLICKHOUSE_SERVER_SOURCES} diff --git a/programs/server/NotFoundHandler.cpp b/programs/server/NotFoundHandler.cpp index 766e8895784..26df1a1b7d4 100644 --- a/programs/server/NotFoundHandler.cpp +++ b/programs/server/NotFoundHandler.cpp @@ -17,16 +17,14 @@ void NotFoundHandler::handleRequest( try { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_NOT_FOUND); - response.send() << "There is no handle " << request.getURI() << "\n\n" - << "Use / or /ping for health checks.\n" - << "Or /replicas_status for more sophisticated health checks.\n\n" - << "Send queries from your program with POST method or GET /?query=...\n\n" - << "Use clickhouse-client:\n\n" - << "For interactive data analysis:\n" - << " clickhouse-client\n\n" - << "For batch query processing:\n" - << " clickhouse-client --query='SELECT 1' > result\n" - << " clickhouse-client < query > result\n"; + + std::stringstream output_description; + output_description << "There is no handle " << request.getURI() << "\n\n"; + + if (!no_handler_description.empty()) + output_description << no_handler_description << "\n"; + + response.send() << output_description.str(); } catch (...) { diff --git a/programs/server/NotFoundHandler.h b/programs/server/NotFoundHandler.h index 7f758e49d0d..caf527726c4 100644 --- a/programs/server/NotFoundHandler.h +++ b/programs/server/NotFoundHandler.h @@ -10,9 +10,12 @@ namespace DB class NotFoundHandler : public Poco::Net::HTTPRequestHandler { public: - void handleRequest( - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response) override; + NotFoundHandler(const std::string & no_handler_description_) : no_handler_description(no_handler_description_) {} + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + const std::string no_handler_description; }; } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 710506151c4..3490ff6a445 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -466,7 +466,6 @@ int Server::main(const std::vector & /*args*/) //setTextLog(global_context->getTextLog()); //buildLoggers(*config, logger()); global_context->setClustersConfig(config); - global_context->setCustomExecutorConfig(config); global_context->setMacros(std::make_unique(*config, "macros")); /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) diff --git a/programs/server/config.xml b/programs/server/config.xml index 706149b66b1..c4ba4e66ab3 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -525,30 +525,30 @@ --> - - - - - - - query - - - - + + + /ping + /replicas_status + query + + Use / or /ping for health checks. Or /replicas_status for more sophisticated health checks. Send queries from your program with POST method or GET /?query=... Use clickhouse-client: For interactive data analysis: clickhouse-client For batch query processing: clickhouse-client --query='SELECT 1' > result clickhouse-client < query > result + + + diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index 25c2974056c..c61572d7097 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -124,7 +124,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() #if defined(POCO_CLICKHOUSE_PATCH) *response_header_ostr << "Content-Encoding: deflate\r\n"; #else - response.set("Content-Encoding", "gzip"); + response.set("Content-Encoding", "deflate"); response_body_ostr = &(response.send()); #endif out_raw = std::make_unique(*response_body_ostr); @@ -135,18 +135,15 @@ void WriteBufferFromHTTPServerResponse::nextImpl() else if (compression_method == CompressionMethod::Brotli) { #if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: deflate\r\n"; + *response_header_ostr << "Content-Encoding: br\r\n"; #else response.set("Content-Encoding", content_encoding_name); #endif - /// Newline autosent by response.send() - /// This may result in an extra empty line in the response body - response_body_ostr = &(response.send()); -#endif - - out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); - out = &*out_raw; + else + throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse", + ErrorCodes::LOGICAL_ERROR); + /// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy. } else { @@ -154,7 +151,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() response_body_ostr = &(response.send()); #endif - out_raw.emplace(*response_body_ostr, working_buffer.size(), working_buffer.begin()); + out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); out = &*out_raw; } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b479a7bc46d..647c3fb8020 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -45,7 +44,6 @@ #include #include #include -#include #include #include #include @@ -350,9 +348,6 @@ struct ContextShared std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config - std::unique_ptr custom_executors; - ConfigurationPtr custom_executors_config; - mutable std::mutex custom_executors_mutex; /// Guards custom executors and their config #if USE_EMBEDDED_COMPILER std::shared_ptr compiled_expression_cache; @@ -1548,6 +1543,7 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptrclusters->setCluster(cluster_name, cluster); } + void Context::initializeSystemLogs() { auto lock = getLock(); @@ -2047,31 +2043,6 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } -void Context::setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix) -{ - std::lock_guard lock(shared->custom_executors_mutex); - - shared->custom_executors_config = config; - - if (!shared->custom_executors) - shared->custom_executors = std::make_unique(*shared->custom_executors_config, settings, config_prefix); - else - shared->custom_executors->updateCustomExecutors(*shared->custom_executors_config, settings, config_prefix); -} - -std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) -{ - std::lock_guard lock(shared->custom_executors_mutex); - - if (!shared->custom_executors) - { - auto & config = shared->custom_executors_config ? *shared->custom_executors_config : getConfigRef(); - shared->custom_executors = std::make_unique(config, settings); - } - - return shared->custom_executors->getCustomExecutor(*this, request, params); -} - StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where) const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bef448a2389..c9213c806ff 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -42,7 +42,6 @@ namespace zkutil class ZooKeeper; } -struct HTMLForm; namespace DB { @@ -107,7 +106,6 @@ class Volume; using VolumePtr = std::shared_ptr; struct NamedSession; - #if USE_EMBEDDED_COMPILER class CompiledExpressionCache; #endif @@ -491,9 +489,6 @@ public: Compiler & getCompiler(); - void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "custom_http"); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params); - /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index e2231e35f03..e55c0b05a2d 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -56,7 +55,6 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) IColumn & temp_column = *temp_column_ptr; ReadBufferFromString read_buffer{value}; FormatSettings format_settings; - data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings); if (!read_buffer.eof()) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index efe47da9f46..76d9e9238be 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -698,10 +698,10 @@ class ClickHouseInstance: raise Exception("ClickHouse HTTP server returned " + http_code_and_message()) return open_result.read() - # Connects to the instance via HTTP interface, sends a query and returns the answer - def http_request(self, url, method='GET', params=None, data=None): + # Connects to the instance via HTTP interface, sends a query and returns the answer + def http_request(self, url, method='GET', params=None, data=None, headers=None): url = "http://" + self.ip_address + ":8123/"+url - return requests.request(method=method, url=url, params=params, data=data).content + return requests.request(method=method, url=url, params=params, data=data, headers=headers).content # Connects to the instance via HTTP interface, sends a query, expects an error and return the error message def http_query_and_get_error(self, sql, data=None, params=None, user=None, password=None): diff --git a/tests/integration/test_https_replication/configs/config.xml b/tests/integration/test_https_replication/configs/config.xml index 35a43b2fc54..be81ccfc145 100644 --- a/tests/integration/test_https_replication/configs/config.xml +++ b/tests/integration/test_https_replication/configs/config.xml @@ -354,6 +354,15 @@ + + + /ping + /replicas_status + query + + Use / or /ping for health checks. Or /replicas_status for more sophisticated health checks. Send queries from your program with POST method or GET /?query=... Use clickhouse-client: For interactive data analysis: clickhouse-client For batch query processing: clickhouse-client --query='SELECT 1' > result clickhouse-client < query > result + + From b82eee5255dc88b83605ba9d46b7c9b995a135ff Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 29 Nov 2019 13:46:46 +0800 Subject: [PATCH 408/743] ISSUES-5436 fix review suggestions & add some '?' re2 test --- dbms/programs/server/HTTPHandlerFactory.cpp | 12 ++--- .../ExtractorContextChange.h | 4 +- .../HTTPQueryRequestHandler.cpp | 2 +- ...TPQueryRequestHandlerMatcherAndCreator.cpp | 45 +++++++++++-------- ...HTTPQueryRequestHandlerMatcherAndCreator.h | 4 +- .../HTTPSessionContextHolder.h | 1 + .../other_tests_configs/config.xml | 4 +- .../test_http_handlers_config/test.py | 18 ++++++-- .../config.xml | 12 ++--- .../users.xml | 0 .../config.xml | 4 +- .../config.xml | 20 ++++----- .../users.xml | 0 programs/server/config.xml | 6 +-- 14 files changed, 76 insertions(+), 56 deletions(-) rename dbms/tests/integration/test_http_handlers_config/{test_insert_and_select_predefine => test_insert_and_select_predefined}/config.xml (75%) rename dbms/tests/integration/test_http_handlers_config/{test_insert_and_select_predefine => test_insert_and_select_predefined}/users.xml (100%) rename dbms/tests/integration/test_http_handlers_config/{test_param_and_settings_predefine => test_param_and_settings_predefined}/config.xml (73%) rename dbms/tests/integration/test_http_handlers_config/{test_param_and_settings_predefine => test_param_and_settings_predefined}/users.xml (100%) diff --git a/dbms/programs/server/HTTPHandlerFactory.cpp b/dbms/programs/server/HTTPHandlerFactory.cpp index 9a2d1e52f09..b2c07075d54 100644 --- a/dbms/programs/server/HTTPHandlerFactory.cpp +++ b/dbms/programs/server/HTTPHandlerFactory.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; - extern const int UNKNOW_HTTP_HANDLER_TYPE; + extern const int UNKNOWN_HTTP_HANDLER_TYPE; extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; } @@ -99,13 +99,13 @@ HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPredefineQueryHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer &, const String &); HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPredefineQueryHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer &, const String &); void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) { @@ -126,14 +126,14 @@ void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfigura handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); else if (startsWith(http_handler_type_name, "dynamic_query_handler")) handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "predefine_query_handler")) - handlers_creator.push_back({createPredefineQueryHandlerMatcher(server, handler_key), createPredefineQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "predefined_query_handler")) + handlers_creator.push_back({createPredefinedQueryHandlerMatcher(server, handler_key), createPredefinedQueryHandlerCreator(server, handler_key)}); else if (startsWith(http_handler_type_name, "replicas_status_handler")) handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); else if (http_handler_type_name == "no_handler_description") no_handler_description = configuration.getString(key + ".no_handler_description"); else - throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOW_HTTP_HANDLER_TYPE); + throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOWN_HTTP_HANDLER_TYPE); } } diff --git a/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h b/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h index 0fdfd73b4f6..707e86a217d 100644 --- a/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h +++ b/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h @@ -56,7 +56,7 @@ public: /// Settings can be overridden in the query. /// Some parameters (database, default_format, everything used in the code above) do not /// belong to the Settings class. - becomeReadonlyIfNeed(request); + becomeReadonlyIfNeeded(request); changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); if (is_multipart_data || settings_may_in_post) @@ -82,7 +82,7 @@ private: /// In theory if initially readonly = 0, the client can change any setting and then set readonly /// to some other value. /// Only readonly queries are allowed for HTTP GET requests. - void becomeReadonlyIfNeed(Poco::Net::HTTPServerRequest & request) + void becomeReadonlyIfNeeded(Poco::Net::HTTPServerRequest & request) { if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) { diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp index 1bb93908638..ca22496665c 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp @@ -174,6 +174,6 @@ HTTPResponseBufferPtr HTTPQueryRequestHandler::createRespon template class HTTPQueryRequestHandler; -template class HTTPQueryRequestHandler; +template class HTTPQueryRequestHandler; } diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp index 321a585d939..81f25f0bd08 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp @@ -9,12 +9,12 @@ namespace DB namespace ErrorCodes { - extern const int EMPTY_PREDEFINE_QUERY; + extern const int EMPTY_PREDEFINED_QUERY; extern const int CANNOT_COMPILE_REGEXP; extern const int UNKNOWN_QUERY_PARAMETER; extern const int DUPLICATE_CAPTURE_QUERY_PARAM; extern const int ILLEGAL_HTTP_HANDLER_PARAM_NAME; - extern const int TOO_MANY_INSERT_QUERY_WITH_PREDEFINE_QUERY; + extern const int TOO_MANY_INSERT_QUERY_WITH_PREDEFINED_QUERY; } ExtractorDynamicQueryParameters::ExtractorDynamicQueryParameters( @@ -62,6 +62,9 @@ ExtractorDynamicQueryParameters::ExtractorDynamicQueryParameters( template void extractParamWithRegex(Context & context, const RegexRule & regex, const std::map & extract_params, const String & value) { + if (value.empty()) + return; + int num_captures = regex->NumberOfCapturingGroups() + 1; re2_st::StringPiece matches[num_captures]; @@ -71,16 +74,19 @@ void extractParamWithRegex(Context & context, const RegexRule & regex, const std { for (const auto & [capturing_name, capturing_index] : extract_params) { - String param_name = capturing_name; const auto & capturing_value = matches[capturing_index]; - if constexpr (remove_prefix_for_param) + if (capturing_value.data()) { - const static size_t prefix_size = strlen("param_"); - param_name = capturing_name.substr(prefix_size); - } + String param_name = capturing_name; + if constexpr (remove_prefix_for_param) + { + const static size_t prefix_size = strlen("param_"); + param_name = capturing_name.substr(prefix_size); + } - context.setQueryParameter(param_name, String(capturing_value.data(), capturing_value.size())); + context.setQueryParameter(param_name, String(capturing_value.data(), capturing_value.size())); + } } } } @@ -93,7 +99,7 @@ ExtractRes ExtractorDynamicQueryParameters::extract(Context & context, Poco::Net if (!extract_from_headers.empty()) for (const auto & [header_name, extract_params] : extract_from_headers) - extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name)); + extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name, "")); String extracted_query_from_params; const static size_t prefix_size = strlen("param_"); @@ -112,7 +118,7 @@ ExtractRes ExtractorDynamicQueryParameters::extract(Context & context, Poco::Net return {{extracted_query_from_params, true}}; } -ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( +ExtractorPredefinedQueryParameters::ExtractorPredefinedQueryParameters( Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) : url_regex(url_regex_), headers_regex(headers_regex_) { @@ -120,7 +126,7 @@ ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( configuration.keys(key + ".queries", queries_key); if (queries_key.empty()) - throw Exception("There must be at least one predefined query in the predefined HTTPHandler.", ErrorCodes::EMPTY_PREDEFINE_QUERY); + throw Exception("There must be at least one predefined query in the predefined HTTPHandler.", ErrorCodes::EMPTY_PREDEFINED_QUERY); for (const auto & query_key : queries_key) { @@ -136,7 +142,7 @@ ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( bool is_insert_query = extract_query_ast->as(); if (has_insert_query && is_insert_query) - throw Exception("Too many insert queries in predefined queries.", ErrorCodes::TOO_MANY_INSERT_QUERY_WITH_PREDEFINE_QUERY); + throw Exception("Too many insert queries in predefined queries.", ErrorCodes::TOO_MANY_INSERT_QUERY_WITH_PREDEFINED_QUERY); has_insert_query |= is_insert_query; predefine_queries.push_back({predefine_query, is_insert_query}); @@ -186,14 +192,14 @@ ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( } } -ExtractRes ExtractorPredefineQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) +ExtractRes ExtractorPredefinedQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) { if (!extract_from_url.empty()) extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); if (!extract_from_headers.empty()) for (const auto & [header_name, extract_params] : extract_from_headers) - extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name)); + extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name, "")); for (const auto & param : params) if (queries_names.count(param.first)) @@ -277,6 +283,9 @@ HTTPHandlerMatcher HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatche { for (const auto & [header_name, header_rule] : headers_rule) { + if (!request.has(header_name)) + return false; + const String & header_value = request.get(header_name); if (header_value.size() != findFirstMissingMatchPos(*header_rule, header_value)) return false; @@ -294,7 +303,7 @@ HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer & server, const Stri } -HTTPHandlerMatcher createPredefineQueryHandlerMatcher(IServer & server, const String & key) +HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer & server, const String & key) { return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); @@ -314,16 +323,16 @@ HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer & server, const Stri }); } -HTTPHandlerCreator createPredefineQueryHandlerCreator(IServer & server, const String & key) +HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer & server, const String & key) { return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) { - const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); + const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); return [&, query_extract = extract]() { - return new HTTPQueryRequestHandler(server, *query_extract); + return new HTTPQueryRequestHandler(server, *query_extract); }; }); } diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h index 456689d1ff5..0d7bc8356bb 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h @@ -48,10 +48,10 @@ private: std::map> extract_from_headers; }; -class ExtractorPredefineQueryParameters +class ExtractorPredefinedQueryParameters { public: - ExtractorPredefineQueryParameters( + ExtractorPredefinedQueryParameters( Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ ); diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h index 9ef8b8f7abe..deadd3910ad 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h +++ b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h @@ -7,6 +7,7 @@ namespace DB { +/// Manage the lifetime of the session context. struct HTTPSessionContextHolder { ~HTTPSessionContextHolder(); diff --git a/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml index f117eaba2f7..0c908a4f877 100644 --- a/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml +++ b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml @@ -4,14 +4,14 @@ /ping_test - + POST /test_one_handler_with_insert_and_select INSERT INTO test.test VALUES(1) SELECT COUNT() FROM test.test WHERE id = {id:UInt8} - + test not found diff --git a/dbms/tests/integration/test_http_handlers_config/test.py b/dbms/tests/integration/test_http_handlers_config/test.py index 9e332dd8675..a996fbdc908 100644 --- a/dbms/tests/integration/test_http_handlers_config/test.py +++ b/dbms/tests/integration/test_http_handlers_config/test.py @@ -35,8 +35,8 @@ def test_dynamic_query_handler_with_insert_and_select(): assert cluster.instance.http_request('query_data_from_test?max_threads=1&test_select_query_param=' + select_data_query, method='GET') == '1\n2\n3\n4\n' -def test_predefine_query_handler_with_insert_and_select(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefine_insert_and_select", "test_insert_and_select_predefine")) as cluster: +def test_predefined_query_handler_with_insert_and_select(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefined_insert_and_select", "test_insert_and_select_predefined")) as cluster: assert cluster.instance.http_request('create_test_table?max_threads=1', method='PUT') == '' assert cluster.instance.http_request('insert_data_to_test?max_threads=1', method='POST', data='(1)(2)(3)(4)') == '' assert cluster.instance.http_request('query_data_from_test?max_threads=1', method='GET') == '1\n2\n3\n4\n' @@ -54,18 +54,25 @@ def test_dynamic_query_handler_with_params_and_settings(): assert 'Syntax error' in cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + test_query + '&' + query_param, method='POST', data=settings) assert cluster.instance.http_request('get_query_params_and_settings?get_query_param=' + quoted_test_query + '&' + query_param + '&' + settings) == '1\n2\n' + assert cluster.instance.http_request('query_param_with_url/123/max_threads?query_param=' + quoted_test_query + '&' + settings + '¶m_name_2=max_alter_threads') == '1\n2\n' assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?query_param=' + quoted_test_query + '&' + settings) == '1\n2\n' + assert '`name_2` is not set' in cluster.instance.http_request('query_param_with_url/123/max_threads?query_param=' + quoted_test_query + '&' + settings) assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?query_param=' + quoted_test_query + '&' + query_param + '&' + settings) assert cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\n2\n' + assert cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings + '¶m_name_2=max_alter_threads', headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) == '1\n2\n' + assert '`name_2` is not set' in cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) + assert 'There is no handle /test_match_headers' in cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings) -def test_predefine_query_handler_with_params_and_settings(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefine_params_and_settings", "test_param_and_settings_predefine")) as cluster: +def test_predefined_query_handler_with_params_and_settings(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefined_params_and_settings", "test_param_and_settings_predefined")) as cluster: settings = 'max_threads=1&max_alter_threads=2' query_param = 'name_1=max_threads&name_2=max_alter_threads' assert cluster.instance.http_request('get_query_params_and_settings?' + query_param + '&' + settings, method='GET') == '1\nmax_alter_threads\t2\n' assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?' + settings) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('query_param_with_url/123/max_threads?' + settings + '&name_2=max_alter_threads') == '1\nmax_alter_threads\t2\n' + assert '`name_2` is not set' in cluster.instance.http_request('query_param_with_url/123/max_threads?' + settings) assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?' + query_param + '&' + settings) assert cluster.instance.http_request('post_query_params_and_settings?' + query_param, method='POST', data=settings) == '1\nmax_alter_threads\t2\n' @@ -73,6 +80,9 @@ def test_predefine_query_handler_with_params_and_settings(): assert cluster.instance.http_request('post_query_params_and_settings?' + query_param + '&' + settings, method='POST') == '1\nmax_alter_threads\t2\n' assert cluster.instance.http_request('post_query_params_and_settings', method='POST', data=query_param + '&' + settings) == '1\nmax_alter_threads\t2\n' assert cluster.instance.http_request('test_match_headers?' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('test_match_headers?' + settings + '&name_2=max_alter_threads', headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) == '1\nmax_alter_threads\t2\n' + assert '`name_2` is not set' in cluster.instance.http_request('test_match_headers?' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) + assert 'There is no handle /test_match_headers' in cluster.instance.http_request('test_match_headers?' + settings) def test_other_configs(): diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml similarity index 75% rename from dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml rename to dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml index 0f0ab81d139..6319bbf327f 100644 --- a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml @@ -2,25 +2,25 @@ - + PUT /create_test_table CREATE DATABASE test CREATE TABLE test.test (id UInt8) Engine = Memory - + - + POST /insert_data_to_test INSERT INTO test.test VALUES - + - + GET /query_data_from_test SELECT * FROM test.test ORDER BY id - + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml rename to dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml index 06c7143f3e8..f9c576e5546 100644 --- a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml @@ -16,14 +16,14 @@ GET - .+)/(?P.+)]]> + [^/]+)(/(?P[^/]+))?]]> query_param TEST_HEADER_VALUE - .+)/(?P.+)]]> + [^/]+)(/(?P[^/]+))?]]> query_param diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml similarity index 73% rename from dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml rename to dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml index cca4bc04a69..1ad1fecb2e5 100644 --- a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml @@ -2,42 +2,42 @@ - + POST /post_query_params_and_settings SELECT value FROM system.settings WHERE name = {name_1:String} SELECT name, value FROM system.settings WHERE name = {name_2:String} - + - + GET /get_query_params_and_settings SELECT value FROM system.settings WHERE name = {name_1:String} SELECT name, value FROM system.settings WHERE name = {name_2:String} - + - + GET - .+)/(?P.+)]]> + [^/]+)(/(?P[^/]+))?]]> SELECT value FROM system.settings WHERE name = {name_1:String} SELECT name, value FROM system.settings WHERE name = {name_2:String} - + - + TEST_HEADER_VALUE - .+)/(?P.+)]]> + [^/]+)(/(?P[^/]+))?]]> SELECT value FROM system.settings WHERE name = {name_1:String} SELECT name, value FROM system.settings WHERE name = {name_2:String} - + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml rename to dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml diff --git a/programs/server/config.xml b/programs/server/config.xml index c4ba4e66ab3..74557a6ab7d 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -537,8 +537,8 @@ url - RE2 compatible regular expression (optional) method - HTTP method(optional) headers - HTTP Header(optional) - queries - predefine queries (mandatory) - + queries - predefined queries (mandatory) + /test_simple_predefine GET default @@ -546,7 +546,7 @@ SELECT 1, {query_prepared_param_1:String} SELECT 1, {query_prepared_param_2:String} - + --> From 1eda48b4b136a8f12520377f3948a8964a284c0d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 9 Dec 2019 12:47:26 +0800 Subject: [PATCH 409/743] fix bad git rebase --- .../server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp index ca22496665c..77e72972551 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp @@ -34,7 +34,6 @@ #include #include #include -#include #include #include #include From 835dc4c4452ca3ec2a3dc39947fa6df13d0f21ad Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 3 Apr 2020 17:09:19 +0800 Subject: [PATCH 410/743] After merge upsream master fix --- .../DataStreams/HTTPInputStreams.cpp | 22 +-- dbms/{src => }/DataStreams/HTTPInputStreams.h | 6 +- .../DataStreams/HTTPOutputStreams.cpp | 0 .../{src => }/DataStreams/HTTPOutputStreams.h | 0 .../Interpreters/QueryParameterVisitor.h | 0 dbms/programs/server/HTTPHandlerFactory.cpp | 140 ------------------ dbms/programs/server/HTTPHandlerFactory.h | 51 ------- programs/server/HTTPHandler.cpp | 7 - programs/server/HTTPHandlerFactory.cpp | 135 ++++++++++++++--- programs/server/HTTPHandlerFactory.h | 136 ++++------------- .../HTTPRequestHandler/ExtractorClientInfo.h | 0 .../ExtractorContextChange.h | 5 - .../HTTPExceptionHandler.cpp | 0 .../HTTPRequestHandler/HTTPExceptionHandler.h | 0 .../HTTPPingRequestHandler.cpp | 0 .../HTTPPingRequestHandler.h | 0 .../HTTPQueryRequestHandler.cpp | 11 +- .../HTTPQueryRequestHandler.h | 0 ...TPQueryRequestHandlerMatcherAndCreator.cpp | 0 ...HTTPQueryRequestHandlerMatcherAndCreator.h | 0 .../HTTPReplicasStatusRequestHandler.cpp | 0 .../HTTPReplicasStatusRequestHandler.h | 0 .../HTTPRootRequestHandler.cpp | 0 .../HTTPRootRequestHandler.h | 0 .../HTTPSessionContextHolder.cpp | 0 .../HTTPSessionContextHolder.h | 0 src/IO/WriteBufferFromHTTPServerResponse.cpp | 64 +++----- src/IO/WriteBufferFromHTTPServerResponse.h | 3 +- src/Interpreters/Context.h | 1 + .../ReplaceQueryParameterVisitor.cpp | 2 +- 30 files changed, 188 insertions(+), 395 deletions(-) rename dbms/{src => }/DataStreams/HTTPInputStreams.cpp (74%) rename dbms/{src => }/DataStreams/HTTPInputStreams.h (62%) rename dbms/{src => }/DataStreams/HTTPOutputStreams.cpp (100%) rename dbms/{src => }/DataStreams/HTTPOutputStreams.h (100%) rename dbms/{src => }/Interpreters/QueryParameterVisitor.h (100%) delete mode 100644 dbms/programs/server/HTTPHandlerFactory.cpp delete mode 100644 dbms/programs/server/HTTPHandlerFactory.h rename {dbms/programs => programs}/server/HTTPRequestHandler/ExtractorClientInfo.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/ExtractorContextChange.h (98%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPExceptionHandler.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPExceptionHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPPingRequestHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp (94%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPQueryRequestHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPRootRequestHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPSessionContextHolder.h (100%) diff --git a/dbms/src/DataStreams/HTTPInputStreams.cpp b/dbms/DataStreams/HTTPInputStreams.cpp similarity index 74% rename from dbms/src/DataStreams/HTTPInputStreams.cpp rename to dbms/DataStreams/HTTPInputStreams.cpp index 3e5d6c01ab3..e760069a040 100644 --- a/dbms/src/DataStreams/HTTPInputStreams.cpp +++ b/dbms/DataStreams/HTTPInputStreams.cpp @@ -18,9 +18,9 @@ namespace ErrorCodes } HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from) - : in(createRawInBuffer(request)) - , in_maybe_compressed(createCompressedBuffer(request, in)) - , in_maybe_internal_compressed(createInternalCompressedBuffer(from, in_maybe_compressed)) + : in(plainBuffer(request)) + , in_maybe_compressed(compressedBuffer(request, in)) + , in_maybe_internal_compressed(internalCompressedBuffer(from, in_maybe_compressed)) { /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, /// checksums of client data compressed with internal algorithm are not checked. @@ -31,12 +31,12 @@ HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & reques } } -std::unique_ptr HTTPInputStreams::createRawInBuffer(HTTPServerRequest & request) const +std::unique_ptr HTTPInputStreams::plainBuffer(HTTPServerRequest & request) const { return std::make_unique(request.stream()); } -std::unique_ptr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & request, std::unique_ptr & raw_buffer) const +std::unique_ptr HTTPInputStreams::compressedBuffer(HTTPServerRequest & request, std::unique_ptr & plain_buffer) const { /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_compressed_method = request.get("Content-Encoding", ""); @@ -44,22 +44,22 @@ std::unique_ptr HTTPInputStreams::createCompressedBuffer(HTTPServerR if (!http_compressed_method.empty()) { if (http_compressed_method == "gzip") - return std::make_unique(std::move(raw_buffer), CompressionMethod::Gzip); + return std::make_unique(std::move(plain_buffer), CompressionMethod::Gzip); else if (http_compressed_method == "deflate") - return std::make_unique(std::move(raw_buffer), CompressionMethod::Zlib); + return std::make_unique(std::move(plain_buffer), CompressionMethod::Zlib); #if USE_BROTLI else if (http_compressed_method == "br") - return std::make_unique(std::move(raw_buffer)); + return std::make_unique(std::move(plain_buffer)); #endif else throw Exception("Unknown Content-Encoding of HTTP request: " + http_compressed_method, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); } - return std::move(raw_buffer); + return std::move(plain_buffer); } -std::unique_ptr HTTPInputStreams::createInternalCompressedBuffer( - HTMLForm & params, std::unique_ptr & http_maybe_encoding_buffer) const +std::unique_ptr HTTPInputStreams::internalCompressedBuffer( + HTMLForm ¶ms, std::unique_ptr &http_maybe_encoding_buffer) const { /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. diff --git a/dbms/src/DataStreams/HTTPInputStreams.h b/dbms/DataStreams/HTTPInputStreams.h similarity index 62% rename from dbms/src/DataStreams/HTTPInputStreams.h rename to dbms/DataStreams/HTTPInputStreams.h index 652f5370a5a..f2325da676f 100644 --- a/dbms/src/DataStreams/HTTPInputStreams.h +++ b/dbms/DataStreams/HTTPInputStreams.h @@ -21,9 +21,9 @@ struct HTTPInputStreams HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); - ReadBufferUniquePtr createRawInBuffer(HTTPServerRequest & request) const; - ReadBufferUniquePtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferUniquePtr & raw_buffer) const; - ReadBufferUniquePtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferUniquePtr & http_maybe_encoding_buffer) const; + ReadBufferUniquePtr plainBuffer(HTTPServerRequest & request) const; + ReadBufferUniquePtr compressedBuffer(HTTPServerRequest & request, ReadBufferUniquePtr & plain_buffer) const; + ReadBufferUniquePtr internalCompressedBuffer(HTMLForm & params, ReadBufferUniquePtr & http_maybe_encoding_buffer) const; }; } diff --git a/dbms/src/DataStreams/HTTPOutputStreams.cpp b/dbms/DataStreams/HTTPOutputStreams.cpp similarity index 100% rename from dbms/src/DataStreams/HTTPOutputStreams.cpp rename to dbms/DataStreams/HTTPOutputStreams.cpp diff --git a/dbms/src/DataStreams/HTTPOutputStreams.h b/dbms/DataStreams/HTTPOutputStreams.h similarity index 100% rename from dbms/src/DataStreams/HTTPOutputStreams.h rename to dbms/DataStreams/HTTPOutputStreams.h diff --git a/dbms/src/Interpreters/QueryParameterVisitor.h b/dbms/Interpreters/QueryParameterVisitor.h similarity index 100% rename from dbms/src/Interpreters/QueryParameterVisitor.h rename to dbms/Interpreters/QueryParameterVisitor.h diff --git a/dbms/programs/server/HTTPHandlerFactory.cpp b/dbms/programs/server/HTTPHandlerFactory.cpp deleted file mode 100644 index b2c07075d54..00000000000 --- a/dbms/programs/server/HTTPHandlerFactory.cpp +++ /dev/null @@ -1,140 +0,0 @@ -#include "HTTPHandlerFactory.h" - -#include "NotFoundHandler.h" -#include "HTTPRequestHandler/HTTPRootRequestHandler.h" -#include "HTTPRequestHandler/HTTPPingRequestHandler.h" -#include "HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h" - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SYNTAX_ERROR; - extern const int UNKNOWN_HTTP_HANDLER_TYPE; - extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; -} - -InterserverIOHTTPHandlerFactory::InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_) - : server(server_), log(&Logger::get(name_)), name(name_) -{ -} - -Poco::Net::HTTPRequestHandler * InterserverIOHTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) -{ - LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " << request.getMethod() - << ", Address: " << request.clientAddress().toString() - << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); - - const auto & uri = request.getURI(); - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) - { - if (uri == "/") - return new HTTPRootRequestHandler(server); - if (uri == "/ping") - return new HTTPPingRequestHandler(server); - else if (startsWith(uri, "/replicas_status")) - return new HTTPReplicasStatusRequestHandler(server.context()); - } - - if (uri.find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - { - return new InterserverIOHTTPHandler(server); - } - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - { - return new NotFoundHandler( - "Use / or /ping for health checks.\n" - "Or /replicas_status for more sophisticated health checks.\n" - "Send queries from your program with POST method or GET /?query=...\n\n" - " Use clickhouse-client:\n\n" - " For interactive data analysis:\n" - " clickhouse-client\n\n" - " For batch query processing:\n" - " clickhouse-client --query='SELECT 1' > result\n" - " clickhouse-client < query > result" - ); - } - - return nullptr; -} - -HTTPHandlerFactory::HTTPHandlerFactory(IServer & server_, const std::string & name_) - : server(server_), log(&Logger::get(name_)), name(name_) -{ - updateHTTPHandlersCreator(server.config()); - - if (handlers_creator.empty()) - throw Exception("The HTTPHandlers does not exist in the config.xml", ErrorCodes::EMPTY_HTTP_HANDLER_IN_CONFIG); -} - -Poco::Net::HTTPRequestHandler * HTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) -{ - LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " << request.getMethod() - << ", Address: " << request.clientAddress().toString() - << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); - - for (const auto & [matcher, creator] : handlers_creator) - { - if (matcher(request)) - return creator(); - } - - return new NotFoundHandler(no_handler_description); -} - -HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer &, const String &); - -HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer &, const String &); - -void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) -{ - Poco::Util::AbstractConfiguration::Keys http_handlers_item_key; - configuration.keys(key, http_handlers_item_key); - - handlers_creator.reserve(http_handlers_item_key.size()); - for (const auto & http_handler_type_name : http_handlers_item_key) - { - if (http_handler_type_name.find('.') != String::npos) - throw Exception("HTTPHandler type name with dots are not supported: '" + http_handler_type_name + "'", ErrorCodes::SYNTAX_ERROR); - - const auto & handler_key = key + "." + http_handler_type_name; - - if (startsWith(http_handler_type_name, "root_handler")) - handlers_creator.push_back({createRootHandlerMatcher(server, handler_key), createRootHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "ping_handler")) - handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "dynamic_query_handler")) - handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "predefined_query_handler")) - handlers_creator.push_back({createPredefinedQueryHandlerMatcher(server, handler_key), createPredefinedQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "replicas_status_handler")) - handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); - else if (http_handler_type_name == "no_handler_description") - no_handler_description = configuration.getString(key + ".no_handler_description"); - else - throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOWN_HTTP_HANDLER_TYPE); - } -} - -} diff --git a/dbms/programs/server/HTTPHandlerFactory.h b/dbms/programs/server/HTTPHandlerFactory.h deleted file mode 100644 index 5add7b619af..00000000000 --- a/dbms/programs/server/HTTPHandlerFactory.h +++ /dev/null @@ -1,51 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include "IServer.h" -#include "InterserverIOHTTPHandler.h" - - -namespace DB -{ - -class InterserverIOHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -public: - InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - -private: - IServer & server; - Logger * log; - std::string name; -}; - -using HTTPHandlerCreator = std::function; -using HTTPHandlerMatcher = std::function; -using HTTPHandlerMatcherAndCreator = std::pair; -using HTTPHandlersMatcherAndCreator = std::vector; - -class HTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -public: - HTTPHandlerFactory(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - - void updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key = "http_handlers"); - -private: - IServer & server; - Logger * log; - std::string name; - - String no_handler_description; - HTTPHandlersMatcherAndCreator handlers_creator; -}; - -} diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index cfbefb94ee4..593a156ca52 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -328,15 +328,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne internal_compression = params.getParsed("compress", false); /// Workaround. Poco does not detect 411 Length Required case. -<<<<<<< HEAD:programs/server/HTTPHandler.cpp if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); -======= - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST - && !request.getChunkedTransferEncoding() - && !request.hasContentLength()) - throw Exception("There is neither Transfer-Encoding header nor Content-Length header", ErrorCodes::HTTP_LENGTH_REQUIRED); ->>>>>>> ISSUES-5436 fix build failure & fix test failure:dbms/programs/server/HTTPHandler.cpp { Context query_context = server.context(); diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index 7a1bcaf4fc6..b2c07075d54 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -1,43 +1,140 @@ #include "HTTPHandlerFactory.h" +#include "NotFoundHandler.h" +#include "HTTPRequestHandler/HTTPRootRequestHandler.h" +#include "HTTPRequestHandler/HTTPPingRequestHandler.h" +#include "HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h" + namespace DB { -HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_) +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int UNKNOWN_HTTP_HANDLER_TYPE; + extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; +} + +InterserverIOHTTPHandlerFactory::InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_) : server(server_), log(&Logger::get(name_)), name(name_) { } -Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHandler( - const Poco::Net::HTTPServerRequest & request) // override +Poco::Net::HTTPRequestHandler * InterserverIOHTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) { LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " - << request.getMethod() - << ", Address: " - << request.clientAddress().toString() - << ", User-Agent: " - << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); - for (auto & handler_factory : child_handler_factories) + const auto & uri = request.getURI(); + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) { - auto handler = handler_factory->createRequestHandler(request); - if (handler != nullptr) - return handler; + if (uri == "/") + return new HTTPRootRequestHandler(server); + if (uri == "/ping") + return new HTTPPingRequestHandler(server); + else if (startsWith(uri, "/replicas_status")) + return new HTTPReplicasStatusRequestHandler(server.context()); } - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD + if (uri.find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + { + return new InterserverIOHTTPHandler(server); + } + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { - return new NotFoundHandler; + return new NotFoundHandler( + "Use / or /ping for health checks.\n" + "Or /replicas_status for more sophisticated health checks.\n" + "Send queries from your program with POST method or GET /?query=...\n\n" + " Use clickhouse-client:\n\n" + " For interactive data analysis:\n" + " clickhouse-client\n\n" + " For batch query processing:\n" + " clickhouse-client --query='SELECT 1' > result\n" + " clickhouse-client < query > result" + ); } return nullptr; } +HTTPHandlerFactory::HTTPHandlerFactory(IServer & server_, const std::string & name_) + : server(server_), log(&Logger::get(name_)), name(name_) +{ + updateHTTPHandlersCreator(server.config()); + + if (handlers_creator.empty()) + throw Exception("The HTTPHandlers does not exist in the config.xml", ErrorCodes::EMPTY_HTTP_HANDLER_IN_CONFIG); +} + +Poco::Net::HTTPRequestHandler * HTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +{ + LOG_TRACE(log, "HTTP Request for " << name << ". " + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); + + for (const auto & [matcher, creator] : handlers_creator) + { + if (matcher(request)) + return creator(); + } + + return new NotFoundHandler(no_handler_description); +} + +HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer &, const String &); + +HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer &, const String &); + +void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) +{ + Poco::Util::AbstractConfiguration::Keys http_handlers_item_key; + configuration.keys(key, http_handlers_item_key); + + handlers_creator.reserve(http_handlers_item_key.size()); + for (const auto & http_handler_type_name : http_handlers_item_key) + { + if (http_handler_type_name.find('.') != String::npos) + throw Exception("HTTPHandler type name with dots are not supported: '" + http_handler_type_name + "'", ErrorCodes::SYNTAX_ERROR); + + const auto & handler_key = key + "." + http_handler_type_name; + + if (startsWith(http_handler_type_name, "root_handler")) + handlers_creator.push_back({createRootHandlerMatcher(server, handler_key), createRootHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "ping_handler")) + handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "dynamic_query_handler")) + handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "predefined_query_handler")) + handlers_creator.push_back({createPredefinedQueryHandlerMatcher(server, handler_key), createPredefinedQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "replicas_status_handler")) + handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); + else if (http_handler_type_name == "no_handler_description") + no_handler_description = configuration.getString(key + ".no_handler_description"); + else + throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOWN_HTTP_HANDLER_TYPE); + } +} + } diff --git a/programs/server/HTTPHandlerFactory.h b/programs/server/HTTPHandlerFactory.h index fcd7fb5d4a2..5add7b619af 100644 --- a/programs/server/HTTPHandlerFactory.h +++ b/programs/server/HTTPHandlerFactory.h @@ -1,127 +1,51 @@ #pragma once -#include #include #include +#include +#include #include #include "IServer.h" -#include "HTTPHandler.h" #include "InterserverIOHTTPHandler.h" -#include "NotFoundHandler.h" -#include "PingRequestHandler.h" -#include "PrometheusRequestHandler.h" -#include "ReplicasStatusHandler.h" -#include "RootRequestHandler.h" namespace DB { -/// Handle request using child handlers -class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactory +class InterserverIOHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory { -private: - using TThis = HTTPRequestHandlerFactoryMain; +public: + InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_); + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; + +private: + IServer & server; + Logger * log; + std::string name; +}; + +using HTTPHandlerCreator = std::function; +using HTTPHandlerMatcher = std::function; +using HTTPHandlerMatcherAndCreator = std::pair; +using HTTPHandlersMatcherAndCreator = std::vector; + +class HTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +public: + HTTPHandlerFactory(IServer & server_, const std::string & name_); + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; + + void updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key = "http_handlers"); + +private: IServer & server; Logger * log; std::string name; - std::vector> child_handler_factories; - -public: - HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - - template - TThis * addHandler(TArgs &&... args) - { - child_handler_factories.emplace_back(std::make_unique(server, std::forward(args)...)); - return this; - } + String no_handler_description; + HTTPHandlersMatcherAndCreator handlers_creator; }; - -/// Handle POST or GET with params -template -class HTTPQueryRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -private: - IServer & server; - -public: - HTTPQueryRequestHandlerFactory(IServer & server_) : server(server_) {} - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override - { - if (request.getURI().find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - return new HandleType(server); - return nullptr; - } -}; - - -/// Handle GET or HEAD endpoint on specified path -template -class HTTPGetRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -private: - IServer & server; -public: - HTTPGetRequestHandlerFactory(IServer & server_) : server(server_) {} - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override - { - auto & method = request.getMethod(); - if (!(method == Poco::Net::HTTPRequest::HTTP_GET || method == Poco::Net::HTTPRequest::HTTP_HEAD)) - return nullptr; - - auto & uri = request.getURI(); - bool uri_match = TGetEndpoint::strict_path ? uri == TGetEndpoint::path : startsWith(uri, TGetEndpoint::path); - if (uri_match) - return new typename TGetEndpoint::HandleType(server); - - return nullptr; - } -}; - - -struct RootEndpoint -{ - static constexpr auto path = "/"; - static constexpr auto strict_path = true; - using HandleType = RootRequestHandler; -}; - -struct PingEndpoint -{ - static constexpr auto path = "/ping"; - static constexpr auto strict_path = true; - using HandleType = PingRequestHandler; -}; - -struct ReplicasStatusEndpoint -{ - static constexpr auto path = "/replicas_status"; - static constexpr auto strict_path = false; - using HandleType = ReplicasStatusHandler; -}; - -using HTTPRootRequestHandlerFactory = HTTPGetRequestHandlerFactory; -using HTTPPingRequestHandlerFactory = HTTPGetRequestHandlerFactory; -using HTTPReplicasStatusRequestHandlerFactory = HTTPGetRequestHandlerFactory; - -template -HTTPRequestHandlerFactoryMain * createDefaultHandlerFatory(IServer & server, const std::string & name) -{ - auto handlerFactory = new HTTPRequestHandlerFactoryMain(server, name); - handlerFactory->addHandler() - ->addHandler() - ->addHandler() - ->addHandler>(); - return handlerFactory; -} - - } diff --git a/dbms/programs/server/HTTPRequestHandler/ExtractorClientInfo.h b/programs/server/HTTPRequestHandler/ExtractorClientInfo.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/ExtractorClientInfo.h rename to programs/server/HTTPRequestHandler/ExtractorClientInfo.h diff --git a/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h b/programs/server/HTTPRequestHandler/ExtractorContextChange.h similarity index 98% rename from dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h rename to programs/server/HTTPRequestHandler/ExtractorContextChange.h index 707e86a217d..1d4b4e5f58a 100644 --- a/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h +++ b/programs/server/HTTPRequestHandler/ExtractorContextChange.h @@ -9,11 +9,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int UNKNOWN_HTTP_PARAM; -} - class ExtractorContextChange { public: diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp b/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h b/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h rename to programs/server/HTTPRequestHandler/HTTPExceptionHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h rename to programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp similarity index 94% rename from dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp index 77e72972551..723ff85abe3 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp +++ b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp @@ -156,19 +156,16 @@ HTTPResponseBufferPtr HTTPQueryRequestHandler::createRespon /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. /// NOTE parsing of the list of methods is slightly incorrect. if (std::string::npos != http_response_compression_methods.find("gzip")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, true, CompressionMethod::Gzip); else if (std::string::npos != http_response_compression_methods.find("deflate")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, true, CompressionMethod::Zlib); #if USE_BROTLI else if (http_response_compression_methods == "br") - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, true, CompressionMethod::Brotli); #endif } - return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, false, CompressionMethod{}); } diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h rename to programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp rename to programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h rename to programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h rename to programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h rename to programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp b/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp rename to programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h b/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h rename to programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index c61572d7097..0f30f1352e3 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -107,57 +107,36 @@ void WriteBufferFromHTTPServerResponse::nextImpl() { if (compress) { - if (compression_method == CompressionMethod::Gzip) - { + auto content_encoding_name = toContentEncodingName(compression_method); + #if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: gzip\r\n"; -#else - response.set("Content-Encoding", "gzip"); - response_body_ostr = &(response.send()); -#endif - out_raw = std::make_unique(*response_body_ostr); - deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); - out = &*deflating_buf; - } - else if (compression_method == CompressionMethod::Zlib) - { -#if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: deflate\r\n"; -#else - response.set("Content-Encoding", "deflate"); - response_body_ostr = &(response.send()); -#endif - out_raw = std::make_unique(*response_body_ostr); - deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); - out = &*deflating_buf; - } -#if USE_BROTLI - else if (compression_method == CompressionMethod::Brotli) - { -#if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: br\r\n"; + *response_header_ostr << "Content-Encoding: " << content_encoding_name << "\r\n"; #else response.set("Content-Encoding", content_encoding_name); #endif - - else - throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse", - ErrorCodes::LOGICAL_ERROR); - /// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy. } - else - { + #if !defined(POCO_CLICKHOUSE_PATCH) - response_body_ostr = &(response.send()); + response_body_ostr = &(response.send()); #endif - out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); - out = &*out_raw; - } + /// We reuse our buffer in "out" to avoid extra allocations and copies. + + if (compress) + out = wrapWriteBufferWithCompressionMethod( + std::make_unique(*response_body_ostr), + compress ? compression_method : CompressionMethod::None, + compression_level, + working_buffer.size(), + working_buffer.begin()); + else + out = std::make_unique( + *response_body_ostr, + working_buffer.size(), + working_buffer.begin()); } finishSendHeaders(); - } if (out) @@ -174,9 +153,8 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerResponse & response_, unsigned keep_alive_timeout_, bool compress_, - CompressionMethod compression_method_, - size_t size) - : BufferWithOwnMemory(size) + CompressionMethod compression_method_) + : BufferWithOwnMemory(DBMS_DEFAULT_BUFFER_SIZE) , request(request_) , response(response_) , keep_alive_timeout(keep_alive_timeout_) diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index 528ba8ffe23..c10288fff9f 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -95,8 +95,7 @@ public: Poco::Net::HTTPServerResponse & response_, unsigned keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. - CompressionMethod compression_method_ = CompressionMethod::Gzip, - size_t size = DBMS_DEFAULT_BUFFER_SIZE); + CompressionMethod compression_method_ = CompressionMethod::None); /// Writes progess in repeating HTTP headers. void onProgress(const Progress & progress); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c9213c806ff..1f81cdbc58b 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -106,6 +106,7 @@ class Volume; using VolumePtr = std::shared_ptr; struct NamedSession; + #if USE_EMBEDDED_COMPILER class CompiledExpressionCache; #endif diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index e55c0b05a2d..5c29c722f88 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -55,7 +55,7 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) IColumn & temp_column = *temp_column_ptr; ReadBufferFromString read_buffer{value}; FormatSettings format_settings; - data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings); + data_type->deserializeAsTextEscaped(temp_column, read_buffer, format_settings); if (!read_buffer.eof()) throw Exception("Value " + value + " cannot be parsed as " + type_name + " for query parameter '" + ast_param.name + "'" From 5262512f0c4c3a825b455ebbec5d85081af4853f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Apr 2020 22:02:36 +0300 Subject: [PATCH 411/743] Flush output for interactive applications --- src/Interpreters/executeQuery.cpp | 1 + src/Processors/Formats/IOutputFormat.cpp | 3 +++ src/Processors/Formats/IOutputFormat.h | 5 +++++ 3 files changed, 9 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 63c64a84308..c37b41c11ae 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -721,6 +721,7 @@ void executeQuery( }); auto out = context.getOutputFormatProcessor(format_name, *out_buf, pipeline.getHeader()); + out->setAutoFlush(); /// Save previous progress callback if any. TODO Do it more conveniently. auto previous_progress_callback = context.getProgressCallback(); diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index ff4ac393471..543a854f75e 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -69,6 +69,9 @@ void IOutputFormat::work() break; } + if (auto_flush) + flush(); + has_input = false; } diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 1137dd78446..71a0d2f0066 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -34,6 +34,9 @@ protected: bool finished = false; bool finalized = false; + /// Flush data on each consumed chunk. This is intented for interactive applications to output data as soon as it's ready. + bool auto_flush = false; + RowsBeforeLimitCounterPtr rows_before_limit_counter; virtual void consume(Chunk) = 0; @@ -50,6 +53,8 @@ public: /// Flush output buffers if any. virtual void flush(); + void setAutoFlush() { auto_flush = true; } + /// Value for rows_before_limit_at_least field. virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {} From 57cbecf93511e36e2fef1bfa6612e05e0445872c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 4 Apr 2020 16:57:16 +0800 Subject: [PATCH 412/743] ISSUES-5436 reworking predefine http --- dbms/DataStreams/HTTPInputStreams.cpp | 73 --- dbms/DataStreams/HTTPInputStreams.h | 29 - dbms/DataStreams/HTTPOutputStreams.cpp | 213 ------ dbms/DataStreams/HTTPOutputStreams.h | 48 -- programs/server/CMakeLists.txt | 11 +- programs/server/HTTPHandler.cpp | 612 +++++++++++++++--- programs/server/HTTPHandler.h | 85 ++- programs/server/HTTPHandlerFactory.cpp | 200 +++--- programs/server/HTTPHandlerFactory.h | 126 +++- programs/server/HTTPHandlerRequestFilter.h | 98 +++ .../HTTPRequestHandler/ExtractorClientInfo.h | 38 -- .../ExtractorContextChange.h | 116 ---- .../HTTPExceptionHandler.cpp | 154 ----- .../HTTPRequestHandler/HTTPExceptionHandler.h | 19 - .../HTTPPingRequestHandler.cpp | 45 -- .../HTTPPingRequestHandler.h | 26 - .../HTTPQueryRequestHandler.cpp | 175 ----- .../HTTPQueryRequestHandler.h | 48 -- ...TPQueryRequestHandlerMatcherAndCreator.cpp | 339 ---------- ...HTTPQueryRequestHandlerMatcherAndCreator.h | 92 --- .../HTTPReplicasStatusRequestHandler.cpp | 118 ---- .../HTTPReplicasStatusRequestHandler.h | 25 - .../HTTPRootRequestHandler.cpp | 45 -- .../HTTPRootRequestHandler.h | 26 - .../HTTPSessionContextHolder.cpp | 105 --- .../HTTPSessionContextHolder.h | 25 - programs/server/NotFoundHandler.cpp | 20 +- programs/server/NotFoundHandler.h | 11 +- programs/server/PingRequestHandler.cpp | 31 - programs/server/PingRequestHandler.h | 27 - programs/server/ReplicasStatusHandler.h | 2 +- programs/server/RootRequestHandler.cpp | 33 - programs/server/RootRequestHandler.h | 27 - programs/server/Server.cpp | 44 +- programs/server/StaticRequestHandler.cpp | 86 +++ programs/server/StaticRequestHandler.h | 28 + src/Common/HTMLForm.h | 6 - .../Interpreters/QueryParameterVisitor.h | 14 + 38 files changed, 1043 insertions(+), 2177 deletions(-) delete mode 100644 dbms/DataStreams/HTTPInputStreams.cpp delete mode 100644 dbms/DataStreams/HTTPInputStreams.h delete mode 100644 dbms/DataStreams/HTTPOutputStreams.cpp delete mode 100644 dbms/DataStreams/HTTPOutputStreams.h create mode 100644 programs/server/HTTPHandlerRequestFilter.h delete mode 100644 programs/server/HTTPRequestHandler/ExtractorClientInfo.h delete mode 100644 programs/server/HTTPRequestHandler/ExtractorContextChange.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPExceptionHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h delete mode 100644 programs/server/PingRequestHandler.cpp delete mode 100644 programs/server/PingRequestHandler.h delete mode 100644 programs/server/RootRequestHandler.cpp delete mode 100644 programs/server/RootRequestHandler.h create mode 100644 programs/server/StaticRequestHandler.cpp create mode 100644 programs/server/StaticRequestHandler.h rename {dbms => src}/Interpreters/QueryParameterVisitor.h (59%) diff --git a/dbms/DataStreams/HTTPInputStreams.cpp b/dbms/DataStreams/HTTPInputStreams.cpp deleted file mode 100644 index e760069a040..00000000000 --- a/dbms/DataStreams/HTTPInputStreams.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include -#include - -#include - -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_COMPRESSION_METHOD; -} - -HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from) - : in(plainBuffer(request)) - , in_maybe_compressed(compressedBuffer(request, in)) - , in_maybe_internal_compressed(internalCompressedBuffer(from, in_maybe_compressed)) -{ - /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, - /// checksums of client data compressed with internal algorithm are not checked. - if (context.getSettingsRef().http_native_compression_disable_checksumming_on_decompress) - { - if (CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) - compressed_buffer->disableChecksumming(); - } -} - -std::unique_ptr HTTPInputStreams::plainBuffer(HTTPServerRequest & request) const -{ - return std::make_unique(request.stream()); -} - -std::unique_ptr HTTPInputStreams::compressedBuffer(HTTPServerRequest & request, std::unique_ptr & plain_buffer) const -{ - /// Request body can be compressed using algorithm specified in the Content-Encoding header. - String http_compressed_method = request.get("Content-Encoding", ""); - - if (!http_compressed_method.empty()) - { - if (http_compressed_method == "gzip") - return std::make_unique(std::move(plain_buffer), CompressionMethod::Gzip); - else if (http_compressed_method == "deflate") - return std::make_unique(std::move(plain_buffer), CompressionMethod::Zlib); -#if USE_BROTLI - else if (http_compressed_method == "br") - return std::make_unique(std::move(plain_buffer)); -#endif - else - throw Exception("Unknown Content-Encoding of HTTP request: " + http_compressed_method, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); - } - - return std::move(plain_buffer); -} - -std::unique_ptr HTTPInputStreams::internalCompressedBuffer( - HTMLForm ¶ms, std::unique_ptr &http_maybe_encoding_buffer) const -{ - /// The data can also be compressed using incompatible internal algorithm. This is indicated by - /// 'decompress' query parameter. - std::unique_ptr in_post_maybe_compressed; - if (params.getParsed("decompress", false)) - return std::make_unique(*http_maybe_encoding_buffer); - - return std::move(http_maybe_encoding_buffer); -} - -} diff --git a/dbms/DataStreams/HTTPInputStreams.h b/dbms/DataStreams/HTTPInputStreams.h deleted file mode 100644 index f2325da676f..00000000000 --- a/dbms/DataStreams/HTTPInputStreams.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -using HTTPServerRequest = Poco::Net::HTTPServerRequest; - -struct HTTPInputStreams -{ - using ReadBufferUniquePtr = std::unique_ptr; - - ReadBufferUniquePtr in; - ReadBufferUniquePtr in_maybe_compressed; - ReadBufferUniquePtr in_maybe_internal_compressed; - - HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); - - ReadBufferUniquePtr plainBuffer(HTTPServerRequest & request) const; - ReadBufferUniquePtr compressedBuffer(HTTPServerRequest & request, ReadBufferUniquePtr & plain_buffer) const; - ReadBufferUniquePtr internalCompressedBuffer(HTMLForm & params, ReadBufferUniquePtr & http_maybe_encoding_buffer) const; -}; - -} diff --git a/dbms/DataStreams/HTTPOutputStreams.cpp b/dbms/DataStreams/HTTPOutputStreams.cpp deleted file mode 100644 index 31f4929bef1..00000000000 --- a/dbms/DataStreams/HTTPOutputStreams.cpp +++ /dev/null @@ -1,213 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "HTTPOutputStreams.h" - - -namespace DB -{ - -namespace -{ - inline void listeningProgress(Context & context, ProgressCallback listener) - { - auto prev = context.getProgressCallback(); - context.setProgressCallback([prev, listener] (const Progress & progress) - { - if (prev) - prev(progress); - - listener(progress); - }); - } - - inline ProgressCallback cancelListener(Context & context, Poco::Net::StreamSocket & socket) - { - /// Assume that at the point this method is called no one is reading data from the socket any more. - /// True for read-only queries. - return [&context, &socket](const Progress &) - { - try - { - char b; - int status = socket.receiveBytes(&b, 1, MSG_DONTWAIT | MSG_PEEK); - if (status == 0) - context.killCurrentQuery(); - } - catch (Poco::TimeoutException &) - { - } - catch (...) - { - context.killCurrentQuery(); - } - }; - } -} - -HTTPOutputStreams::HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, bool internal_compress) - : out(raw_out) - , out_maybe_compressed(createMaybeCompressionOut(internal_compress, out)) - , out_maybe_delayed_and_compressed(out_maybe_compressed) -{ -} - -HTTPOutputStreams::HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, Context & context, HTTPServerRequest & request, HTMLForm & form) - : out(raw_out) - , out_maybe_compressed(createMaybeCompressionOut(form.getParsed("compress", false), out)) - , out_maybe_delayed_and_compressed(createMaybeDelayedAndCompressionOut(context, form, out_maybe_compressed)) -{ - Settings & settings = context.getSettingsRef(); - - /// HTTP response compression is turned on only if the client signalled that they support it - /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. - out->setCompression(out->getCompression() && settings.enable_http_compression); - if (out->getCompression()) - out->setCompressionLevel(settings.http_zlib_compression_level); - - out->setSendProgressInterval(settings.http_headers_progress_interval_ms); - - /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed Origin header. - out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); - - /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. - if (settings.send_progress_in_http_headers) - listeningProgress(context, [this] (const Progress & progress) { out->onProgress(progress); }); - - if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) - { - Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); - - listeningProgress(context, cancelListener(context, socket)); - } -} - -WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(bool compression, HTTPResponseBufferPtr & out_) -{ - /// Client can pass a 'compress' flag in the query string. In this case the query result is - /// compressed using internal algorithm. This is not reflected in HTTP headers. - return compression ? std::make_shared(*out_) : WriteBufferPtr(out_); -} - -WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_) -{ - /// If it is specified, the whole result will be buffered. - /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. - bool buffer_until_eof = form.getParsed("wait_end_of_query", false); - - /// At least, we should postpone sending of first buffer_size result bytes - size_t buffer_size_total = std::max(form.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); - - size_t buffer_size_memory = (buffer_size_total > DBMS_DEFAULT_BUFFER_SIZE) ? buffer_size_total : 0; - - if (buffer_size_memory > 0 || buffer_until_eof) - { - CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; - CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; - - if (buffer_size_memory > 0) - cascade_buffer1.emplace_back(std::make_shared(buffer_size_memory)); - - if (buffer_until_eof) - { - std::string tmp_path_template = context.getTemporaryPath() + "http_buffers/"; - - auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) - { - return WriteBufferFromTemporaryFile::create(tmp_path_template); - }; - - cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); - } - else - { - auto push_memory_buffer_and_continue = [next_buffer = out_] (const WriteBufferPtr & prev_buf) - { - auto prev_memory_buffer = typeid_cast(prev_buf.get()); - if (!prev_memory_buffer) - throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); - - auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); - copyData(*rdbuf , *next_buffer); - - return next_buffer; - }; - - cascade_buffer2.emplace_back(push_memory_buffer_and_continue); - } - - return std::make_shared(std::move(cascade_buffer1), std::move(cascade_buffer2)); - } - - return out_; -} - -HTTPOutputStreams::~HTTPOutputStreams() -{ - /// This could be a broken HTTP Request - /// Because it does not call finalize or writes some data to output stream after call finalize - /// In this case we need to clean up its broken state to ensure that they are not sent to the client - - /// For delayed stream, we destory CascadeBuffer and without sending any data to client. - if (out_maybe_delayed_and_compressed != out_maybe_compressed) - out_maybe_delayed_and_compressed.reset(); - - if (out->count() == out->offset()) - { - /// If buffer has data and server never sends data to client - /// no need to send that data - out_maybe_compressed->position() = out_maybe_compressed->buffer().begin(); - out->position() = out->buffer().begin(); - } -} - -void HTTPOutputStreams::finalize() const -{ - if (out_maybe_delayed_and_compressed != out_maybe_compressed) - { - /// TODO: set Content-Length if possible - std::vector write_buffers; - std::vector read_buffers; - std::vector read_buffers_raw_ptr; - - auto cascade_buffer = typeid_cast(out_maybe_delayed_and_compressed.get()); - if (!cascade_buffer) - throw Exception("Expected CascadeWriteBuffer", ErrorCodes::LOGICAL_ERROR); - - cascade_buffer->getResultBuffers(write_buffers); - - if (write_buffers.empty()) - throw Exception("At least one buffer is expected to overwrite result into HTTP response", ErrorCodes::LOGICAL_ERROR); - - for (auto & write_buf : write_buffers) - { - IReadableWriteBuffer * write_buf_concrete; - ReadBufferPtr reread_buf; - - if (write_buf - && (write_buf_concrete = dynamic_cast(write_buf.get())) - && (reread_buf = write_buf_concrete->tryGetReadBuffer())) - { - read_buffers.emplace_back(reread_buf); - read_buffers_raw_ptr.emplace_back(reread_buf.get()); - } - } - - ConcatReadBuffer concat_read_buffer(read_buffers_raw_ptr); - copyData(concat_read_buffer, *out_maybe_compressed); - } - - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. - out_maybe_compressed->next(); - out->next(); - out->finalize(); -} - -} diff --git a/dbms/DataStreams/HTTPOutputStreams.h b/dbms/DataStreams/HTTPOutputStreams.h deleted file mode 100644 index fba122ea294..00000000000 --- a/dbms/DataStreams/HTTPOutputStreams.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -using HTTPServerRequest = Poco::Net::HTTPServerRequest; -using HTTPServerResponse = Poco::Net::HTTPServerResponse; -using HTTPResponseBufferPtr = std::shared_ptr; - -/* Raw data - * ↓ - * CascadeWriteBuffer out_maybe_delayed_and_compressed (optional) - * ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults) - * CompressedWriteBuffer out_maybe_compressed (optional) - * ↓ - * WriteBufferFromHTTPServerResponse out - */ -struct HTTPOutputStreams -{ - HTTPResponseBufferPtr out; - /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. - std::shared_ptr out_maybe_compressed; - /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. - std::shared_ptr out_maybe_delayed_and_compressed; - - ~HTTPOutputStreams(); - - void finalize() const; - - WriteBufferPtr createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_); - - WriteBufferPtr createMaybeCompressionOut(bool compression, std::shared_ptr & out_); - - HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, bool internal_compress); - - HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, Context & context, HTTPServerRequest & request, HTMLForm & form); -}; - -using HTTPOutputStreamsPtr = std::unique_ptr; - -} diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 064cda4847a..a252310cc93 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -4,21 +4,12 @@ set(CLICKHOUSE_SERVER_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/InterserverIOHTTPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/MetricsTransmitter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/NotFoundHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/PingRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/PrometheusMetricsWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/PrometheusRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/ReplicasStatusHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/RootRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/StaticRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/Server.cpp ${CMAKE_CURRENT_SOURCE_DIR}/TCPHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPHandlerFactory.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPQueryRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPPingRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPRootRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPSessionContextHolder.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPExceptionHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp ) set(CLICKHOUSE_SERVER_SOURCES diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 593a156ca52..98093e94c79 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -1,5 +1,8 @@ #include "HTTPHandler.h" +#include "HTTPHandlerFactory.h" +#include "HTTPHandlerRequestFilter.h" + #include #include #include @@ -7,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -32,6 +36,7 @@ #include #include #include +#include #include #include @@ -78,6 +83,7 @@ namespace ErrorCodes extern const int UNKNOWN_FORMAT; extern const int UNKNOWN_DATABASE_ENGINE; extern const int UNKNOWN_TYPE_OF_QUERY; + extern const int NO_ELEMENTS_IN_CONFIG; extern const int QUERY_IS_TOO_LARGE; @@ -90,7 +96,6 @@ namespace ErrorCodes extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; - extern const int UNKNOW_QUERY_EXECUTOR; } @@ -118,8 +123,7 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti exception_code == ErrorCodes::INCORRECT_DATA || exception_code == ErrorCodes::TYPE_MISMATCH) return HTTPResponse::HTTP_BAD_REQUEST; - else if (exception_code == ErrorCodes::UNKNOW_QUERY_EXECUTOR || - exception_code == ErrorCodes::UNKNOWN_TABLE || + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_TYPE || @@ -172,44 +176,67 @@ static std::chrono::steady_clock::duration parseSessionTimeout( } -HTTPHandler::HTTPHandler(IServer & server_) - : server(server_), log(&Logger::get("HTTPHandler")) +void HTTPHandler::pushDelayedResults(Output & used_output) +{ + std::vector write_buffers; + std::vector read_buffers; + std::vector read_buffers_raw_ptr; + + auto cascade_buffer = typeid_cast(used_output.out_maybe_delayed_and_compressed.get()); + if (!cascade_buffer) + throw Exception("Expected CascadeWriteBuffer", ErrorCodes::LOGICAL_ERROR); + + cascade_buffer->getResultBuffers(write_buffers); + + if (write_buffers.empty()) + throw Exception("At least one buffer is expected to overwrite result into HTTP response", ErrorCodes::LOGICAL_ERROR); + + for (auto & write_buf : write_buffers) + { + IReadableWriteBuffer * write_buf_concrete; + ReadBufferPtr reread_buf; + + if (write_buf + && (write_buf_concrete = dynamic_cast(write_buf.get())) + && (reread_buf = write_buf_concrete->tryGetReadBuffer())) + { + read_buffers.emplace_back(reread_buf); + read_buffers_raw_ptr.emplace_back(reread_buf.get()); + } + } + + ConcatReadBuffer concat_read_buffer(read_buffers_raw_ptr); + copyData(concat_read_buffer, *used_output.out_maybe_compressed); +} + + +HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) + : server(server_) + , log(&Logger::get(name)) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } -HTTPHandler::SessionContextHolder::~SessionContextHolder() +void HTTPHandler::processQuery( + Poco::Net::HTTPServerRequest & request, + HTMLForm & params, + Poco::Net::HTTPServerResponse & response, + Output & used_output) { - if (session_context) - session_context->releaseSession(session_id, session_timeout); -} + Context context = server.context(); + CurrentThread::QueryScope query_scope(context); -HTTPHandler::SessionContextHolder::SessionContextHolder(Context & query_context_, HTTPRequest & request, HTMLForm & params) - : query_context(query_context_) -{ - authentication(request, params); + LOG_TRACE(log, "Request URI: " << request.getURI()); - { - session_id = params.get("session_id", ""); + std::istream & istr = request.stream(); - if (!session_id.empty()) - { - session_timeout = parseSessionTimeout(query_context.getConfigRef(), params); - session_context = query_context.acquireSession(session_id, session_timeout, params.check("session_check", "1")); - - query_context = *session_context; - query_context.setSessionContext(*session_context); - } - } -} - -void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & request, HTMLForm & params) -{ - auto user = request.get("X-ClickHouse-User", ""); - auto password = request.get("X-ClickHouse-Key", ""); - auto quota_key = request.get("X-ClickHouse-Quota", ""); + /// The user and password can be passed by headers (similar to X-Auth-*), + /// which is used by load balancers to pass authentication information. + std::string user = request.get("X-ClickHouse-User", ""); + std::string password = request.get("X-ClickHouse-Key", ""); + std::string quota_key = request.get("X-ClickHouse-Quota", ""); if (user.empty() && password.empty() && quota_key.empty()) { @@ -243,26 +270,331 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque } std::string query_id = params.get("query_id", ""); - query_context.setUser(user, password, request.clientAddress(), quota_key); - query_context.setCurrentQueryId(query_id); + context.setUser(user, password, request.clientAddress(), quota_key); + context.setCurrentQueryId(query_id); + + /// The user could specify session identifier and session timeout. + /// It allows to modify settings, create temporary tables and reuse them in subsequent requests. + + std::shared_ptr session; + String session_id; + std::chrono::steady_clock::duration session_timeout; + bool session_is_set = params.has("session_id"); + const auto & config = server.config(); + + if (session_is_set) + { + session_id = params.get("session_id"); + session_timeout = parseSessionTimeout(config, params); + std::string session_check = params.get("session_check", ""); + + session = context.acquireNamedSession(session_id, session_timeout, session_check == "1"); + + context = session->context; + context.setSessionContext(session->context); + } + + SCOPE_EXIT({ + if (session) + session->release(); + }); + + /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). + String http_response_compression_methods = request.get("Accept-Encoding", ""); + CompressionMethod http_response_compression_method = CompressionMethod::None; + + if (!http_response_compression_methods.empty()) + { + /// If client supports brotli - it's preferred. + /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. + /// NOTE parsing of the list of methods is slightly incorrect. + + if (std::string::npos != http_response_compression_methods.find("br")) + http_response_compression_method = CompressionMethod::Brotli; + else if (std::string::npos != http_response_compression_methods.find("gzip")) + http_response_compression_method = CompressionMethod::Gzip; + else if (std::string::npos != http_response_compression_methods.find("deflate")) + http_response_compression_method = CompressionMethod::Zlib; + } + + bool client_supports_http_compression = http_response_compression_method != CompressionMethod::None; + + /// Client can pass a 'compress' flag in the query string. In this case the query result is + /// compressed using internal algorithm. This is not reflected in HTTP headers. + bool internal_compression = params.getParsed("compress", false); + + /// At least, we should postpone sending of first buffer_size result bytes + size_t buffer_size_total = std::max( + params.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); + + /// If it is specified, the whole result will be buffered. + /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. + bool buffer_until_eof = params.getParsed("wait_end_of_query", false); + + size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE; + size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; + + unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); + + used_output.out = std::make_shared( + request, response, keep_alive_timeout, client_supports_http_compression, http_response_compression_method); + + if (internal_compression) + used_output.out_maybe_compressed = std::make_shared(*used_output.out); + else + used_output.out_maybe_compressed = used_output.out; + + if (buffer_size_memory > 0 || buffer_until_eof) + { + CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; + CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; + + if (buffer_size_memory > 0) + cascade_buffer1.emplace_back(std::make_shared(buffer_size_memory)); + + if (buffer_until_eof) + { + const std::string tmp_path(context.getTemporaryVolume()->getNextDisk()->getPath()); + const std::string tmp_path_template(tmp_path + "http_buffers/"); + + auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) + { + return WriteBufferFromTemporaryFile::create(tmp_path_template); + }; + + cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); + } + else + { + auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed] (const WriteBufferPtr & prev_buf) + { + auto prev_memory_buffer = typeid_cast(prev_buf.get()); + if (!prev_memory_buffer) + throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); + + auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); + copyData(*rdbuf , *next_buffer); + + return next_buffer; + }; + + cascade_buffer2.emplace_back(push_memory_buffer_and_continue); + } + + used_output.out_maybe_delayed_and_compressed = std::make_shared( + std::move(cascade_buffer1), std::move(cascade_buffer2)); + } + else + { + used_output.out_maybe_delayed_and_compressed = used_output.out_maybe_compressed; + } + + std::unique_ptr in_post_raw = std::make_unique(istr); + + /// Request body can be compressed using algorithm specified in the Content-Encoding header. + String http_request_compression_method_str = request.get("Content-Encoding", ""); + std::unique_ptr in_post = wrapReadBufferWithCompressionMethod( + std::make_unique(istr), chooseCompressionMethod({}, http_request_compression_method_str)); + + /// The data can also be compressed using incompatible internal algorithm. This is indicated by + /// 'decompress' query parameter. + std::unique_ptr in_post_maybe_compressed; + bool in_post_compressed = false; + if (params.getParsed("decompress", false)) + { + in_post_maybe_compressed = std::make_unique(*in_post); + in_post_compressed = true; + } + else + in_post_maybe_compressed = std::move(in_post); + + std::unique_ptr in; + + static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check"}; + + Names reserved_param_suffixes; + + auto param_could_be_skipped = [&] (const String & name) + { + if (reserved_param_names.count(name)) + return true; + + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } + + return false; + }; + + /// Settings can be overridden in the query. + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + + /// In theory if initially readonly = 0, the client can change any setting and then set readonly + /// to some other value. + const auto & settings = context.getSettingsRef(); + + /// Only readonly queries are allowed for HTTP GET requests. + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + { + if (settings.readonly == 0) + context.setSetting("readonly", 2); + } + + bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); + + if (has_external_data) + { + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + reserved_param_suffixes.reserve(3); + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + reserved_param_suffixes.emplace_back("_format"); + reserved_param_suffixes.emplace_back("_types"); + reserved_param_suffixes.emplace_back("_structure"); + } + + SettingsChanges settings_changes; + for (const auto & [key, value] : params) + { + if (key == "database") + { + context.setCurrentDatabase(value); + } + else if (key == "default_format") + { + context.setDefaultFormat(value); + } + else if (param_could_be_skipped(key)) + { + } + else + { + /// Other than query parameters are treated as settings. + if (!customizeQueryParam(context, key, value)) + settings_changes.push_back({key, value}); + } + } + + /// For external data we also want settings + context.checkSettingsConstraints(settings_changes); + context.applySettingsChanges(settings_changes); + + const auto & query = getQuery(request, params, context); + std::unique_ptr in_param = std::make_unique(query); + in = has_external_data ? std::move(in_param) : std::make_unique(*in_param, *in_post_maybe_compressed); + + /// HTTP response compression is turned on only if the client signalled that they support it + /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. + used_output.out->setCompression(client_supports_http_compression && settings.enable_http_compression); + if (client_supports_http_compression) + used_output.out->setCompressionLevel(settings.http_zlib_compression_level); + + used_output.out->setSendProgressInterval(settings.http_headers_progress_interval_ms); + + /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, + /// checksums of client data compressed with internal algorithm are not checked. + if (in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress) + static_cast(*in_post_maybe_compressed).disableChecksumming(); + + /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed + /// Origin header. + used_output.out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); + + ClientInfo & client_info = context.getClientInfo(); + client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + client_info.interface = ClientInfo::Interface::HTTP; + + /// Query sent through HTTP interface is initial. + client_info.initial_user = client_info.current_user; + client_info.initial_query_id = client_info.current_query_id; + client_info.initial_address = client_info.current_address; + + ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + http_method = ClientInfo::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) + http_method = ClientInfo::HTTPMethod::POST; + + client_info.http_method = http_method; + client_info.http_user_agent = request.get("User-Agent", ""); + + auto append_callback = [&context] (ProgressCallback callback) + { + auto prev = context.getProgressCallback(); + + context.setProgressCallback([prev, callback] (const Progress & progress) + { + if (prev) + prev(progress); + + callback(progress); + }); + }; + + /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. + if (settings.send_progress_in_http_headers) + append_callback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); + + if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) + { + Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); + + append_callback([&context, &socket](const Progress &) + { + /// Assume that at the point this method is called no one is reading data from the socket any more. + /// True for read-only queries. + try + { + char b; + //FIXME looks like MSG_DONTWAIT is useless because of POCO_BROKEN_TIMEOUTS + int status = socket.receiveBytes(&b, 1, MSG_DONTWAIT | MSG_PEEK); + if (status == 0) + context.killCurrentQuery(); + } + catch (Poco::TimeoutException &) + { + } + catch (...) + { + context.killCurrentQuery(); + } + }); + } + + customizeContext(context); + + executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone) + { + response.setContentType(content_type); + response.add("X-ClickHouse-Query-Id", current_query_id); + response.add("X-ClickHouse-Format", format); + response.add("X-ClickHouse-Timezone", timezone); + } + ); + + if (used_output.hasDelayed()) + { + /// TODO: set Content-Length if possible + pushDelayedResults(used_output); + } + + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to + /// the client. + used_output.out->finalize(); } -void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response, HTTPResponseBufferPtr & response_out) -{ - const auto & name_with_custom_executor = context.getCustomExecutor(request, params); - LOG_TRACE(log, "Using '" << name_with_custom_executor.first << "' CustomExecutor to execute URI: " << request.getURI()); - - ExtractorClientInfo{context.getClientInfo()}.extract(request); - ExtractorContextChange{context, name_with_custom_executor.second}.extract(request, params); - - HTTPInputStreams input_streams{context, request, params}; - HTTPOutputStreams output_streams = HTTPOutputStreams(response_out, context, request, params); - name_with_custom_executor.second->executeQuery(context, request, response, params, input_streams, output_streams); -} - -void HTTPHandler::trySendExceptionToClient( - const std::string & message, int exception_code, HTTPRequest & request, - HTTPResponse & response, HTTPResponseBufferPtr response_out, bool compression) +void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_code, + Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + Output & used_output) { try { @@ -270,14 +602,19 @@ void HTTPHandler::trySendExceptionToClient( /// If HTTP method is POST and Keep-Alive is turned on, we should read the whole request body /// to avoid reading part of the current request body in the next request. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() - && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + && response.getKeepAlive() + && !request.stream().eof() + && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) { request.stream().ignore(std::numeric_limits::max()); } - if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::REQUIRED_PASSWORD) + bool auth_fail = exception_code == ErrorCodes::UNKNOWN_USER || + exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD; + + if (auth_fail) { response.requireAuthentication("ClickHouse server HTTP API"); } @@ -286,16 +623,36 @@ void HTTPHandler::trySendExceptionToClient( response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); } - if (!response_out && !response.sent()) - response.send() << message << std::endl; - else + if (!response.sent() && !used_output.out_maybe_compressed) { - HTTPOutputStreams output_streams(response_out, compression); + /// If nothing was sent yet and we don't even know if we must compress the response. + response.send() << s << std::endl; + } + else if (used_output.out_maybe_compressed) + { + /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references + if (used_output.hasDelayed()) + used_output.out_maybe_delayed_and_compressed.reset(); - writeString(message, *output_streams.out_maybe_compressed); - writeChar('\n', *output_streams.out_maybe_compressed); + /// Send the error message into already used (and possibly compressed) stream. + /// Note that the error message will possibly be sent after some data. + /// Also HTTP code 200 could have already been sent. - output_streams.finalize(); + /// If buffer has data, and that data wasn't sent yet, then no need to send that data + bool data_sent = used_output.out->count() != used_output.out->offset(); + + if (!data_sent) + { + used_output.out_maybe_compressed->position() = used_output.out_maybe_compressed->buffer().begin(); + used_output.out->position() = used_output.out->buffer().begin(); + } + + writeString(s, *used_output.out_maybe_compressed); + writeChar('\n', *used_output.out_maybe_compressed); + + used_output.out_maybe_compressed->next(); + used_output.out->next(); + used_output.out->finalize(); } } catch (...) @@ -304,41 +661,37 @@ void HTTPHandler::trySendExceptionToClient( } } + void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) { setThreadName("HTTPHandler"); ThreadStatus thread_status; + Output used_output; + /// In case of exception, send stack trace to client. - HTTPResponseBufferPtr response_out; - bool with_stacktrace = false, internal_compression = false; + bool with_stacktrace = false; try { - response_out = createResponseOut(request, response); - response.set("Content-Type", "text/plain; charset=UTF-8"); + response.setContentType("text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); - /// For keep-alive to work. if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); HTMLForm params(request); with_stacktrace = params.getParsed("stacktrace", false); - internal_compression = params.getParsed("compress", false); /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) - throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); - + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && + !request.hasContentLength()) { - Context query_context = server.context(); - CurrentThread::QueryScope query_scope(query_context); - - SessionContextHolder holder{query_context, request, params}; - processQuery(holder.query_context, request, params, response, response_out); - LOG_INFO(log, "Done processing query"); + throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); } + + processQuery(request, params, response, used_output); + LOG_INFO(log, "Done processing query"); } catch (...) { @@ -347,36 +700,103 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne /** If exception is received from remote server, then stack trace is embedded in message. * If exception is thrown on local server, then stack trace is in separate field. */ - int exception_code = getCurrentExceptionCode(); std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - trySendExceptionToClient(exception_message, exception_code, request, response, response_out, internal_compression); + int exception_code = getCurrentExceptionCode(); + + trySendExceptionToClient(exception_message, exception_code, request, response, used_output); } } -HTTPResponseBufferPtr HTTPHandler::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response) +DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & param_name_) + : HTTPHandler(server_, "DynamicQueryHandler"), param_name(param_name_) { - size_t keep_alive = server.config().getUInt("keep_alive_timeout", 10); - /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). - String http_response_compression_methods = request.get("Accept-Encoding", ""); +} - if (!http_response_compression_methods.empty()) +bool DynamicQueryHandler::customizeQueryParam(Context & context, const std::string & key, const std::string & value) +{ + if (key == param_name) + return true; /// do nothing + + if (startsWith(key, "param_")) { - /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. - /// NOTE parsing of the list of methods is slightly incorrect. - if (std::string::npos != http_response_compression_methods.find("gzip")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); - else if (std::string::npos != http_response_compression_methods.find("deflate")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); -#if USE_BROTLI - else if (http_response_compression_methods == "br") - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); -#endif + /// Save name and values of substitution in dictionary. + const String parameter_name = key.substr(strlen("param_")); + context.setQueryParameter(parameter_name, value); + return true; } - return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); + return false; } +std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) +{ + + if (likely(!startsWith(request.getContentType(), "multipart/form-data"))) + { + /// Part of the query can be passed in the 'query' parameter and the rest in the request body + /// (http method need not necessarily be POST). In this case the entire query consists of the + /// contents of the 'query' parameter, a line break and the request body. + std::string query_param = params.get(param_name, ""); + return query_param.empty() ? query_param : query_param + "\n"; + } + + /// Support for "external data for query processing". + /// Used in case of POST request with form-data, but it isn't expected to be deleted after that scope. + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); + + std::string full_query; + /// Params are of both form params POST and uri (GET params) + for (const auto & it : params) + if (it.first == param_name) + full_query += it.second; + + return full_query; } + +PredefineQueryHandler::PredefineQueryHandler(IServer & server, const NameSet & receive_params_, const std::string & predefine_query_) + : HTTPHandler(server, "PredefineQueryHandler"), receive_params(receive_params_), predefine_query(predefine_query_) +{ +} + +bool PredefineQueryHandler::customizeQueryParam(Context & context, const std::string & key, const std::string & value) +{ + if (receive_params.count(key)) + { + context.setQueryParameter(key, value); + return true; + } + + return false; +} + +std::string PredefineQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) +{ + if (unlikely(startsWith(request.getContentType(), "multipart/form-data"))) + { + /// Support for "external data for query processing". + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); + } + + return predefine_query; +} + +Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix) +{ + const auto & query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); + return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory(server, query_param_name), server.config(), config_prefix); +} + + +Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & server, const std::string & config_prefix) +{ + if (!server.config().has(config_prefix + ".handler.query")) + throw Exception("There is no path '" + config_prefix + ".handler.query" + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + const auto & predefine_query = server.config().getString(config_prefix + ".handler.query"); + + return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( + server, analyzeReceiveQueryParams(predefine_query), predefine_query), server.config(), config_prefix); +} +} \ No newline at end of file diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 1faa0efe907..f5139b6a9cc 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -7,8 +7,6 @@ #include #include -#include - namespace CurrentMetrics { @@ -26,26 +24,39 @@ class WriteBufferFromHTTPServerResponse; class HTTPHandler : public Poco::Net::HTTPRequestHandler { public: - explicit HTTPHandler(IServer & server_); + explicit HTTPHandler(IServer & server_, const std::string & name); void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + /// This method is called right before the query execution. + virtual void customizeContext(Context & /* context */) {} + + virtual bool customizeQueryParam(Context & context, const std::string & key, const std::string & value) = 0; + + virtual std::string getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) = 0; + private: - using HTTPRequest = Poco::Net::HTTPServerRequest; - using HTTPResponse = Poco::Net::HTTPServerResponse; - - struct SessionContextHolder + struct Output { - ~SessionContextHolder(); + /* Raw data + * ↓ + * CascadeWriteBuffer out_maybe_delayed_and_compressed (optional) + * ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults) + * CompressedWriteBuffer out_maybe_compressed (optional) + * ↓ + * WriteBufferFromHTTPServerResponse out + */ - void authentication(HTTPServerRequest & request, HTMLForm & params); + std::shared_ptr out; + /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. + std::shared_ptr out_maybe_compressed; + /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. + std::shared_ptr out_maybe_delayed_and_compressed; - SessionContextHolder(Context & query_context_, HTTPRequest & request, HTMLForm & params); - - String session_id; - Context & query_context; - std::shared_ptr session_context = nullptr; - std::chrono::steady_clock::duration session_timeout; + inline bool hasDelayed() const + { + return out_maybe_delayed_and_compressed != out_maybe_compressed; + } }; IServer & server; @@ -56,16 +67,46 @@ private: CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; - size_t getKeepAliveTimeout() { return server.config().getUInt("keep_alive_timeout", 10); } - - HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response); - - void processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response, HTTPResponseBufferPtr & response_out); + /// Also initializes 'used_output'. + void processQuery( + Poco::Net::HTTPServerRequest & request, + HTMLForm & params, + Poco::Net::HTTPServerResponse & response, + Output & used_output); void trySendExceptionToClient( - const std::string & message, int exception_code, HTTPRequest & request, - HTTPResponse & response, HTTPResponseBufferPtr response_out, bool compression); + const std::string & s, + int exception_code, + Poco::Net::HTTPServerRequest & request, + Poco::Net::HTTPServerResponse & response, + Output & used_output); + static void pushDelayedResults(Output & used_output); +}; + +class DynamicQueryHandler : public HTTPHandler +{ +private: + std::string param_name; +public: + explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query"); + + std::string getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) override; + + bool customizeQueryParam(Context &context, const std::string &key, const std::string &value) override; +}; + +class PredefineQueryHandler : public HTTPHandler +{ +private: + NameSet receive_params; + std::string predefine_query; +public: + explicit PredefineQueryHandler(IServer & server, const NameSet & receive_params, const std::string & predefine_query_); + + std::string getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) override; + + bool customizeQueryParam(Context &context, const std::string &key, const std::string &value) override; }; } diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index b2c07075d54..8dee745d0e3 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -1,9 +1,21 @@ #include "HTTPHandlerFactory.h" +#include +#include +#include +#include + +#include "HTTPHandler.h" #include "NotFoundHandler.h" -#include "HTTPRequestHandler/HTTPRootRequestHandler.h" -#include "HTTPRequestHandler/HTTPPingRequestHandler.h" -#include "HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h" +#include "StaticRequestHandler.h" +#include "ReplicasStatusHandler.h" +#include "InterserverIOHTTPHandler.h" + +#if USE_RE2_ST +#include +#else +#define re2_st re2 +#endif namespace DB @@ -11,130 +23,128 @@ namespace DB namespace ErrorCodes { - extern const int SYNTAX_ERROR; - extern const int UNKNOWN_HTTP_HANDLER_TYPE; - extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; + extern const int CANNOT_COMPILE_REGEXP; + extern const int NO_ELEMENTS_IN_CONFIG; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -InterserverIOHTTPHandlerFactory::InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_) +HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_) : server(server_), log(&Logger::get(name_)), name(name_) { } -Poco::Net::HTTPRequestHandler * InterserverIOHTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHandler(const Poco::Net::HTTPServerRequest & request) // override { LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " << request.getMethod() - << ", Address: " << request.clientAddress().toString() - << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); + << "Method: " + << request.getMethod() + << ", Address: " + << request.clientAddress().toString() + << ", User-Agent: " + << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); - const auto & uri = request.getURI(); - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) + for (auto & handler_factory : child_handler_factories) { - if (uri == "/") - return new HTTPRootRequestHandler(server); - if (uri == "/ping") - return new HTTPPingRequestHandler(server); - else if (startsWith(uri, "/replicas_status")) - return new HTTPReplicasStatusRequestHandler(server.context()); + auto handler = handler_factory->createRequestHandler(request); + if (handler != nullptr) + return handler; } - if (uri.find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - { - return new InterserverIOHTTPHandler(server); - } - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { - return new NotFoundHandler( - "Use / or /ping for health checks.\n" - "Or /replicas_status for more sophisticated health checks.\n" - "Send queries from your program with POST method or GET /?query=...\n\n" - " Use clickhouse-client:\n\n" - " For interactive data analysis:\n" - " clickhouse-client\n\n" - " For batch query processing:\n" - " clickhouse-client --query='SELECT 1' > result\n" - " clickhouse-client < query > result" - ); + return new NotFoundHandler; } return nullptr; } -HTTPHandlerFactory::HTTPHandlerFactory(IServer & server_, const std::string & name_) - : server(server_), log(&Logger::get(name_)), name(name_) +HTTPRequestHandlerFactoryMain::~HTTPRequestHandlerFactoryMain() { - updateHTTPHandlersCreator(server.config()); - - if (handlers_creator.empty()) - throw Exception("The HTTPHandlers does not exist in the config.xml", ErrorCodes::EMPTY_HTTP_HANDLER_IN_CONFIG); + while (!child_handler_factories.empty()) + delete child_handler_factories.back(), child_handler_factories.pop_back(); } -Poco::Net::HTTPRequestHandler * HTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +HTTPRequestHandlerFactoryMain::TThis * HTTPRequestHandlerFactoryMain::addHandler(Poco::Net::HTTPRequestHandlerFactory * child_factory) { - LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " << request.getMethod() - << ", Address: " << request.clientAddress().toString() - << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); - - for (const auto & [matcher, creator] : handlers_creator) - { - if (matcher(request)) - return creator(); - } - - return new NotFoundHandler(no_handler_description); + child_handler_factories.emplace_back(child_factory); + return this; } -HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer &, const String &); - -HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer &, const String &); - -void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) +static inline auto createHandlersFactoryFromConfig(IServer & server, const std::string & name, const String & prefix) { - Poco::Util::AbstractConfiguration::Keys http_handlers_item_key; - configuration.keys(key, http_handlers_item_key); + auto main_handler_factory = new HTTPRequestHandlerFactoryMain(server, name); - handlers_creator.reserve(http_handlers_item_key.size()); - for (const auto & http_handler_type_name : http_handlers_item_key) + Poco::Util::AbstractConfiguration::Keys keys; + server.config().keys(prefix, keys); + + for (const auto & key : keys) { - if (http_handler_type_name.find('.') != String::npos) - throw Exception("HTTPHandler type name with dots are not supported: '" + http_handler_type_name + "'", ErrorCodes::SYNTAX_ERROR); + if (!startsWith(key, "routing_rule")) + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - const auto & handler_key = key + "." + http_handler_type_name; + const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); - if (startsWith(http_handler_type_name, "root_handler")) - handlers_creator.push_back({createRootHandlerMatcher(server, handler_key), createRootHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "ping_handler")) - handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "dynamic_query_handler")) - handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "predefined_query_handler")) - handlers_creator.push_back({createPredefinedQueryHandlerMatcher(server, handler_key), createPredefinedQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "replicas_status_handler")) - handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); - else if (http_handler_type_name == "no_handler_description") - no_handler_description = configuration.getString(key + ".no_handler_description"); + if (handler_type == "static") + main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix)); + else if (handler_type == "dynamic_query_handler") + main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix)); + else if (handler_type == "predefine_query_handler") + main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix)); else - throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOWN_HTTP_HANDLER_TYPE); + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + + return main_handler_factory; +} + +static const auto ping_response_expression = "Ok.\n"; +static const auto root_response_expression = "config://http_server_default_response"; + +static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IServer & server, const std::string & name) +{ + if (server.config().has("routing_rules")) + return createHandlersFactoryFromConfig(server, name, "routing_rules"); + else + { + return (new HTTPRequestHandlerFactoryMain(server, name)) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) + ->attachStrictPath("/")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) + ->attachStrictPath("/ping")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server)) + ->attachNonStrictPath("/replicas_status")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, "query"))->allowPostAndGetParamsRequest()); + /// TODO: +// if (configuration.has("prometheus") && configuration.getInt("prometheus.port", 0) == 0) +// handler_factory->addHandler(async_metrics); } } +static inline Poco::Net::HTTPRequestHandlerFactory * createInterserverHTTPHandlerFactory(IServer & server, const std::string & name) +{ + return (new HTTPRequestHandlerFactoryMain(server, name)) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) + ->attachStrictPath("/")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) + ->attachStrictPath("/ping")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server)) + ->attachNonStrictPath("/replicas_status")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server))->allowPostAndGetParamsRequest()); } + +Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, const std::string & name) +{ + if (name == "HTTPHandler-factory" || name == "HTTPSHandler-factory") + return createHTTPHandlerFactory(server, name); + else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") + return createInterserverHTTPHandlerFactory(server, name); + + throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR); +} + +} \ No newline at end of file diff --git a/programs/server/HTTPHandlerFactory.h b/programs/server/HTTPHandlerFactory.h index 5add7b619af..15f15db9768 100644 --- a/programs/server/HTTPHandlerFactory.h +++ b/programs/server/HTTPHandlerFactory.h @@ -1,51 +1,113 @@ #pragma once +#include "IServer.h" +#include +#include #include #include #include -#include -#include -#include "IServer.h" -#include "InterserverIOHTTPHandler.h" - namespace DB { -class InterserverIOHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +/// Handle request using child handlers +class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactory { -public: - InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - private: - IServer & server; - Logger * log; - std::string name; -}; + using TThis = HTTPRequestHandlerFactoryMain; -using HTTPHandlerCreator = std::function; -using HTTPHandlerMatcher = std::function; -using HTTPHandlerMatcherAndCreator = std::pair; -using HTTPHandlersMatcherAndCreator = std::vector; - -class HTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -public: - HTTPHandlerFactory(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - - void updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key = "http_handlers"); - -private: IServer & server; Logger * log; std::string name; - String no_handler_description; - HTTPHandlersMatcherAndCreator handlers_creator; + std::vector child_handler_factories; +public: + + ~HTTPRequestHandlerFactoryMain(); + + HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_); + + TThis * addHandler(Poco::Net::HTTPRequestHandlerFactory * child_factory); + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; }; +template +class RoutingRuleHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +public: + using TThis = RoutingRuleHTTPHandlerFactory; + using Filter = std::function; + + template + RoutingRuleHTTPHandlerFactory(TArgs &&... args) + { + creator = [args = std::tuple(std::forward(args) ...)]() + { + return std::apply([&](auto && ... endpoint_args) + { + return new TEndpoint(std::forward(endpoint_args)...); + }, std::move(args)); + }; + } + + TThis * addFilter(Filter cur_filter) + { + Filter prev_filter = filter; + filter = [prev_filter, cur_filter](const auto & request) + { + return prev_filter ? prev_filter(request) && cur_filter(request) : cur_filter(request); + }; + + return this; + } + + TThis * attachStrictPath(const String & strict_path) + { + return addFilter([strict_path](const auto & request) { return request.getURI() == strict_path; }); + } + + TThis * attachNonStrictPath(const String & non_strict_path) + { + return addFilter([non_strict_path](const auto & request) { return startsWith(request.getURI(), non_strict_path); }); + } + + /// Handle GET or HEAD endpoint on specified path + TThis * allowGetAndHeadRequest() + { + return addFilter([](const auto & request) + { + return request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; + }); + } + + /// Handle POST or GET with params + TThis * allowPostAndGetParamsRequest() + { + return addFilter([](const auto & request) + { + return request.getURI().find('?') != std::string::npos + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST; + }); + } + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override + { + return filter(request) ? creator() : nullptr; + } + +private: + Filter filter; + std::function creator; +}; + +Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, const std::string & name); + +Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & server, const std::string & config_prefix); + +Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix); + +Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & server, const std::string & config_prefix); + } diff --git a/programs/server/HTTPHandlerRequestFilter.h b/programs/server/HTTPHandlerRequestFilter.h new file mode 100644 index 00000000000..5f794884239 --- /dev/null +++ b/programs/server/HTTPHandlerRequestFilter.h @@ -0,0 +1,98 @@ +#pragma once + +#include "HTTPHandlerFactory.h" + +#include +#include +#include +#include + +#include + +#if USE_RE2_ST +#include +#else +#define re2_st re2 +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPILE_REGEXP; +} + +static inline std::string uriPathGetter(const Poco::Net::HTTPServerRequest & request) +{ + const auto & uri = request.getURI(); + const auto & end = find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()); + + return std::string(uri.data(), end - uri.data()); +} + +static inline std::function headerGetter(const std::string & header_name) +{ + return [header_name](const Poco::Net::HTTPServerRequest & request) { return request.get(header_name, ""); }; +} + +static inline auto methodsExpressionFilter(const std::string &methods_expression) +{ + Poco::StringTokenizer tokenizer(Poco::toUpper(Poco::trim(methods_expression)), ","); + return [methods = std::vector(tokenizer.begin(), tokenizer.end())](const Poco::Net::HTTPServerRequest & request) + { + return std::count(methods.begin(), methods.end(), request.getMethod()); + }; +} + +template +static inline auto regularExpressionFilter(const std::string & regular_expression, const GetFunction & get) +{ + auto compiled_regex = std::make_shared(regular_expression); + + if (!compiled_regex->ok()) + throw Exception("cannot compile re2: " + regular_expression + " for routing_rule, error: " + compiled_regex->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); + + return std::make_pair(compiled_regex, [get = std::move(get), compiled_regex](const Poco::Net::HTTPServerRequest & request) + { + const auto & test_content = get(request); + int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(test_content.data(), test_content.size()); + return compiled_regex->Match(input, 0, test_content.size(), re2_st::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); + }); +} + +template +static inline std::function expressionFilter(const std::string & expression, const GetFunction & get) +{ + if (startsWith(expression, "regex:")) + return regularExpressionFilter(expression, get).second; + + return [expression, get = std::move(get)](const Poco::Net::HTTPServerRequest & request) { return get(request) == expression; }; +} + +template +static inline Poco::Net::HTTPRequestHandlerFactory * addFiltersFromConfig( + RoutingRuleHTTPHandlerFactory * factory, Poco::Util::AbstractConfiguration & config, const std::string & prefix) +{ + Poco::Util::AbstractConfiguration::Keys filters_type; + config.keys(prefix, filters_type); + + for (const auto & filter_type : filters_type) + { + if (filter_type == "handler") + continue; /// Skip handler config + else if (filter_type == "method") + factory->addFilter(methodsExpressionFilter(config.getString(prefix + "." + filter_type))); + else + factory->addFilter(expressionFilter(config.getString(prefix + "." + filter_type), filter_type == "url" + ? uriPathGetter : headerGetter(filter_type))); + } + + return factory; +} + +} diff --git a/programs/server/HTTPRequestHandler/ExtractorClientInfo.h b/programs/server/HTTPRequestHandler/ExtractorClientInfo.h deleted file mode 100644 index 76840f3d682..00000000000 --- a/programs/server/HTTPRequestHandler/ExtractorClientInfo.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class ExtractorClientInfo -{ -public: - ExtractorClientInfo(ClientInfo & info_) : client_info(info_) {} - - void extract(Poco::Net::HTTPServerRequest & request) - { - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.interface = ClientInfo::Interface::HTTP; - - /// Query sent through HTTP interface is initial. - client_info.initial_user = client_info.current_user; - client_info.initial_query_id = client_info.current_query_id; - client_info.initial_address = client_info.current_address; - - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) - http_method = ClientInfo::HTTPMethod::GET; - else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) - http_method = ClientInfo::HTTPMethod::POST; - - client_info.http_method = http_method; - client_info.http_user_agent = request.get("User-Agent", ""); - } - -private: - ClientInfo & client_info; -}; - -} diff --git a/programs/server/HTTPRequestHandler/ExtractorContextChange.h b/programs/server/HTTPRequestHandler/ExtractorContextChange.h deleted file mode 100644 index 1d4b4e5f58a..00000000000 --- a/programs/server/HTTPRequestHandler/ExtractorContextChange.h +++ /dev/null @@ -1,116 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -class ExtractorContextChange -{ -public: - ExtractorContextChange(Context & context_, bool settings_may_in_post_) : context(context_), settings_may_in_post(settings_may_in_post_) {} - - static const NameSet & getReservedParamNames() - { - static const NameSet reserved_param_names{ - "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check" - }; - - return reserved_param_names; - } - - static std::function reservedParamSuffixesFilter(bool reserved) - { - if (!reserved) - return [&](const String &) { return false; }; - - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - return [&](const String & param_name) - { - if (endsWith(param_name, "_format")) - return true; - else if (endsWith(param_name, "_types")) - return true; - else if (endsWith(param_name, "_structure")) - return true; - - return false; - }; - } - - void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) - { - bool is_multipart_data = startsWith(request.getContentType().data(), "multipart/form-data"); - - /// Settings can be overridden in the query. - /// Some parameters (database, default_format, everything used in the code above) do not - /// belong to the Settings class. - becomeReadonlyIfNeeded(request); - changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); - - if (is_multipart_data || settings_may_in_post) - { - ExternalTablesHandler handler(context, params); - params.load(request, request.stream(), handler); - - /// We use the `Post Request Body Settings` to override the `Qeruy String Param settings` - if (settings_may_in_post) - changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); - } - } - -private: - Context & context; - bool settings_may_in_post; - - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. - - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - /// Only readonly queries are allowed for HTTP GET requests. - void becomeReadonlyIfNeeded(Poco::Net::HTTPServerRequest & request) - { - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) - { - Settings & settings = context.getSettingsRef(); - - if (settings.readonly == 0) - settings.readonly = 2; - } - } - - - void changeSettingsFromParams(HTMLForm & params, const std::function & reserved_param_suffixes) - { - SettingsChanges settings_changes; - const auto & reserved_param_names = getReservedParamNames(); - - for (const auto & [name, value] : params) - { - if (name == "database") - context.setCurrentDatabase(value); - else if (name == "default_format") - context.setDefaultFormat(value); - else if (!reserved_param_names.count(name) && !reserved_param_suffixes(name)) - { - if (Settings::findIndex(name) != Settings::npos) - settings_changes.push_back({name, value}); - } - } - - /// For external data we also want settings - context.checkSettingsConstraints(settings_changes); - context.applySettingsChanges(settings_changes); - } -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp b/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp deleted file mode 100644 index 8bfa351aab2..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp +++ /dev/null @@ -1,154 +0,0 @@ -#include "HTTPExceptionHandler.h" - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int READONLY; - extern const int UNKNOWN_COMPRESSION_METHOD; - - extern const int CANNOT_PARSE_TEXT; - extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; - extern const int CANNOT_PARSE_QUOTED_STRING; - extern const int CANNOT_PARSE_DATE; - extern const int CANNOT_PARSE_DATETIME; - extern const int CANNOT_PARSE_NUMBER; - extern const int CANNOT_OPEN_FILE; - - extern const int UNKNOWN_ELEMENT_IN_AST; - extern const int UNKNOWN_TYPE_OF_AST_NODE; - extern const int TOO_DEEP_AST; - extern const int TOO_BIG_AST; - extern const int UNEXPECTED_AST_STRUCTURE; - - extern const int SYNTAX_ERROR; - - extern const int INCORRECT_DATA; - extern const int TYPE_MISMATCH; - - extern const int UNKNOWN_TABLE; - extern const int UNKNOWN_FUNCTION; - extern const int UNKNOWN_IDENTIFIER; - extern const int UNKNOWN_TYPE; - extern const int UNKNOWN_STORAGE; - extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_SETTING; - extern const int UNKNOWN_DIRECTION_OF_SORTING; - extern const int UNKNOWN_AGGREGATE_FUNCTION; - extern const int UNKNOWN_FORMAT; - extern const int UNKNOWN_DATABASE_ENGINE; - extern const int UNKNOWN_TYPE_OF_QUERY; - - extern const int QUERY_IS_TOO_LARGE; - - extern const int NOT_IMPLEMENTED; - extern const int SOCKET_TIMEOUT; - - extern const int UNKNOWN_USER; - extern const int WRONG_PASSWORD; - extern const int REQUIRED_PASSWORD; - - extern const int INVALID_SESSION_TIMEOUT; - extern const int HTTP_LENGTH_REQUIRED; -} - -static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code) -{ - using namespace Poco::Net; - - if (exception_code == ErrorCodes::REQUIRED_PASSWORD) - return HTTPResponse::HTTP_UNAUTHORIZED; - else if (exception_code == ErrorCodes::CANNOT_PARSE_TEXT || - exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || - exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || - exception_code == ErrorCodes::CANNOT_PARSE_DATE || - exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || - exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || - - exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || - exception_code == ErrorCodes::TOO_DEEP_AST || - exception_code == ErrorCodes::TOO_BIG_AST || - exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || - - exception_code == ErrorCodes::SYNTAX_ERROR || - - exception_code == ErrorCodes::INCORRECT_DATA || - exception_code == ErrorCodes::TYPE_MISMATCH) - return HTTPResponse::HTTP_BAD_REQUEST; - else if (exception_code == ErrorCodes::UNKNOWN_TABLE || - exception_code == ErrorCodes::UNKNOWN_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || - exception_code == ErrorCodes::UNKNOWN_TYPE || - exception_code == ErrorCodes::UNKNOWN_STORAGE || - exception_code == ErrorCodes::UNKNOWN_DATABASE || - exception_code == ErrorCodes::UNKNOWN_SETTING || - exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || - exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_FORMAT || - exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || - - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY) - return HTTPResponse::HTTP_NOT_FOUND; - else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE) - return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE; - else if (exception_code == ErrorCodes::NOT_IMPLEMENTED) - return HTTPResponse::HTTP_NOT_IMPLEMENTED; - else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || - exception_code == ErrorCodes::CANNOT_OPEN_FILE) - return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; - else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) - return HTTPResponse::HTTP_LENGTH_REQUIRED; - - return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR; -} - - -void HTTPExceptionHandler::handle( - const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response, std::shared_ptr response_out, - bool compression, Poco::Logger * log) -{ - try - { - /// If HTTP method is POST and Keep-Alive is turned on, we should read the whole request body - /// to avoid reading part of the current request body in the next request. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() - && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) - { - request.stream().ignore(std::numeric_limits::max()); - } - - if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::REQUIRED_PASSWORD) - { - response.requireAuthentication("ClickHouse server HTTP API"); - } - else - { - response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); - } - - if (!response_out && !response.sent()) - response.send() << message << std::endl; - else - { - HTTPOutputStreams output_streams(response_out, compression); - - writeString(message, *output_streams.out_maybe_compressed); - writeChar('\n', *output_streams.out_maybe_compressed); - - output_streams.finalize(); - } - } - catch (...) - { - tryLogCurrentException(log, "Cannot send exception to client"); - } -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h b/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h deleted file mode 100644 index d2a5383ed4a..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h +++ /dev/null @@ -1,19 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class HTTPExceptionHandler -{ -public: - static void handle(const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response, std::shared_ptr response_out, bool compression, - Poco::Logger * log); -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp deleted file mode 100644 index 292f98ba0eb..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp +++ /dev/null @@ -1,45 +0,0 @@ -#include "HTTPPingRequestHandler.h" - -#include - -#include - -#include -#include - -namespace DB -{ - -void HTTPPingRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) -{ - try - { - const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - const char * data = "Ok.\n"; - response.sendBuffer(data, strlen(data)); - } - catch (...) - { - tryLogCurrentException("HTTPPingRequestHandler"); - } -} - -HTTPHandlerMatcher createPingHandlerMatcher(IServer & server, const String & key) -{ - const auto & path = server.config().getString(key, "/ping"); - - return [&, path = path](const Poco::Net::HTTPServerRequest & request) - { - return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) && - request.getURI() == path; - }; -} - -HTTPHandlerCreator createPingHandlerCreator(IServer & server, const String &) -{ - return [&]() { return new HTTPPingRequestHandler(server); }; -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h deleted file mode 100644 index 7d524dd638d..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h +++ /dev/null @@ -1,26 +0,0 @@ -#pragma once - -#include "../IServer.h" -#include "../HTTPHandlerFactory.h" - -#include - - -namespace DB -{ - -/// Response with "Ok.\n". Used for availability checks. -class HTTPPingRequestHandler : public Poco::Net::HTTPRequestHandler -{ -public: - explicit HTTPPingRequestHandler(const IServer & server_) : server(server_) - { - } - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - const IServer & server; -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp deleted file mode 100644 index 723ff85abe3..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp +++ /dev/null @@ -1,175 +0,0 @@ -#include "HTTPQueryRequestHandler.h" - -#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 -#include -#include - -#include "ExtractorClientInfo.h" -#include "ExtractorContextChange.h" -#include "HTTPQueryRequestHandlerMatcherAndCreator.h" -#include "HTTPSessionContextHolder.h" -#include "HTTPExceptionHandler.h" - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int HTTP_LENGTH_REQUIRED; -} - -template -HTTPQueryRequestHandler::HTTPQueryRequestHandler(const IServer & server_, const QueryParamExtractor & extractor_) - : server(server_), log(&Logger::get("HTTPQueryRequestHandler")), extractor(extractor_) -{ - server_display_name = server.config().getString("display_name", getFQDNOrHostName()); -} - -template -void HTTPQueryRequestHandler::processQuery( - Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, - Poco::Net::HTTPServerResponse & response, HTTPResponseBufferPtr & response_out) -{ - ExtractorClientInfo{context.getClientInfo()}.extract(request); - ExtractorContextChange{context, extractor.loadSettingsFromPost()}.extract(request, params); - - HTTPInputStreams input_streams{context, request, params}; - HTTPOutputStreams output_streams = HTTPOutputStreams(response_out, context, request, params); - - const auto & queries = extractor.extract(context, request, params); - - for (const auto & [execute_query, not_touch_post] : queries) - { - ReadBufferPtr temp_query_buf; - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - if (not_touch_post && !startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, - context, [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); - } - - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. - output_streams.finalize(); -} - -template -void HTTPQueryRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) -{ - setThreadName("HTTPHandler"); - ThreadStatus thread_status; - - /// In case of exception, send stack trace to client. - HTTPResponseBufferPtr response_out; - bool with_stacktrace = false, internal_compression = false; - - try - { - response_out = createResponseOut(request, response); - response.set("Content-Type", "text/plain; charset=UTF-8"); - response.set("X-ClickHouse-Server-Display-Name", server_display_name); - - /// For keep-alive to work. - if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) - response.setChunkedTransferEncoding(true); - - HTMLForm params(request); - with_stacktrace = params.getParsed("stacktrace", false); - internal_compression = params.getParsed("compress", false); - - /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) - throw Exception("There is neither Transfer-Encoding header nor Content-Length header", ErrorCodes::HTTP_LENGTH_REQUIRED); - - { - Context query_context = server.context(); - CurrentThread::QueryScope query_scope(query_context); - - HTTPSessionContextHolder holder{query_context, request, params}; - processQuery(holder.query_context, request, params, response, response_out); - LOG_INFO(log, "Done processing query"); - } - } - catch (...) - { - tryLogCurrentException(log); - - /** If exception is received from remote server, then stack trace is embedded in message. - * If exception is thrown on local server, then stack trace is in separate field. - */ - int exception_code = getCurrentExceptionCode(); - std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - HTTPExceptionHandler::handle(exception_message, exception_code, request, response, response_out, internal_compression, log); - } -} - -template -HTTPResponseBufferPtr HTTPQueryRequestHandler::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response) -{ - size_t keep_alive = server.config().getUInt("keep_alive_timeout", 10); - /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). - String http_response_compression_methods = request.get("Accept-Encoding", ""); - - if (!http_response_compression_methods.empty()) - { - /// Both gzip and deflate are supported. If the client supports both, gzip is preferred. - /// NOTE parsing of the list of methods is slightly incorrect. - if (std::string::npos != http_response_compression_methods.find("gzip")) - return std::make_shared(request, response, keep_alive, true, CompressionMethod::Gzip); - else if (std::string::npos != http_response_compression_methods.find("deflate")) - return std::make_shared(request, response, keep_alive, true, CompressionMethod::Zlib); -#if USE_BROTLI - else if (http_response_compression_methods == "br") - return std::make_shared(request, response, keep_alive, true, CompressionMethod::Brotli); -#endif - } - - return std::make_shared(request, response, keep_alive, false, CompressionMethod{}); -} - - -template class HTTPQueryRequestHandler; -template class HTTPQueryRequestHandler; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h deleted file mode 100644 index 5303b0cda89..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include "../IServer.h" - -#include - -#include -#include - -#include - - -namespace CurrentMetrics -{ - extern const Metric HTTPConnection; -} - -namespace Poco { class Logger; } - -namespace DB -{ - -template -class HTTPQueryRequestHandler : public Poco::Net::HTTPRequestHandler -{ -public: - explicit HTTPQueryRequestHandler(const IServer & server_, const QueryParamExtractor & extractor_); - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - const IServer & server; - Poco::Logger * log; - QueryParamExtractor extractor; - - /// It is the name of the server that will be sent in an http-header X-ClickHouse-Server-Display-Name. - String server_display_name; - - CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; - - HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response); - - void processQuery( - Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, - Poco::Net::HTTPServerResponse & response, HTTPResponseBufferPtr & response_out); -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp deleted file mode 100644 index 81f25f0bd08..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp +++ /dev/null @@ -1,339 +0,0 @@ -#include "HTTPQueryRequestHandlerMatcherAndCreator.h" - -#include "../HTTPHandlerFactory.h" -#include "HTTPQueryRequestHandler.h" -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int EMPTY_PREDEFINED_QUERY; - extern const int CANNOT_COMPILE_REGEXP; - extern const int UNKNOWN_QUERY_PARAMETER; - extern const int DUPLICATE_CAPTURE_QUERY_PARAM; - extern const int ILLEGAL_HTTP_HANDLER_PARAM_NAME; - extern const int TOO_MANY_INSERT_QUERY_WITH_PREDEFINED_QUERY; -} - -ExtractorDynamicQueryParameters::ExtractorDynamicQueryParameters( - Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) - : url_regex(url_regex_), headers_regex(headers_regex_) -{ - dynamic_param_name = configuration.getString(key + ".query_param_name", "query"); - - NameSet extracted_names; - - if (url_regex) - { - for (const auto & [capturing_name, capturing_index] : url_regex->NamedCapturingGroups()) - { - if (startsWith(capturing_name, "param_")) - { - if (extracted_names.count(capturing_name)) - throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); - - extracted_names.emplace(capturing_name); - extract_from_url[capturing_name] = capturing_index; - } - } - } - - if (!headers_regex.empty()) - { - for (const auto & [header_name, header_regex] : headers_regex) - { - for (const auto & [capturing_name, capturing_index] : header_regex->NamedCapturingGroups()) - { - if (startsWith(capturing_name, "param_")) - { - if (extracted_names.count(capturing_name)) - throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); - - extracted_names.emplace(capturing_name); - extract_from_headers[header_name][capturing_name] = capturing_index; - } - } - } - } -} - -template -void extractParamWithRegex(Context & context, const RegexRule & regex, const std::map & extract_params, const String & value) -{ - if (value.empty()) - return; - - int num_captures = regex->NumberOfCapturingGroups() + 1; - - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(value.data(), value.size()); - - if (regex->Match(input, 0, value.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) - { - for (const auto & [capturing_name, capturing_index] : extract_params) - { - const auto & capturing_value = matches[capturing_index]; - - if (capturing_value.data()) - { - String param_name = capturing_name; - if constexpr (remove_prefix_for_param) - { - const static size_t prefix_size = strlen("param_"); - param_name = capturing_name.substr(prefix_size); - } - - context.setQueryParameter(param_name, String(capturing_value.data(), capturing_value.size())); - } - } - } -} - -ExtractRes ExtractorDynamicQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) -{ - if (!extract_from_url.empty()) - extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); - - - if (!extract_from_headers.empty()) - for (const auto & [header_name, extract_params] : extract_from_headers) - extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name, "")); - - String extracted_query_from_params; - const static size_t prefix_size = strlen("param_"); - - for (const auto & [param_name, param_value] : params) - { - if (param_name == dynamic_param_name) - extracted_query_from_params += param_value; - else if (startsWith(param_name, "param_")) - context.setQueryParameter(param_name.substr(prefix_size), param_value); - } - - if (!extracted_query_from_params.empty()) - extracted_query_from_params += "\n"; - - return {{extracted_query_from_params, true}}; -} - -ExtractorPredefinedQueryParameters::ExtractorPredefinedQueryParameters( - Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) - : url_regex(url_regex_), headers_regex(headers_regex_) -{ - Poco::Util::AbstractConfiguration::Keys queries_key; - configuration.keys(key + ".queries", queries_key); - - if (queries_key.empty()) - throw Exception("There must be at least one predefined query in the predefined HTTPHandler.", ErrorCodes::EMPTY_PREDEFINED_QUERY); - - for (const auto & query_key : queries_key) - { - const auto & predefine_query = configuration.getString(key + ".queries." + query_key); - - const char * query_begin = predefine_query.data(); - const char * query_end = predefine_query.data() + predefine_query.size(); - - ParserQuery parser(query_end, false); - ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); - QueryParameterVisitor{queries_names}.visit(extract_query_ast); - - bool is_insert_query = extract_query_ast->as(); - - if (has_insert_query && is_insert_query) - throw Exception("Too many insert queries in predefined queries.", ErrorCodes::TOO_MANY_INSERT_QUERY_WITH_PREDEFINED_QUERY); - - has_insert_query |= is_insert_query; - predefine_queries.push_back({predefine_query, is_insert_query}); - } - - const auto & reserved_params_name = ExtractorContextChange::getReservedParamNames(); - for (const auto & predefine_query_name : queries_names) - { - if (Settings::findIndex(predefine_query_name) != Settings::npos || reserved_params_name.count(predefine_query_name)) - throw Exception("Illegal http_handler param name '" + predefine_query_name + - "', Because it's reserved name or Settings name", ErrorCodes::ILLEGAL_HTTP_HANDLER_PARAM_NAME); - } - - NameSet extracted_names; - - if (url_regex) - { - for (const auto & [capturing_name, capturing_index] : url_regex->NamedCapturingGroups()) - { - if (queries_names.count(capturing_name)) - { - if (extracted_names.count(capturing_name)) - throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); - - extracted_names.emplace(capturing_name); - extract_from_url[capturing_name] = capturing_index; - } - } - } - - if (!headers_regex.empty()) - { - for (const auto & [header_name, header_regex] : headers_regex) - { - for (const auto & [capturing_name, capturing_index] : header_regex->NamedCapturingGroups()) - { - if (queries_names.count(capturing_name)) - { - if (extracted_names.count(capturing_name)) - throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); - - extracted_names.emplace(capturing_name); - extract_from_headers[header_name][capturing_name] = capturing_index; - } - } - } - } -} - -ExtractRes ExtractorPredefinedQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) -{ - if (!extract_from_url.empty()) - extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); - - if (!extract_from_headers.empty()) - for (const auto & [header_name, extract_params] : extract_from_headers) - extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name, "")); - - for (const auto & param : params) - if (queries_names.count(param.first)) - context.setQueryParameter(param.first, param.second); - - return predefine_queries; -} - -RegexRule HTTPQueryRequestHandlerMatcherAndCreator::createRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key) -{ - if (!configuration.has(key)) - return {}; - - const auto & regex_str = configuration.getString(key); - const auto & url_regex_rule = std::make_shared(regex_str); - - if (!url_regex_rule->ok()) - throw Exception("cannot compile re2: " + regex_str + " for HTTPHandler url, error: " + url_regex_rule->error() + - ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); - - return url_regex_rule; -} - -HeadersRegexRule HTTPQueryRequestHandlerMatcherAndCreator::createHeadersRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key) -{ - if (!configuration.has(key)) - return {}; - - Poco::Util::AbstractConfiguration::Keys headers_names; - configuration.keys(key, headers_names); - - HeadersRegexRule headers_regex_rule; - for (const auto & header_name : headers_names) - { - if (headers_regex_rule.count(header_name)) - throw Exception("Duplicate header match declaration '" + header_name + "'", ErrorCodes::LOGICAL_ERROR); - - headers_regex_rule[header_name] = createRegexRule(configuration, key + "." + header_name); - } - - return headers_regex_rule; -} - -size_t findFirstMissingMatchPos(const re2_st::RE2 & regex_rule, const String & match_content) -{ - int num_captures = regex_rule.NumberOfCapturingGroups() + 1; - - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(match_content.data(), match_content.size()); - if (regex_rule.Match(input, 0, match_content.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) - return matches[0].size(); - - return size_t(0); -} - -HTTPHandlerMatcher HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher( - const String & method, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) -{ - return [method = Poco::toLower(method), url_rule = url_rule, headers_rule = headers_rule](const Poco::Net::HTTPServerRequest & request) - { - if (!method.empty() && Poco::toLower(request.getMethod()) != method) - return false; - - if (url_rule) - { - Poco::URI uri(request.getURI()); - const auto & request_uri = uri.getPath(); - size_t first_missing_pos = findFirstMissingMatchPos(*url_rule, request_uri); - - const char * url_end = request_uri.data() + request_uri.size(); - const char * first_missing = request_uri.data() + first_missing_pos; - - if (first_missing != url_end && *first_missing == '/') - ++first_missing; - - if (first_missing != url_end && *first_missing != '?') - return false; /// Not full matched - } - - if (!headers_rule.empty()) - { - for (const auto & [header_name, header_rule] : headers_rule) - { - if (!request.has(header_name)) - return false; - - const String & header_value = request.get(header_name); - if (header_value.size() != findFirstMissingMatchPos(*header_rule, header_value)) - return false; - } - } - - return true; - }; -} - -HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer & server, const String & key) -{ - return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, - HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); -} - - -HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer & server, const String & key) -{ - return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, - HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); -} - -HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer & server, const String & key) -{ - return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( - server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) - { - const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); - - return [&, query_extract = extract]() - { - return new HTTPQueryRequestHandler(server, *query_extract); - }; - }); -} - -HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer & server, const String & key) -{ - return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( - server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) - { - const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); - - return [&, query_extract = extract]() - { - return new HTTPQueryRequestHandler(server, *query_extract); - }; - }); -} -} diff --git a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h deleted file mode 100644 index 0d7bc8356bb..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h +++ /dev/null @@ -1,92 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include "ExtractorContextChange.h" -#include "../HTTPHandlerFactory.h" - -#if USE_RE2_ST -#include -#else -#define re2_st re2 -#endif - -namespace DB -{ - -using RegexRule = std::shared_ptr; -using HeadersRegexRule = std::map; -using ExtractRes = std::vector>; - -class ExtractorDynamicQueryParameters -{ -public: - ExtractorDynamicQueryParameters( - Poco::Util::AbstractConfiguration & configuration, const String & key, - const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ - ); - - bool loadSettingsFromPost() const { return false; } - - ExtractRes extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params); - -private: - const RegexRule url_regex; - const HeadersRegexRule headers_regex; - - String dynamic_param_name; - std::map extract_from_url; - std::map> extract_from_headers; -}; - -class ExtractorPredefinedQueryParameters -{ -public: - ExtractorPredefinedQueryParameters( - Poco::Util::AbstractConfiguration & configuration, const String & key, - const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ - ); - - bool loadSettingsFromPost() const { return !has_insert_query; } - - ExtractRes extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params); - -private: - const RegexRule url_regex; - const HeadersRegexRule headers_regex; - - NameSet queries_names; - bool has_insert_query{false}; - ExtractRes predefine_queries; - std::map extract_from_url; - std::map> extract_from_headers; -}; - -class HTTPQueryRequestHandlerMatcherAndCreator -{ -public: - template - static auto invokeWithParsedRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key, const NestedFunction & fun) - { - return fun(configuration.getString(key + ".method", ""), createRegexRule(configuration, key + ".url"), - createHeadersRegexRule(configuration, key + ".headers")); - } - - static HTTPHandlerMatcher createHandlerMatcher(const String & method, const RegexRule & url_rule, const HeadersRegexRule & headers_rule); - -private: - static RegexRule createRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key); - - static HeadersRegexRule createHeadersRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key); -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp deleted file mode 100644 index ea70abbcc6f..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp +++ /dev/null @@ -1,118 +0,0 @@ -#include "HTTPReplicasStatusRequestHandler.h" - -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - - -HTTPReplicasStatusRequestHandler::HTTPReplicasStatusRequestHandler(Context & context_) - : context(context_) -{ -} - -void HTTPReplicasStatusRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) -{ - try - { - HTMLForm params(request); - - /// Even if lag is small, output detailed information about the lag. - bool verbose = params.get("verbose", "") == "1"; - - const MergeTreeSettings & settings = context.getMergeTreeSettings(); - - bool ok = true; - std::stringstream message; - - auto databases = context.getDatabases(); - - /// Iterate through all the replicated tables. - for (const auto & db : databases) - { - /// Lazy database can not contain replicated tables - if (db.second->getEngineName() == "Lazy") - continue; - - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) - { - auto & table = iterator->table(); - StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); - - if (!table_replicated) - continue; - - time_t absolute_delay = 0; - time_t relative_delay = 0; - - table_replicated->getReplicaDelays(absolute_delay, relative_delay); - - if ((settings.min_absolute_delay_to_close && absolute_delay >= static_cast(settings.min_absolute_delay_to_close)) - || (settings.min_relative_delay_to_close && relative_delay >= static_cast(settings.min_relative_delay_to_close))) - ok = false; - - message << backQuoteIfNeed(db.first) << "." << backQuoteIfNeed(iterator->name()) - << ":\tAbsolute delay: " << absolute_delay << ". Relative delay: " << relative_delay << ".\n"; - } - } - - const auto & config = context.getConfigRef(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - if (ok && !verbose) - { - const char * data = "Ok.\n"; - response.sendBuffer(data, strlen(data)); - } - else - { - response.send() << message.rdbuf(); - } - } - catch (...) - { - tryLogCurrentException("HTTPReplicasStatusRequestHandler"); - - try - { - response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); - - if (!response.sent()) - { - /// We have not sent anything yet and we don't even know if we need to compress response. - response.send() << getCurrentExceptionMessage(false) << std::endl; - } - } - catch (...) - { - LOG_ERROR((&Logger::get("HTTPReplicasStatusRequestHandler")), "Cannot send exception to client"); - } - } -} - -HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer & server, const String & key) -{ - const auto & prefix = server.config().getString(key, "/replicas_status"); - - return [&, prefix = prefix](const Poco::Net::HTTPServerRequest & request) - { - return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) && - startsWith(request.getURI(), prefix); - }; -} - -HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer & server, const String &) -{ - return [&]() { return new HTTPReplicasStatusRequestHandler(server.context()); }; -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h deleted file mode 100644 index 277e59eb02c..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include "../HTTPHandlerFactory.h" -#include - - -namespace DB -{ - -class Context; - -/// Replies "Ok.\n" if all replicas on this server don't lag too much. Otherwise output lag information. -class HTTPReplicasStatusRequestHandler : public Poco::Net::HTTPRequestHandler -{ -public: - explicit HTTPReplicasStatusRequestHandler(Context & context_); - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - Context & context; -}; - - -} diff --git a/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp deleted file mode 100644 index 032b51d5b7a..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp +++ /dev/null @@ -1,45 +0,0 @@ -#include "HTTPRootRequestHandler.h" - -#include - -#include - -#include -#include - -namespace DB -{ - -void HTTPRootRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) -{ - try - { - const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - response.setContentType("text/html; charset=UTF-8"); - - const std::string data = config.getString("http_server_default_response", "Ok.\n"); - response.sendBuffer(data.data(), data.size()); - } - catch (...) - { - tryLogCurrentException("HTTPRootRequestHandler"); - } -} - -HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &) -{ - return [&](const Poco::Net::HTTPServerRequest & request) -> bool - { - return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) - && request.getURI() == "/"; - }; -} - -HTTPHandlerCreator createRootHandlerCreator(IServer & server, const String &) -{ - return [&]() { return new HTTPRootRequestHandler(server); }; -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h deleted file mode 100644 index 136f3292385..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h +++ /dev/null @@ -1,26 +0,0 @@ -#pragma once - -#include "../IServer.h" -#include "../HTTPHandlerFactory.h" - -#include - - -namespace DB -{ - -/// Response with custom string. Can be used for browser. -class HTTPRootRequestHandler : public Poco::Net::HTTPRequestHandler -{ -public: - explicit HTTPRootRequestHandler(const IServer & server_) : server(server_) - { - } - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - const IServer & server; -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp b/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp deleted file mode 100644 index 840044634aa..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp +++ /dev/null @@ -1,105 +0,0 @@ -#include "HTTPSessionContextHolder.h" -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int REQUIRED_PASSWORD; - extern const int INVALID_SESSION_TIMEOUT; -} - -static std::chrono::steady_clock::duration parseSessionTimeout( - const Poco::Util::AbstractConfiguration & config, - const HTMLForm & params) -{ - unsigned session_timeout = config.getInt("default_session_timeout", 60); - - if (params.has("session_timeout")) - { - unsigned max_session_timeout = config.getUInt("max_session_timeout", 3600); - std::string session_timeout_str = params.get("session_timeout"); - - ReadBufferFromString buf(session_timeout_str); - if (!tryReadIntText(session_timeout, buf) || !buf.eof()) - throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); - - if (session_timeout > max_session_timeout) - throw Exception("Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) - + ". Maximum session timeout could be modified in configuration file.", - ErrorCodes::INVALID_SESSION_TIMEOUT); - } - - return std::chrono::seconds(session_timeout); -} - -HTTPSessionContextHolder::~HTTPSessionContextHolder() -{ - if (session_context) - session_context->releaseSession(session_id, session_timeout); -} - -void HTTPSessionContextHolder::authentication(Poco::Net::HTTPServerRequest & request, HTMLForm & params) -{ - auto user = request.get("X-ClickHouse-User", ""); - auto password = request.get("X-ClickHouse-Key", ""); - auto quota_key = request.get("X-ClickHouse-Quota", ""); - - if (user.empty() && password.empty() && quota_key.empty()) - { - /// User name and password can be passed using query parameters - /// or using HTTP Basic auth (both methods are insecure). - if (request.hasCredentials()) - { - Poco::Net::HTTPBasicCredentials credentials(request); - - user = credentials.getUsername(); - password = credentials.getPassword(); - } - else - { - user = params.get("user", "default"); - password = params.get("password", ""); - } - - quota_key = params.get("quota_key", ""); - } - else - { - /// It is prohibited to mix different authorization schemes. - if (request.hasCredentials() - || params.has("user") - || params.has("password") - || params.has("quota_key")) - { - throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); - } - } - - std::string query_id = params.get("query_id", ""); - query_context.setUser(user, password, request.clientAddress(), quota_key); - query_context.setCurrentQueryId(query_id); -} - -HTTPSessionContextHolder::HTTPSessionContextHolder(Context & query_context_, Poco::Net::HTTPServerRequest & request, HTMLForm & params) - : query_context(query_context_) -{ - authentication(request, params); - - { - session_id = params.get("session_id", ""); - - if (!session_id.empty()) - { - session_timeout = parseSessionTimeout(query_context.getConfigRef(), params); - session_context = query_context.acquireSession(session_id, session_timeout, params.check("session_check", "1")); - - query_context = *session_context; - query_context.setSessionContext(*session_context); - } - } -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h b/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h deleted file mode 100644 index deadd3910ad..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -/// Manage the lifetime of the session context. -struct HTTPSessionContextHolder -{ - ~HTTPSessionContextHolder(); - - void authentication(Poco::Net::HTTPServerRequest & request, HTMLForm & params); - - HTTPSessionContextHolder(Context & query_context_, Poco::Net::HTTPServerRequest & request, HTMLForm & params); - - String session_id; - Context & query_context; - std::shared_ptr session_context = nullptr; - std::chrono::steady_clock::duration session_timeout; -}; - -} diff --git a/programs/server/NotFoundHandler.cpp b/programs/server/NotFoundHandler.cpp index 26df1a1b7d4..f9b51719b0d 100644 --- a/programs/server/NotFoundHandler.cpp +++ b/programs/server/NotFoundHandler.cpp @@ -17,14 +17,16 @@ void NotFoundHandler::handleRequest( try { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_NOT_FOUND); - - std::stringstream output_description; - output_description << "There is no handle " << request.getURI() << "\n\n"; - - if (!no_handler_description.empty()) - output_description << no_handler_description << "\n"; - - response.send() << output_description.str(); + response.send() << "There is no handle " << request.getURI() << "\n\n" + << "Use / or /ping for health checks.\n" + << "Or /replicas_status for more sophisticated health checks.\n\n" + << "Send queries from your program with POST method or GET /?query=...\n\n" + << "Use clickhouse-client:\n\n" + << "For interactive data analysis:\n" + << " clickhouse-client\n\n" + << "For batch query processing:\n" + << " clickhouse-client --query='SELECT 1' > result\n" + << " clickhouse-client < query > result\n"; } catch (...) { @@ -32,4 +34,4 @@ void NotFoundHandler::handleRequest( } } -} +} \ No newline at end of file diff --git a/programs/server/NotFoundHandler.h b/programs/server/NotFoundHandler.h index caf527726c4..0f76fe9c46e 100644 --- a/programs/server/NotFoundHandler.h +++ b/programs/server/NotFoundHandler.h @@ -10,12 +10,9 @@ namespace DB class NotFoundHandler : public Poco::Net::HTTPRequestHandler { public: - NotFoundHandler(const std::string & no_handler_description_) : no_handler_description(no_handler_description_) {} - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - const std::string no_handler_description; + void handleRequest( + Poco::Net::HTTPServerRequest & request, + Poco::Net::HTTPServerResponse & response) override; }; -} +} \ No newline at end of file diff --git a/programs/server/PingRequestHandler.cpp b/programs/server/PingRequestHandler.cpp deleted file mode 100644 index 141161ef45c..00000000000 --- a/programs/server/PingRequestHandler.cpp +++ /dev/null @@ -1,31 +0,0 @@ -#include "PingRequestHandler.h" - -#include - -#include - -#include -#include - -namespace DB -{ - -void PingRequestHandler::handleRequest( - Poco::Net::HTTPServerRequest &, - Poco::Net::HTTPServerResponse & response) -{ - try - { - const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - const char * data = "Ok.\n"; - response.sendBuffer(data, strlen(data)); - } - catch (...) - { - tryLogCurrentException("PingRequestHandler"); - } -} - -} diff --git a/programs/server/PingRequestHandler.h b/programs/server/PingRequestHandler.h deleted file mode 100644 index 3728fb40adb..00000000000 --- a/programs/server/PingRequestHandler.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include "IServer.h" - -#include - - -namespace DB -{ - -/// Response with "Ok.\n". Used for availability checks. -class PingRequestHandler : public Poco::Net::HTTPRequestHandler -{ -private: - IServer & server; - -public: - explicit PingRequestHandler(IServer & server_) : server(server_) - { - } - - void handleRequest( - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response) override; -}; - -} diff --git a/programs/server/ReplicasStatusHandler.h b/programs/server/ReplicasStatusHandler.h index 479b013cfe8..a32f1ba905f 100644 --- a/programs/server/ReplicasStatusHandler.h +++ b/programs/server/ReplicasStatusHandler.h @@ -17,7 +17,7 @@ private: Context & context; public: - explicit ReplicasStatusHandler(IServer & server); + explicit ReplicasStatusHandler(IServer & server_); void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; }; diff --git a/programs/server/RootRequestHandler.cpp b/programs/server/RootRequestHandler.cpp deleted file mode 100644 index 86b57b63208..00000000000 --- a/programs/server/RootRequestHandler.cpp +++ /dev/null @@ -1,33 +0,0 @@ -#include "RootRequestHandler.h" - -#include - -#include - -#include -#include - -namespace DB -{ - -void RootRequestHandler::handleRequest( - Poco::Net::HTTPServerRequest &, - Poco::Net::HTTPServerResponse & response) -{ - try - { - const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - response.setContentType("text/html; charset=UTF-8"); - - const std::string data = config.getString("http_server_default_response", "Ok.\n"); - response.sendBuffer(data.data(), data.size()); - } - catch (...) - { - tryLogCurrentException("RootRequestHandler"); - } -} - -} diff --git a/programs/server/RootRequestHandler.h b/programs/server/RootRequestHandler.h deleted file mode 100644 index 1b6d53eeeda..00000000000 --- a/programs/server/RootRequestHandler.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include "IServer.h" - -#include - - -namespace DB -{ - -/// Response with custom string. Can be used for browser. -class RootRequestHandler : public Poco::Net::HTTPRequestHandler -{ -private: - IServer & server; - -public: - explicit RootRequestHandler(IServer & server_) : server(server_) - { - } - - void handleRequest( - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response) override; -}; - -} diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3490ff6a445..96a14dd3d87 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -767,9 +767,9 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, port); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); - auto handler_factory = createDefaultHandlerFatory(*this, "HTTPHandler-factory"); - if (config().has("prometheus") && config().getInt("prometheus.port", 0) == 0) - handler_factory->addHandler(async_metrics); + auto handler_factory = createHandlerFactory(*this, "HTTPHandler-factory"); +// if (config().has("prometheus") && config().getInt("prometheus.port", 0) == 0) +// handler_factory->addHandler(async_metrics); servers.emplace_back(std::make_unique( handler_factory, @@ -789,7 +789,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createDefaultHandlerFatory(*this, "HTTPSHandler-factory"), + createHandlerFactory(*this, "HTTPSHandler-factory"), server_pool, socket, http_params)); @@ -847,7 +847,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createDefaultHandlerFatory(*this, "InterserverIOHTTPHandler-factory"), + createHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params)); @@ -863,7 +863,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createDefaultHandlerFatory(*this, "InterserverIOHTTPHandler-factory"), + createHandlerFactory(*this, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params)); @@ -892,22 +892,22 @@ int Server::main(const std::vector & /*args*/) }); /// Prometheus (if defined and not setup yet with http_port) - create_server("prometheus.port", [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - auto handler_factory = new HTTPRequestHandlerFactoryMain(*this, "PrometheusHandler-factory"); - handler_factory->addHandler(async_metrics); - servers.emplace_back(std::make_unique( - handler_factory, - server_pool, - socket, - http_params)); - - LOG_INFO(log, "Listening for Prometheus: http://" + address.toString()); - }); +// create_server("prometheus.port", [&](UInt16 port) +// { +// Poco::Net::ServerSocket socket; +// auto address = socket_bind_listen(socket, listen_host, port); +// socket.setReceiveTimeout(settings.http_receive_timeout); +// socket.setSendTimeout(settings.http_send_timeout); +// auto handler_factory = new HTTPRequestHandlerFactoryMain(*this, "PrometheusHandler-factory"); +// handler_factory->addHandler(async_metrics); +// servers.emplace_back(std::make_unique( +// handler_factory, +// server_pool, +// socket, +// http_params)); +// +// LOG_INFO(log, "Listening for Prometheus: http://" + address.toString()); +// }); } if (servers.empty()) diff --git a/programs/server/StaticRequestHandler.cpp b/programs/server/StaticRequestHandler.cpp new file mode 100644 index 00000000000..730060dea2e --- /dev/null +++ b/programs/server/StaticRequestHandler.cpp @@ -0,0 +1,86 @@ +#include "StaticRequestHandler.h" + +#include "HTTPHandlerFactory.h" +#include "HTTPHandlerRequestFilter.h" + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_FILE_NAME; + extern const int INVALID_CONFIG_PARAMETER; +} + +void StaticRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) +{ + try + { + setResponseDefaultHeaders(response, server.config().getUInt("keep_alive_timeout", 10)); + + response.setContentType(content_type); + response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status)); + + response.sendBuffer(response_content.data(), response_content.size()); + } + catch (...) + { + tryLogCurrentException("StaticRequestHandler"); + } +} + +StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & expression, int status_, const String & content_type_) + : server(server_), status(status_), content_type(content_type_) +{ + static const String file_prefix = "file://"; + static const String config_prefix = "config://"; + + if (startsWith(expression, file_prefix)) + { + std::string config_dir = Poco::Path(server.context().getPath()).parent().toString(); + const std::string & file_path = config_dir + expression.substr(file_prefix.size(), expression.size() - file_prefix.size()); + + if (!Poco::File(file_path).exists()) + throw Exception("Invalid file name for static HTTPHandler." + file_path, ErrorCodes::INCORRECT_FILE_NAME); + + WriteBufferFromOwnString out; + ReadBufferFromFile in(file_path); + copyData(in, out); + response_content = out.str(); + } + else if (startsWith(expression, config_prefix)) + { + if (expression.size() <= config_prefix.size()) + throw Exception("Static routing rule handler must contain a complete configuration path, for example: config://config_key", + ErrorCodes::INVALID_CONFIG_PARAMETER); + + response_content = server.config().getString(expression.substr(config_prefix.size(), expression.size() - config_prefix.size()), "Ok.\n"); + } + else + response_content = expression; +} + +Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & server, const std::string & config_prefix) +{ + const auto & status = server.config().getInt(config_prefix + ".handler.status", 200); + const auto & response_content = server.config().getRawString(config_prefix + ".handler.response_content", "Ok.\n"); + const auto & response_content_type = server.config().getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8"); + + return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( + server, response_content, status, response_content_type), server.config(), config_prefix); +} + +} diff --git a/programs/server/StaticRequestHandler.h b/programs/server/StaticRequestHandler.h new file mode 100644 index 00000000000..bfe7f7a3e7b --- /dev/null +++ b/programs/server/StaticRequestHandler.h @@ -0,0 +1,28 @@ +#pragma once + +#include "IServer.h" + +#include +#include + + +namespace DB +{ + +/// Response with custom string. Can be used for browser. +class StaticRequestHandler : public Poco::Net::HTTPRequestHandler +{ +private: + IServer & server; + + int status; + String content_type; + String response_content; + +public: + StaticRequestHandler(IServer & server, const String & expression, int status_ = 200, const String & content_type_ = "text/html; charset=UTF-8"); + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; +}; + +} diff --git a/src/Common/HTMLForm.h b/src/Common/HTMLForm.h index fa6f31bf5d2..2490d613160 100644 --- a/src/Common/HTMLForm.h +++ b/src/Common/HTMLForm.h @@ -26,12 +26,6 @@ struct HTMLForm : public Poco::Net::HTMLForm readUrl(istr); } - template - bool check(const std::string & key, T check_value) - { - const auto & value = getParsed(key, T()); - return value == check_value; - } template T getParsed(const std::string & key, T default_value) diff --git a/dbms/Interpreters/QueryParameterVisitor.h b/src/Interpreters/QueryParameterVisitor.h similarity index 59% rename from dbms/Interpreters/QueryParameterVisitor.h rename to src/Interpreters/QueryParameterVisitor.h index d765aa00bea..d3e618058c0 100644 --- a/dbms/Interpreters/QueryParameterVisitor.h +++ b/src/Interpreters/QueryParameterVisitor.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { @@ -32,4 +34,16 @@ private: } }; +NameSet analyzeReceiveQueryParams(const std::string & query) +{ + NameSet query_params; + const char * query_begin = query.data(); + const char * query_end = query.data() + query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "analyzeReceiveQueryParams", 0, 0); + QueryParameterVisitor(query_params).visit(extract_query_ast); + return query_params; +} + } From 0cebfc74b6c2a41a5788e28dae3494cfbbe016dc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Apr 2020 22:22:03 +0300 Subject: [PATCH 413/743] Added a test --- .../01249_flush_interactive.reference | 12 ++++++++++++ .../0_stateless/01249_flush_interactive.sh | 16 ++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01249_flush_interactive.reference create mode 100755 tests/queries/0_stateless/01249_flush_interactive.sh diff --git a/tests/queries/0_stateless/01249_flush_interactive.reference b/tests/queries/0_stateless/01249_flush_interactive.reference new file mode 100644 index 00000000000..6d6abb2d37f --- /dev/null +++ b/tests/queries/0_stateless/01249_flush_interactive.reference @@ -0,0 +1,12 @@ +0 +1 +2 +3 +4 +--- +0 +1 +2 +3 +4 +--- diff --git a/tests/queries/0_stateless/01249_flush_interactive.sh b/tests/queries/0_stateless/01249_flush_interactive.sh new file mode 100755 index 00000000000..a489806ceb9 --- /dev/null +++ b/tests/queries/0_stateless/01249_flush_interactive.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +# A question for curious reader: +# How to break shell pipeline as soon as 5 lines are found in the following command: +# ./my-program | head -n5 +# When I tried to do it, pipeline was not actively terminated, +# unless the my-program will try to output a thousand more lines overflowing pipe buffer and terminating with Broken Pipe. +# But if my program just output 5 (or slightly more) lines and hang up, the pipeline is not terminated. + +timeout 1 ${CLICKHOUSE_LOCAL} --max_execution_time 10 --query "SELECT DISTINCT number % 5 FROM system.numbers" ||: +echo '---' +timeout 1 ${CLICKHOUSE_CURL} -sS --no-buffer "${CLICKHOUSE_URL}&max_execution_time=10" --data-binary "SELECT DISTINCT number % 5 FROM system.numbers" ||: +echo '---' From ca6bea373105c5f3b6dbc2a3caef83f69aaba682 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Apr 2020 22:53:12 +0300 Subject: [PATCH 414/743] Added a patch from Alexander Burmak --- src/Interpreters/SystemLog.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 9b0c273674c..9b6a23ecff6 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -189,7 +189,8 @@ SystemLog::SystemLog(Context & context_, const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_) + size_t flush_interval_milliseconds_, + bool lazy_load) : context(context_) , table_id(database_name_, table_name_) , storage_def(storage_def_), @@ -198,6 +199,11 @@ SystemLog::SystemLog(Context & context_, assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); log = &Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); + if (!lazy_load) + { + prepareTable(); + } + saving_thread = ThreadFromGlobalPool([this] { savingThreadFunction(); }); } From 99667a38c3509d0490380a6e00ccf53bd720cbb6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Apr 2020 22:56:34 +0300 Subject: [PATCH 415/743] Added a test from Svyatoslav Tkhon Il Pak @DeifyTheGod #9642 --- .../configs/config_non_lazy.xml | 4 ++++ .../test_system_tables_non_lazy_load.py | 24 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/integration/test_system_tables_non_lazy_load/configs/config_non_lazy.xml create mode 100644 tests/integration/test_system_tables_non_lazy_load/test_system_tables_non_lazy_load.py diff --git a/tests/integration/test_system_tables_non_lazy_load/configs/config_non_lazy.xml b/tests/integration/test_system_tables_non_lazy_load/configs/config_non_lazy.xml new file mode 100644 index 00000000000..2dfee707a7c --- /dev/null +++ b/tests/integration/test_system_tables_non_lazy_load/configs/config_non_lazy.xml @@ -0,0 +1,4 @@ + + + false + diff --git a/tests/integration/test_system_tables_non_lazy_load/test_system_tables_non_lazy_load.py b/tests/integration/test_system_tables_non_lazy_load/test_system_tables_non_lazy_load.py new file mode 100644 index 00000000000..18841b4c585 --- /dev/null +++ b/tests/integration/test_system_tables_non_lazy_load/test_system_tables_non_lazy_load.py @@ -0,0 +1,24 @@ +import time +import pytest +import os + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', config_dir="configs") + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_system_tables_non_lazy_load(start_cluster): + assert node1.query_and_get_error("SELECT * FROM system.part_log") == "" + assert node1.query_and_get_error("SELECT * FROM system.query_log") == "" + assert node1.query_and_get_error("SELECT * FROM system.query_thread_log") == "" + assert node1.query_and_get_error("SELECT * FROM system.text_log") == "" + assert node1.query_and_get_error("SELECT * FROM system.trace_log") == "" + assert node1.query_and_get_error("SELECT * FROM system.metric_log") == "" From 8d3e6ed5e349bbd14dce8b7b8d783efad181706a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Apr 2020 23:04:59 +0300 Subject: [PATCH 416/743] Added missing code --- src/Interpreters/SystemLog.cpp | 19 +++++++++++-------- src/Interpreters/SystemLog.h | 3 ++- src/Interpreters/TextLog.cpp | 4 ++-- src/Interpreters/TextLog.h | 3 ++- 4 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index fc0f2f98125..3740ac3f4c8 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -30,7 +30,8 @@ std::shared_ptr createSystemLog( const String & default_database_name, const String & default_table_name, const Poco::Util::AbstractConfiguration & config, - const String & config_prefix) + const String & config_prefix, + bool lazy_load) { if (!config.has(config_prefix)) return {}; @@ -63,7 +64,7 @@ std::shared_ptr createSystemLog( size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); - return std::make_shared(context, database, table, engine, flush_interval_milliseconds); + return std::make_shared(context, database, table, engine, flush_interval_milliseconds, lazy_load); } } @@ -71,12 +72,14 @@ std::shared_ptr createSystemLog( SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfiguration & config) { - query_log = createSystemLog(global_context, "system", "query_log", config, "query_log"); - query_thread_log = createSystemLog(global_context, "system", "query_thread_log", config, "query_thread_log"); - part_log = createSystemLog(global_context, "system", "part_log", config, "part_log"); - trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log"); - text_log = createSystemLog(global_context, "system", "text_log", config, "text_log"); - metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log"); + bool lazy_load = config.getBool("system_tables_lazy_load", true); + + query_log = createSystemLog(global_context, "system", "query_log", config, "query_log", lazy_load); + query_thread_log = createSystemLog(global_context, "system", "query_thread_log", config, "query_thread_log", lazy_load); + part_log = createSystemLog(global_context, "system", "part_log", config, "part_log", lazy_load); + trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log", lazy_load); + text_log = createSystemLog(global_context, "system", "text_log", config, "text_log", lazy_load); + metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log", lazy_load); if (metric_log) { diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 9b6a23ecff6..ceffba67b50 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -117,7 +117,8 @@ public: const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_); + size_t flush_interval_milliseconds_, + bool lazy_load); /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index b5f1d987b91..8ecf6f029d2 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -72,9 +72,9 @@ void TextLogElement::appendToBlock(Block & block) const TextLog::TextLog(Context & context_, const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_) + size_t flush_interval_milliseconds_, bool lazy_load) : SystemLog(context_, database_name_, table_name_, - storage_def_, flush_interval_milliseconds_) + storage_def_, flush_interval_milliseconds_, lazy_load) { // SystemLog methods may write text logs, so we disable logging for the text // log table to avoid recursion. diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 73c38429662..aaf64974e05 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -36,7 +36,8 @@ public: const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_); + size_t flush_interval_milliseconds_, + bool lazy_load); }; } From 71c441887671025e7f9f072c6c5123cb9f722ea2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Apr 2020 23:49:13 +0300 Subject: [PATCH 417/743] More memory statistics --- src/Common/MemoryStatisticsOS.cpp | 90 ++++++++++++++++++++++++ src/Common/MemoryStatisticsOS.h | 38 ++++++++++ src/Interpreters/AsynchronousMetrics.cpp | 18 ++--- src/Interpreters/AsynchronousMetrics.h | 2 + 4 files changed, 136 insertions(+), 12 deletions(-) create mode 100644 src/Common/MemoryStatisticsOS.cpp create mode 100644 src/Common/MemoryStatisticsOS.h diff --git a/src/Common/MemoryStatisticsOS.cpp b/src/Common/MemoryStatisticsOS.cpp new file mode 100644 index 00000000000..adc5bf5d904 --- /dev/null +++ b/src/Common/MemoryStatisticsOS.cpp @@ -0,0 +1,90 @@ +#include +#include +#include +#include +#include + +#include "MemoryStatisticsOS.h" + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; +} + +static constexpr auto filename = "/proc/self/statm"; +static constexpr size_t PAGE_SIZE = 4096; + +MemoryStatisticsOS::MemoryStatisticsOS() +{ + fd = ::open(filename, O_RDONLY | O_CLOEXEC); + + if (-1 == fd) + throwFromErrno("Cannot open file " + std::string(filename), errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); +} + +MemoryStatisticsOS::~MemoryStatisticsOS() +{ + if (0 != ::close(fd)) + tryLogCurrentException(__PRETTY_FUNCTION__); +} + +MemoryStatisticsOS::Data MemoryStatisticsOS::get() const +{ + Data data; + + constexpr size_t buf_size = 1024; + char buf[buf_size]; + + ssize_t res = 0; + + do + { + res = ::pread(fd, buf, buf_size, 0); + + if (-1 == res) + { + if (errno == EINTR) + continue; + + throwFromErrno("Cannot read from file " + std::string(filename), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); + } + + assert(res >= 0); + break; + } while (true); + + ReadBufferFromMemory in(buf, res); + + uint64_t unused; + readIntText(data.virt, in); + skipWhitespaceIfAny(in); + readIntText(data.resident, in); + skipWhitespaceIfAny(in); + readIntText(data.shared, in); + skipWhitespaceIfAny(in); + readIntText(data.code, in); + skipWhitespaceIfAny(in); + readIntText(unused, in); + skipWhitespaceIfAny(in); + readIntText(data.data_and_stack, in); + + data.virt *= PAGE_SIZE; + data.resident *= PAGE_SIZE; + data.shared *= PAGE_SIZE; + data.code *= PAGE_SIZE; + data.data_and_stack *= PAGE_SIZE; + + return data; +} + +} diff --git a/src/Common/MemoryStatisticsOS.h b/src/Common/MemoryStatisticsOS.h new file mode 100644 index 00000000000..4601675d115 --- /dev/null +++ b/src/Common/MemoryStatisticsOS.h @@ -0,0 +1,38 @@ +#pragma once +#include + + +namespace DB +{ + +/** Opens a file /proc/self/mstat. Keeps it open and reads memory statistics via 'pread'. + * This is Linux specific. + * See: man procfs + * + * Note: a class is used instead of a single function to avoid excessive file open/close on every use. + * pread is used to avoid lseek. + */ +class MemoryStatisticsOS +{ +public: + /// In number of bytes. + struct Data + { + uint64_t virt; + uint64_t resident; + uint64_t shared; + uint64_t code; + uint64_t data_and_stack; + }; + + MemoryStatisticsOS(); + ~MemoryStatisticsOS(); + + /// Thread-safe. + Data get() const; + +private: + int fd; +}; + +} diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 5aa59bd901a..b635c726b4c 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -9,8 +9,6 @@ #include #include #include -#include -#include #include #include @@ -136,17 +134,13 @@ void AsynchronousMetrics::update() /// Process memory usage according to OS #if defined(OS_LINUX) { - char buf[1024]; - ReadBufferFromFile in("/proc/self/statm", 1024, -1, buf); - size_t memory_virtual = 0; - size_t memory_resident = 0; - readIntText(memory_virtual, in); - skipWhitespaceIfAny(in); - readIntText(memory_resident, in); + MemoryStatisticsOS::Data data = memory_stat.get(); - static constexpr size_t PAGE_SIZE = 4096; - set("MemoryVirtual", memory_virtual * PAGE_SIZE); - set("MemoryResident", memory_resident * PAGE_SIZE); + set("MemoryVirtual", data.virt); + set("MemoryResident", data.resident); + set("MemoryShared", data.shared); + set("MemoryCode", data.code); + set("MemoryDataAndStack", data.data_and_stack); } #endif diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 8ccefb9e930..b5e1765cd25 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -44,6 +45,7 @@ private: Container container; mutable std::mutex container_mutex; + MemoryStatisticsOS memory_stat; ThreadFromGlobalPool thread; void run(); From 1c00953b3e187aca95cee956c190d2436ee37479 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Apr 2020 23:57:34 +0300 Subject: [PATCH 418/743] Added test tool --- src/Common/MemoryStatisticsOS.h | 2 ++ src/Common/tests/CMakeLists.txt | 3 +++ .../tests/memory_statistics_os_perf.cpp | 23 +++++++++++++++++++ 3 files changed, 28 insertions(+) create mode 100644 src/Common/tests/memory_statistics_os_perf.cpp diff --git a/src/Common/MemoryStatisticsOS.h b/src/Common/MemoryStatisticsOS.h index 4601675d115..97caf4e8fbe 100644 --- a/src/Common/MemoryStatisticsOS.h +++ b/src/Common/MemoryStatisticsOS.h @@ -11,6 +11,8 @@ namespace DB * * Note: a class is used instead of a single function to avoid excessive file open/close on every use. * pread is used to avoid lseek. + * + * Actual performance is from 1 to 5 million iterations per second. */ class MemoryStatisticsOS { diff --git a/src/Common/tests/CMakeLists.txt b/src/Common/tests/CMakeLists.txt index 4ed21ba4ef2..2ddbee43f5f 100644 --- a/src/Common/tests/CMakeLists.txt +++ b/src/Common/tests/CMakeLists.txt @@ -68,3 +68,6 @@ target_link_libraries (symbol_index PRIVATE clickhouse_common_io) add_executable (chaos_sanitizer chaos_sanitizer.cpp) target_link_libraries (chaos_sanitizer PRIVATE clickhouse_common_io) + +add_executable (memory_statistics_os_perf memory_statistics_os_perf.cpp) +target_link_libraries (memory_statistics_os_perf PRIVATE clickhouse_common_io) diff --git a/src/Common/tests/memory_statistics_os_perf.cpp b/src/Common/tests/memory_statistics_os_perf.cpp new file mode 100644 index 00000000000..dacd959fe1d --- /dev/null +++ b/src/Common/tests/memory_statistics_os_perf.cpp @@ -0,0 +1,23 @@ +#include +#include + + +int main(int argc, char ** argv) +{ + using namespace DB; + + size_t num_iterations = argc >= 2 ? std::stoull(argv[1]) : 1000000; + MemoryStatisticsOS stats; + + uint64_t counter = 0; + for (size_t i = 0; i < num_iterations; ++i) + { + MemoryStatisticsOS::Data data = stats.get(); + counter += data.resident; + } + + std::cerr << (counter / num_iterations) << '\n'; + return 0; +} + + From e44d5c5749fe5e90236611d9d6b60f1300affed0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Apr 2020 01:11:36 +0300 Subject: [PATCH 419/743] Fix clang readability-container-size-empty warning in StorageDistributed::canForceGroupByNoMerge() --- src/Storages/StorageDistributed.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 22f51e7a5b3..7cca0d94bef 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -430,7 +430,7 @@ bool StorageDistributed::canForceGroupByNoMerge(const Context &context, QueryPro // injective functions are optimized out in optimizeGroupBy() // hence all we need to check is that column in GROUP BY matches sharding expression auto & group_exprs = group_by->children; - if (!group_exprs.size()) + if (group_exprs.empty()) throw Exception("No ASTExpressionList in GROUP BY", ErrorCodes::LOGICAL_ERROR); auto id = group_exprs[0]->as(); From ec6b176f3b2092303aae706ed114605b18818d66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 00:43:06 +0300 Subject: [PATCH 420/743] Rework total memory tracker #10293 --- programs/server/Server.cpp | 6 +++ src/Common/CurrentMetrics.cpp | 2 +- src/Common/MemoryTracker.cpp | 48 ++++++++++++++---------- src/Common/MemoryTracker.h | 13 ++++++- src/Interpreters/AsynchronousMetrics.cpp | 5 +++ src/Interpreters/ProcessList.cpp | 26 +------------ src/Interpreters/ProcessList.h | 3 -- 7 files changed, 52 insertions(+), 51 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3490ff6a445..a5ccd648551 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -81,6 +81,7 @@ namespace CurrentMetrics { extern const Metric Revision; extern const Metric VersionInteger; + extern const Metric MemoryTracking; } namespace @@ -555,6 +556,11 @@ int Server::main(const std::vector & /*args*/) global_context->setFormatSchemaPath(format_schema_path.path()); format_schema_path.createDirectories(); + /// Limit on total memory usage + total_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_all_queries); + total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + LOG_INFO(log, "Loading metadata from " + path); try diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 2a566e3efff..01fb484b449 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -30,7 +30,7 @@ M(QueryThread, "Number of query processing threads") \ M(ReadonlyReplica, "Number of Replicated tables that are currently in readonly state due to re-initialization after ZooKeeper session loss or due to startup without ZooKeeper configured.") \ M(LeaderReplica, "Number of Replicated tables that are leaders. Leader replica is responsible for assigning merges, cleaning old blocks for deduplications and a few more bookkeeping tasks. There may be no more than one leader across all replicas at one moment of time. If there is no leader it will be elected soon or it indicate an issue.") \ - M(MemoryTracking, "Total amount of memory (bytes) allocated in currently executing queries. Note that some memory allocations may not be accounted.") \ + M(MemoryTracking, "Total amount of memory (bytes) allocated by the server.") \ M(MemoryTrackingInBackgroundProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for backround merges, mutations and fetches). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \ M(MemoryTrackingInBackgroundMoveProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for backround moves). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \ M(MemoryTrackingInBackgroundSchedulePool, "Total amount of memory (bytes) allocated in background schedule pool (that is dedicated for bookkeeping tasks of Replicated tables).") \ diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index efc1ff7785e..a6c2eacef57 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -25,10 +25,16 @@ static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; /// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters. static constexpr Int64 untracked_memory_limit = 4 * 1024 * 1024; +MemoryTracker total_memory_tracker(nullptr, VariableContext::Global); + + +MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {} +MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {} + MemoryTracker::~MemoryTracker() { - if (static_cast(level) < static_cast(VariableContext::Process) && peak) + if ((level == VariableContext::Process || level == VariableContext::User) && peak) { try { @@ -39,19 +45,6 @@ MemoryTracker::~MemoryTracker() /// Exception in Logger, intentionally swallow. } } - - /** This is needed for next memory tracker to be consistent with sum of all referring memory trackers. - * - * Sometimes, memory tracker could be destroyed before memory was freed, and on destruction, amount > 0. - * For example, a query could allocate some data and leave it in cache. - * - * If memory will be freed outside of context of this memory tracker, - * but in context of one of the 'next' memory trackers, - * then memory usage of 'next' memory trackers will be underestimated, - * because amount will be decreased twice (first - here, second - when real 'free' happens). - */ - if (auto value = amount.load(std::memory_order_relaxed)) - free(value); } @@ -62,10 +55,11 @@ void MemoryTracker::logPeakMemoryUsage() const << ": " << formatReadableSizeWithBinarySuffix(peak) << "."); } -static void logMemoryUsage(Int64 amount) +void MemoryTracker::logMemoryUsage(Int64 current) const { LOG_DEBUG(&Logger::get("MemoryTracker"), - "Current memory usage: " << formatReadableSizeWithBinarySuffix(amount) << "."); + "Current memory usage" << (description ? " " + std::string(description) : "") + << ": " << formatReadableSizeWithBinarySuffix(current) << "."); } @@ -131,17 +125,24 @@ void MemoryTracker::alloc(Int64 size) throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED); } + updatePeak(will_be); + + if (auto loaded_next = parent.load(std::memory_order_relaxed)) + loaded_next->alloc(size); +} + + +void MemoryTracker::updatePeak(Int64 will_be) +{ auto peak_old = peak.load(std::memory_order_relaxed); if (will_be > peak_old) /// Races doesn't matter. Could rewrite with CAS, but not worth. { peak.store(will_be, std::memory_order_relaxed); - if (level == VariableContext::Process && will_be / log_peak_memory_usage_every > peak_old / log_peak_memory_usage_every) + if ((level == VariableContext::Process || level == VariableContext::Global) + && will_be / log_peak_memory_usage_every > peak_old / log_peak_memory_usage_every) logMemoryUsage(will_be); } - - if (auto loaded_next = parent.load(std::memory_order_relaxed)) - loaded_next->alloc(size); } @@ -198,6 +199,13 @@ void MemoryTracker::reset() } +void MemoryTracker::set(Int64 to) +{ + amount.store(to, std::memory_order_relaxed); + updatePeak(to); +} + + void MemoryTracker::setOrRaiseHardLimit(Int64 value) { /// This is just atomic set to maximum. diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 98f416cec40..1cd85d0a8b2 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -13,6 +13,7 @@ */ class MemoryTracker { +private: std::atomic amount {0}; std::atomic peak {0}; std::atomic hard_limit {0}; @@ -33,9 +34,12 @@ class MemoryTracker /// This description will be used as prefix into log messages (if isn't nullptr) const char * description = nullptr; + void updatePeak(Int64 will_be); + void logMemoryUsage(Int64 current) const; + public: - MemoryTracker(VariableContext level_ = VariableContext::Thread) : level(level_) {} - MemoryTracker(MemoryTracker * parent_, VariableContext level_ = VariableContext::Thread) : parent(parent_), level(level_) {} + MemoryTracker(VariableContext level_ = VariableContext::Thread); + MemoryTracker(MemoryTracker * parent_, VariableContext level_ = VariableContext::Thread); ~MemoryTracker(); @@ -113,6 +117,9 @@ public: /// Reset the accumulated data and the parent. void reset(); + /// Reset current counter to a new value. + void set(Int64 to); + /// Prints info about peak memory consumption into log. void logPeakMemoryUsage() const; @@ -120,6 +127,8 @@ public: DB::SimpleActionBlocker blocker; }; +extern MemoryTracker total_memory_tracker; + /// Convenience methods, that use current thread's memory_tracker if it is available. namespace CurrentMemoryTracker diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index b635c726b4c..88d5dd0562a 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -141,6 +141,11 @@ void AsynchronousMetrics::update() set("MemoryShared", data.shared); set("MemoryCode", data.code); set("MemoryDataAndStack", data.data_and_stack); + + /// We must update the value of total_memory_tracker periodically. + /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. + /// See https://github.com/ClickHouse/ClickHouse/issues/10293 + total_memory_tracker.set(data.resident); } #endif diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index cf8001ab2c4..5ec04d0e11c 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -14,12 +14,6 @@ #include -namespace CurrentMetrics -{ - extern const Metric MemoryTracking; -} - - namespace DB { @@ -68,7 +62,6 @@ static bool isUnlimitedQuery(const IAST * ast) ProcessList::ProcessList(size_t max_size_) : max_size(max_size_) { - total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); } @@ -171,19 +164,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as process_it->setUserProcessList(&user_process_list); - /// Limits are only raised (to be more relaxed) or set to something instead of zero, - /// because settings for different queries will interfere each other: - /// setting from one query effectively sets values for all other queries. - - /// Track memory usage for all simultaneously running queries. - /// You should specify this value in configuration for default profile, - /// not for specific users, sessions or queries, - /// because this setting is effectively global. - total_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_all_queries); - total_memory_tracker.setDescription("(total)"); - /// Track memory usage for all simultaneously running queries from single user. - user_process_list.user_memory_tracker.setParent(&total_memory_tracker); user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user); user_process_list.user_memory_tracker.setDescription("(for user)"); @@ -280,14 +261,9 @@ ProcessListEntry::~ProcessListEntry() if (user_process_list.queries.empty()) user_process_list.resetTrackers(); - /// This removes memory_tracker for all requests. At this time, no other memory_trackers live. + /// Reset throttler, similarly (see above). if (parent.processes.empty()) - { - /// Reset MemoryTracker, similarly (see above). - parent.total_memory_tracker.logPeakMemoryUsage(); - parent.total_memory_tracker.reset(); parent.total_network_throttler.reset(); - } } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index da6799c5eca..15487b8ec0e 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -295,9 +295,6 @@ protected: /// Stores info about queries grouped by their priority QueryPriorities priorities; - /// Limit and counter for memory of all simultaneously running queries. - MemoryTracker total_memory_tracker{VariableContext::Global}; - /// Limit network bandwidth for all users ThrottlerPtr total_network_throttler; From 2987e70aabb459ef45922f7c9081a43d322c9db1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 01:08:09 +0300 Subject: [PATCH 421/743] Introduce "max_server_memory_usage" setting --- programs/server/Server.cpp | 19 ++++++++++++++++++- src/Core/Settings.h | 4 +++- .../ClusterProxy/executeQuery.cpp | 3 --- 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a5ccd648551..9474ca4a29e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -557,7 +557,24 @@ int Server::main(const std::vector & /*args*/) format_schema_path.createDirectories(); /// Limit on total memory usage - total_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_all_queries); + size_t max_server_memory_usage = settings.max_server_memory_usage; + + double max_server_memory_usage_to_ram_ratio = config().getDouble("max_server_memory_usage_to_ram_ratio", 0.9); + size_t default_max_server_memory_usage = memory_amount * max_server_memory_usage_to_ram_ratio; + + if (max_server_memory_usage == 0) + { + max_server_memory_usage = default_max_server_memory_usage; + LOG_INFO(log, "Setting max_server_memory_usage was set to " << formatReadableSizeWithBinarySuffix(max_server_memory_usage)); + } + else if (max_server_memory_usage > default_max_server_memory_usage) + { + max_server_memory_usage = default_max_server_memory_usage; + LOG_INFO(log, "Setting max_server_memory_usage was lowered to " << formatReadableSizeWithBinarySuffix(max_server_memory_usage) + << " because the system has low amount of memory"); + } + + total_memory_tracker.setOrRaiseHardLimit(max_server_memory_usage); total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cffdd4a66e4..c16d19b0364 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -339,7 +339,7 @@ struct Settings : public SettingsCollection \ M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \ M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ - M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Maximum memory usage for processing all concurrently running queries on the server. Zero means unlimited.", 0) \ + M(SettingUInt64, max_server_memory_usage, 0, "Maximum memory usage for server. It can be specified only for default profile.", 0) \ M(SettingUInt64, memory_profiler_step, 0, "Every number of bytes the memory profiler will collect the allocating stack trace. The minimal effective step is 4 MiB (less values will work as clamped to 4 MiB). Zero means disabled memory profiler.", 0) \ \ M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \ @@ -423,6 +423,8 @@ struct Settings : public SettingsCollection M(SettingBool, merge_tree_uniform_read_distribution, true, "Obsolete setting, does nothing. Will be removed after 2020-05-20", 0) \ M(SettingUInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ M(SettingBool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \ + M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ + DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 07802f468c9..16bc6a6962f 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -23,13 +23,10 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin /// Does not matter on remote servers, because queries are sent under different user. new_settings.max_concurrent_queries_for_user = 0; new_settings.max_memory_usage_for_user = 0; - /// This setting is really not for user and should not be sent to remote server. - new_settings.max_memory_usage_for_all_queries = 0; /// Set as unchanged to avoid sending to remote server. new_settings.max_concurrent_queries_for_user.changed = false; new_settings.max_memory_usage_for_user.changed = false; - new_settings.max_memory_usage_for_all_queries.changed = false; if (settings.force_optimize_skip_unused_shards_no_nested) { From 3755a5ba54a899ce68bb756865b117296d40324e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 01:58:59 +0300 Subject: [PATCH 422/743] Added suggestion from Azat --- 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 c16d19b0364..97b42e2df9c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -339,7 +339,7 @@ struct Settings : public SettingsCollection \ M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \ M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ - M(SettingUInt64, max_server_memory_usage, 0, "Maximum memory usage for server. It can be specified only for default profile.", 0) \ + M(SettingUInt64, max_server_memory_usage, 0, "Maximum memory usage for server. Only has meaning at server startup. It can be specified only for default profile.", 0) \ M(SettingUInt64, memory_profiler_step, 0, "Every number of bytes the memory profiler will collect the allocating stack trace. The minimal effective step is 4 MiB (less values will work as clamped to 4 MiB). Zero means disabled memory profiler.", 0) \ \ M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \ From 237db2668874014e960bdacc4b7917cdf93869e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 02:11:41 +0300 Subject: [PATCH 423/743] Less wrong --- src/Interpreters/SystemLog.cpp | 27 ++++++++++++++++----------- src/Interpreters/SystemLog.h | 21 ++++++++++++--------- src/Interpreters/TextLog.cpp | 4 ++-- src/Interpreters/TextLog.h | 3 +-- 4 files changed, 31 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3740ac3f4c8..cfb31ab2a41 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -30,8 +30,7 @@ std::shared_ptr createSystemLog( const String & default_database_name, const String & default_table_name, const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - bool lazy_load) + const String & config_prefix) { if (!config.has(config_prefix)) return {}; @@ -64,7 +63,7 @@ std::shared_ptr createSystemLog( size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); - return std::make_shared(context, database, table, engine, flush_interval_milliseconds, lazy_load); + return std::make_shared(context, database, table, engine, flush_interval_milliseconds); } } @@ -72,14 +71,12 @@ std::shared_ptr createSystemLog( SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfiguration & config) { - bool lazy_load = config.getBool("system_tables_lazy_load", true); - - query_log = createSystemLog(global_context, "system", "query_log", config, "query_log", lazy_load); - query_thread_log = createSystemLog(global_context, "system", "query_thread_log", config, "query_thread_log", lazy_load); - part_log = createSystemLog(global_context, "system", "part_log", config, "part_log", lazy_load); - trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log", lazy_load); - text_log = createSystemLog(global_context, "system", "text_log", config, "text_log", lazy_load); - metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log", lazy_load); + query_log = createSystemLog(global_context, "system", "query_log", config, "query_log"); + query_thread_log = createSystemLog(global_context, "system", "query_thread_log", config, "query_thread_log"); + part_log = createSystemLog(global_context, "system", "part_log", config, "part_log"); + trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log"); + text_log = createSystemLog(global_context, "system", "text_log", config, "text_log"); + metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log"); if (metric_log) { @@ -99,6 +96,14 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi logs.emplace_back(text_log.get()); if (metric_log) logs.emplace_back(metric_log.get()); + + bool lazy_load = config.getBool("system_tables_lazy_load", true); + for (auto & log : logs) + { + if (!lazy_load) + log->prepareTable(); + log->startup(); + } } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index ceffba67b50..066956ed53a 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -73,6 +73,8 @@ public: virtual String getName() = 0; virtual ASTPtr getCreateTableQuery() = 0; virtual void flush() = 0; + virtual void prepareTable() = 0; + virtual void startup() = 0; virtual void shutdown() = 0; virtual ~ISystemLog() = default; }; @@ -117,8 +119,7 @@ public: const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_, - bool lazy_load); + size_t flush_interval_milliseconds_); /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. @@ -130,6 +131,9 @@ public: /// Flush data in the buffer to disk void flush() override; + /// Start the background thread. + void startup() override; + /// Stop the background flush thread before destructor. No more data will be written. void shutdown() override { @@ -178,7 +182,7 @@ private: * Renames old table if its structure is not suitable. * This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created. */ - void prepareTable(); + void prepareTable() override; /// flushImpl can be executed only in saving_thread. void flushImpl(const std::vector & to_flush, uint64_t to_flush_end); @@ -190,8 +194,7 @@ SystemLog::SystemLog(Context & context_, const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_, - bool lazy_load) + size_t flush_interval_milliseconds_) : context(context_) , table_id(database_name_, table_name_) , storage_def(storage_def_), @@ -199,12 +202,12 @@ SystemLog::SystemLog(Context & context_, { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); log = &Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); +} - if (!lazy_load) - { - prepareTable(); - } +template +void SystemLog::startup() +{ saving_thread = ThreadFromGlobalPool([this] { savingThreadFunction(); }); } diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index 8ecf6f029d2..b5f1d987b91 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -72,9 +72,9 @@ void TextLogElement::appendToBlock(Block & block) const TextLog::TextLog(Context & context_, const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_, bool lazy_load) + size_t flush_interval_milliseconds_) : SystemLog(context_, database_name_, table_name_, - storage_def_, flush_interval_milliseconds_, lazy_load) + storage_def_, flush_interval_milliseconds_) { // SystemLog methods may write text logs, so we disable logging for the text // log table to avoid recursion. diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index aaf64974e05..73c38429662 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -36,8 +36,7 @@ public: const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_, - bool lazy_load); + size_t flush_interval_milliseconds_); }; } From d78316254839e8ddfd2f02dc0dc978c34567ceda Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 04:01:31 +0300 Subject: [PATCH 424/743] Compare with fixed string as smaller string is zero padded #9272 --- src/Common/memcmpSmall.h | 125 ++++++++++++++++++++++++++++ src/Functions/FunctionsComparison.h | 38 +++++---- 2 files changed, 147 insertions(+), 16 deletions(-) diff --git a/src/Common/memcmpSmall.h b/src/Common/memcmpSmall.h index 5dc5e04a707..8be098fe7bf 100644 --- a/src/Common/memcmpSmall.h +++ b/src/Common/memcmpSmall.h @@ -64,6 +64,81 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char } +/** Variant when memory regions may have different sizes. + * But compare the regions as the smaller one is padded with zero bytes up to the size of the larger. + * It's needed to hold that: toFixedString('abc', 5) = 'abc' + * for compatibility with SQL standard. + */ +template +inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) +{ + size_t min_size = std::min(a_size, b_size); + + for (size_t offset = 0; offset < min_size; offset += 16) + { + uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8( + _mm_loadu_si128(reinterpret_cast(a + offset)), + _mm_loadu_si128(reinterpret_cast(b + offset)))); + mask = ~mask; + + if (mask) + { + offset += __builtin_ctz(mask); + + if (offset >= min_size) + break; + + return detail::cmp(a[offset], b[offset]); + } + } + + /// The strings are equal up to min_size. + /// If the rest of the larger string is zero bytes then the strings are considered equal. + + size_t max_size; + const Char * longest; + int cmp; + + if (a_size == b_size) + { + return 0; + } + else if (a_size > b_size) + { + max_size = a_size; + longest = a; + cmp = 1; + } + else + { + max_size = b_size; + longest = b; + cmp = -1; + } + + const __m128i zero16 = _mm_setzero_si128(); + + for (size_t offset = min_size; offset < max_size; offset += 16) + { + uint16_t mask = _mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(longest + offset)), + zero16)); + + if (mask) + { + offset += __builtin_ctz(mask); + + if (offset >= max_size) + return 0; + return cmp; + } + } + + return 0; +} + + + /** Variant when memory regions have same size. * TODO Check if the compiler can optimize previous function when the caller pass identical sizes. */ @@ -206,6 +281,46 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char return detail::cmp(a_size, b_size); } +template +inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) +{ + size_t min_size; + size_t max_size; + const Char * longest; + int size_cmp; + + if (a_size == b_size) + { + min_size = a_size; + max_size = a_size; + longest = a; + size_cmp = 0; + } + else if (a_size > b_size) + { + min_size = b_size; + max_size = a_size; + longest = a; + size_cmp = 1; + } + else + { + min_size = a_size; + max_size = b_size; + longest = b; + size_cmp = -1; + } + + if (auto res = memcmp(a, b, min_size)) + return res; + + for (size_t i = min_size; i < max_size; ++i) + if (longest[i] != 0) + return size_cmp; + + return 0; +} + template inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size) { @@ -248,3 +363,13 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) } #endif + + +/** Compare memory regions for equality. + * But if the sizes are different, compare the regions as the smaller one is padded with zero bytes up to the size of the larger. + */ +template +inline bool memequalSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) +{ + return 0 == memcmpSmallLikeZeroPaddedAllowOverflow15(a, a_size, b, b_size); +} diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 97a996bbe7e..2daf09253de 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -160,7 +160,7 @@ struct StringComparisonImpl for (size_t i = 0; i < size; ++i) { - c[i] = Op::apply(memcmpSmallAllowOverflow15( + c[i] = Op::apply(memcmpSmallLikeZeroPaddedAllowOverflow15( a_data.data() + prev_a_offset, a_offsets[i] - prev_a_offset - 1, b_data.data() + i * b_n, b_n), 0); @@ -168,7 +168,7 @@ struct StringComparisonImpl } } - static void NO_INLINE string_vectorConstant( + static void NO_INLINE string_vector_constant( const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets, const ColumnString::Chars & b_data, ColumnString::Offset b_size, PaddedPODArray & c) @@ -239,11 +239,11 @@ struct StringComparisonImpl size_t size = a_data.size() / a_n; for (size_t i = 0; i < size; ++i) - c[i] = Op::apply(memcmpSmallAllowOverflow15(a_data.data() + i * a_n, a_n, b_data.data() + i * b_n, b_n), 0); + c[i] = Op::apply(memcmpSmallLikeZeroPaddedAllowOverflow15(a_data.data() + i * a_n, a_n, b_data.data() + i * b_n, b_n), 0); } } - static void NO_INLINE fixed_string_vectorConstant( + static void NO_INLINE fixed_string_vector_constant( const ColumnString::Chars & a_data, ColumnString::Offset a_n, const ColumnString::Chars & b_data, ColumnString::Offset b_size, PaddedPODArray & c) @@ -262,7 +262,7 @@ struct StringComparisonImpl { size_t size = a_data.size(); for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - c[j] = Op::apply(memcmpSmallAllowOverflow15(a_data.data() + i, a_n, b_data.data(), b_size), 0); + c[j] = Op::apply(0, memcmpSmallLikeZeroPaddedAllowOverflow15(a_data.data() + i, a_n, b_data.data(), b_size)); } } @@ -271,7 +271,7 @@ struct StringComparisonImpl const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, PaddedPODArray & c) { - StringComparisonImpl::string_vectorConstant(b_data, b_offsets, a_data, a_size, c); + StringComparisonImpl::string_vector_constant(b_data, b_offsets, a_data, a_size, c); } static void constant_fixed_string_vector( @@ -279,7 +279,7 @@ struct StringComparisonImpl const ColumnString::Chars & b_data, ColumnString::Offset b_n, PaddedPODArray & c) { - StringComparisonImpl::fixed_string_vectorConstant(b_data, b_n, a_data, a_size, c); + StringComparisonImpl::fixed_string_vector_constant(b_data, b_n, a_data, a_size, c); } static void constantConstant( @@ -331,7 +331,7 @@ struct StringEqualsImpl { auto a_size = a_offsets[i] - prev_a_offset - 1; - c[i] = positive == memequalSmallAllowOverflow15( + c[i] = positive == memequalSmallLikeZeroPaddedAllowOverflow15( a_data.data() + prev_a_offset, a_size, b_data.data() + b_n * i, b_n); @@ -339,7 +339,7 @@ struct StringEqualsImpl } } - static void NO_INLINE string_vectorConstant( + static void NO_INLINE string_vector_constant( const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets, const ColumnString::Chars & b_data, ColumnString::Offset b_size, PaddedPODArray & c) @@ -397,15 +397,21 @@ struct StringEqualsImpl { fixed_string_vector_fixed_string_vector_16(a_data, b_data, c); } + else if (a_n == b_n) + { + size_t size = a_data.size() / a_n; + for (size_t i = 0; i < size; ++i) + c[i] = positive == memequalSmallAllowOverflow15(a_data.data() + i * a_n, a_n, b_data.data() + i * a_n, a_n); + } else { size_t size = a_data.size() / a_n; for (size_t i = 0; i < size; ++i) - c[i] = positive == memequalSmallAllowOverflow15(a_data.data() + i * a_n, a_n, b_data.data() + i * b_n, b_n); + c[i] = positive == memequalSmallLikeZeroPaddedAllowOverflow15(a_data.data() + i * a_n, a_n, b_data.data() + i * b_n, b_n); } } - static void NO_INLINE fixed_string_vectorConstant( + static void NO_INLINE fixed_string_vector_constant( const ColumnString::Chars & a_data, ColumnString::Offset a_n, const ColumnString::Chars & b_data, ColumnString::Offset b_size, PaddedPODArray & c) @@ -418,7 +424,7 @@ struct StringEqualsImpl { size_t size = a_data.size() / a_n; for (size_t i = 0; i < size; ++i) - c[i] = positive == memequalSmallAllowOverflow15(a_data.data() + i * a_n, a_n, b_data.data(), b_size); + c[i] = positive == memequalSmallLikeZeroPaddedAllowOverflow15(a_data.data() + i * a_n, a_n, b_data.data(), b_size); } } @@ -435,7 +441,7 @@ struct StringEqualsImpl const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, PaddedPODArray & c) { - string_vectorConstant(b_data, b_offsets, a_data, a_size, c); + string_vector_constant(b_data, b_offsets, a_data, a_size, c); } static void constant_fixed_string_vector( @@ -443,7 +449,7 @@ struct StringEqualsImpl const ColumnString::Chars & b_data, ColumnString::Offset b_n, PaddedPODArray & c) { - fixed_string_vectorConstant(b_data, b_n, a_data, a_size, c); + fixed_string_vector_constant(b_data, b_n, a_data, a_size, c); } static void constantConstant( @@ -780,7 +786,7 @@ private: c1_fixed_string->getChars(), c1_fixed_string->getN(), c_res->getData()); else if (c0_string && c1_const) - StringImpl::string_vectorConstant( + StringImpl::string_vector_constant( c0_string->getChars(), c0_string->getOffsets(), *c1_const_chars, c1_const_size, c_res->getData()); @@ -795,7 +801,7 @@ private: c1_fixed_string->getChars(), c1_fixed_string->getN(), c_res->getData()); else if (c0_fixed_string && c1_const) - StringImpl::fixed_string_vectorConstant( + StringImpl::fixed_string_vector_constant( c0_fixed_string->getChars(), c0_fixed_string->getN(), *c1_const_chars, c1_const_size, c_res->getData()); From 3af3cb62fbf1738852455d39f0da73ce03da5174 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 04:15:57 +0300 Subject: [PATCH 425/743] Added a test --- .../01250_fixed_string_comparison.reference | 110 ++++++++++++++++++ .../01250_fixed_string_comparison.sql | 45 +++++++ 2 files changed, 155 insertions(+) create mode 100644 tests/queries/0_stateless/01250_fixed_string_comparison.reference create mode 100644 tests/queries/0_stateless/01250_fixed_string_comparison.sql diff --git a/tests/queries/0_stateless/01250_fixed_string_comparison.reference b/tests/queries/0_stateless/01250_fixed_string_comparison.reference new file mode 100644 index 00000000000..19754573478 --- /dev/null +++ b/tests/queries/0_stateless/01250_fixed_string_comparison.reference @@ -0,0 +1,110 @@ +Row 1: +────── +equals(b, b): 1 +greater(b, b): 0 +less(b, b): 0 +equals(b, c): 0 +greater(b, c): 0 +less(b, c): 1 +equals(b, d): 0 +greater(b, d): 0 +less(b, d): 1 +equals(b, bf): 1 +greater(b, bf): 0 +less(b, bf): 0 +equals(b, cf): 0 +greater(b, cf): 0 +less(b, cf): 1 +equals(b, df): 0 +greater(b, df): 0 +less(b, df): 1 +equals(c, b): 0 +greater(c, b): 1 +less(c, b): 0 +equals(c, c): 1 +greater(c, c): 0 +less(c, c): 0 +equals(c, d): 0 +greater(c, d): 0 +less(c, d): 1 +equals(c, bf): 0 +greater(c, bf): 1 +less(c, bf): 0 +equals(c, cf): 1 +greater(c, cf): 0 +less(c, cf): 0 +equals(c, df): 0 +greater(c, df): 0 +less(c, df): 1 +equals(d, b): 0 +greater(d, b): 1 +less(d, b): 0 +equals(d, c): 0 +greater(d, c): 1 +less(d, c): 0 +equals(d, d): 1 +greater(d, d): 0 +less(d, d): 0 +equals(d, bf): 0 +greater(d, bf): 1 +less(d, bf): 0 +equals(d, cf): 0 +greater(d, cf): 1 +less(d, cf): 0 +equals(d, df): 1 +greater(d, df): 0 +less(d, df): 0 +equals(bf, b): 1 +greater(bf, b): 0 +less(bf, b): 0 +equals(bf, c): 0 +greater(bf, c): 0 +less(bf, c): 1 +equals(bf, d): 0 +greater(bf, d): 0 +less(bf, d): 1 +equals(bf, bf): 1 +greater(bf, bf): 0 +less(bf, bf): 0 +equals(bf, cf): 0 +greater(bf, cf): 0 +less(bf, cf): 1 +equals(bf, df): 0 +greater(bf, df): 0 +less(bf, df): 1 +equals(cf, b): 0 +greater(cf, b): 1 +less(cf, b): 0 +equals(cf, c): 1 +greater(cf, c): 0 +less(cf, c): 0 +equals(cf, d): 0 +greater(cf, d): 0 +less(cf, d): 1 +equals(cf, bf): 0 +greater(cf, bf): 1 +less(cf, bf): 0 +equals(cf, cf): 1 +greater(cf, cf): 0 +less(cf, cf): 0 +equals(cf, df): 0 +greater(cf, df): 0 +less(cf, df): 1 +equals(df, b): 0 +greater(df, b): 1 +less(df, b): 0 +equals(df, c): 0 +greater(df, c): 1 +less(df, c): 0 +equals(df, d): 1 +greater(df, d): 0 +less(df, d): 0 +equals(df, bf): 0 +greater(df, bf): 1 +less(df, bf): 0 +equals(df, cf): 0 +greater(df, cf): 1 +less(df, cf): 0 +equals(df, df): 1 +greater(df, df): 0 +less(df, df): 0 diff --git a/tests/queries/0_stateless/01250_fixed_string_comparison.sql b/tests/queries/0_stateless/01250_fixed_string_comparison.sql new file mode 100644 index 00000000000..a68803c15aa --- /dev/null +++ b/tests/queries/0_stateless/01250_fixed_string_comparison.sql @@ -0,0 +1,45 @@ +WITH 'abb' AS b, 'abc' AS c, 'abd' AS d, toFixedString(b, 5) AS bf, toFixedString(c, 5) AS cf, toFixedString(d, 5) AS df +SELECT + b = b, b > b, b < b, + b = c, b > c, b < c, + b = d, b > d, b < d, + b = bf, b > bf, b < bf, + b = cf, b > cf, b < cf, + b = df, b > df, b < df, + + c = b, c > b, c < b, + c = c, c > c, c < c, + c = d, c > d, c < d, + c = bf, c > bf, c < bf, + c = cf, c > cf, c < cf, + c = df, c > df, c < df, + + d = b, d > b, d < b, + d = c, d > c, d < c, + d = d, d > d, d < d, + d = bf, d > bf, d < bf, + d = cf, d > cf, d < cf, + d = df, d > df, d < df, + + bf = b, bf > b, bf < b, + bf = c, bf > c, bf < c, + bf = d, bf > d, bf < d, + bf = bf, bf > bf, bf < bf, + bf = cf, bf > cf, bf < cf, + bf = df, bf > df, bf < df, + + cf = b, cf > b, cf < b, + cf = c, cf > c, cf < c, + cf = d, cf > d, cf < d, + cf = bf, cf > bf, cf < bf, + cf = cf, cf > cf, cf < cf, + cf = df, cf > df, cf < df, + + df = b, df > b, df < b, + df = c, df > c, df < c, + df = d, df > d, df < d, + df = bf, df > bf, df < bf, + df = cf, df > cf, df < cf, + df = df, df > df, df < df + +FORMAT Vertical; \ No newline at end of file From d906f6c4bf115c536a94c6997fad66fca2fbe6fd Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 20 Apr 2020 04:19:00 +0300 Subject: [PATCH 426/743] Update 01250_fixed_string_comparison.sql --- tests/queries/0_stateless/01250_fixed_string_comparison.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01250_fixed_string_comparison.sql b/tests/queries/0_stateless/01250_fixed_string_comparison.sql index a68803c15aa..8481b3572bb 100644 --- a/tests/queries/0_stateless/01250_fixed_string_comparison.sql +++ b/tests/queries/0_stateless/01250_fixed_string_comparison.sql @@ -42,4 +42,4 @@ SELECT df = cf, df > cf, df < cf, df = df, df > df, df < df -FORMAT Vertical; \ No newline at end of file +FORMAT Vertical; From 59a9edd0183d7e1646b820251c93fa3be56bf472 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 04:43:03 +0300 Subject: [PATCH 427/743] Added forgotten modification --- src/Functions/FunctionsComparison.h | 21 +-------------------- 1 file changed, 1 insertion(+), 20 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 2daf09253de..8cc1d5114b8 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -281,14 +281,6 @@ struct StringComparisonImpl { StringComparisonImpl::fixed_string_vector_constant(b_data, b_n, a_data, a_size, c); } - - static void constantConstant( - const ColumnString::Chars & a_data, ColumnString::Offset a_size, - const ColumnString::Chars & b_data, ColumnString::Offset b_size, - UInt8 & c) - { - c = Op::apply(memcmpSmallAllowOverflow15(a_data.data(), a_size, b_data.data(), b_size), 0); - } }; @@ -451,14 +443,6 @@ struct StringEqualsImpl { fixed_string_vector_constant(b_data, b_n, a_data, a_size, c); } - - static void constantConstant( - const ColumnString::Chars & a_data, ColumnString::Offset a_size, - const ColumnString::Chars & b_data, ColumnString::Offset b_size, - UInt8 & c) - { - c = positive == memequalSmallAllowOverflow15(a_data.data(), a_size, b_data.data(), b_size); - } }; @@ -764,10 +748,7 @@ private: if (c0_const && c1_const) { - UInt8 res = 0; - StringImpl::constantConstant(*c0_const_chars, c0_const_size, *c1_const_chars, c1_const_size, res); - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(c0_const->size(), toField(res)); - return true; + return executeString(block, result, c0_const->convertToFullColumn().get(), c1_const->convertToFullColumn().get()); } else { From 7023e096ec42e82bacb633467ce0b70a3001eddc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 04:44:24 +0300 Subject: [PATCH 428/743] Check for error code in part check #6269 --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +-------- .../ReplicatedMergeTreePartCheckThread.cpp | 8 ++++++-- src/Storages/MergeTree/checkDataPart.cpp | 16 ++++++++++++++++ src/Storages/MergeTree/checkDataPart.h | 3 +++ 4 files changed, 26 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index aaacea6f1e7..c5a4a5e07f8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -94,15 +94,11 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; extern const int BAD_TYPE_OF_FIELD; extern const int BAD_ARGUMENTS; - extern const int MEMORY_LIMIT_EXCEEDED; extern const int INVALID_PARTITION_VALUE; extern const int METADATA_MISMATCH; extern const int PART_IS_TEMPORARILY_LOCKED; extern const int TOO_MANY_PARTS; extern const int INCOMPATIBLE_COLUMNS; - extern const int CANNOT_ALLOCATE_MEMORY; - extern const int CANNOT_MUNMAP; - extern const int CANNOT_MREMAP; extern const int BAD_TTL_EXPRESSION; extern const int INCORRECT_FILE_NAME; extern const int BAD_DATA_PART_NAME; @@ -954,10 +950,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) /// Don't count the part as broken if there is not enough memory to load it. /// In fact, there can be many similar situations. /// But it is OK, because there is a safety guard against deleting too many parts. - if (e.code() == ErrorCodes::MEMORY_LIMIT_EXCEEDED - || e.code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY - || e.code() == ErrorCodes::CANNOT_MUNMAP - || e.code() == ErrorCodes::CANNOT_MREMAP) + if (isNotEnoughMemoryErrorCode(e.code())) throw; broken = true; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index b587b5f71c0..98b30498839 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -248,9 +248,13 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na LOG_INFO(log, "Part " << part_name << " looks good."); } - catch (const Exception &) + catch (const Exception & e) { - /// TODO Better to check error code. + /// Don't count the part as broken if there is not enough memory to load it. + /// In fact, there can be many similar situations. + /// But it is OK, because there is a safety guard against deleting too many parts. + if (isNotEnoughMemoryErrorCode(e.code())) + throw; tryLogCurrentException(log, __PRETTY_FUNCTION__); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 52d9a2750c7..c12ed12c2e0 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -24,6 +24,22 @@ namespace ErrorCodes { extern const int CORRUPTED_DATA; extern const int UNKNOWN_PART_TYPE; + extern const int MEMORY_LIMIT_EXCEEDED; + extern const int CANNOT_ALLOCATE_MEMORY; + extern const int CANNOT_MUNMAP; + extern const int CANNOT_MREMAP; +} + + +bool isNotEnoughMemoryErrorCode(int code) +{ + /// Don't count the part as broken if there is not enough memory to load it. + /// In fact, there can be many similar situations. + /// But it is OK, because there is a safety guard against deleting too many parts. + return code == ErrorCodes::MEMORY_LIMIT_EXCEEDED + || code == ErrorCodes::CANNOT_ALLOCATE_MEMORY + || code == ErrorCodes::CANNOT_MUNMAP + || code == ErrorCodes::CANNOT_MREMAP; } diff --git a/src/Storages/MergeTree/checkDataPart.h b/src/Storages/MergeTree/checkDataPart.h index 9293c60ded4..9b905c5835e 100644 --- a/src/Storages/MergeTree/checkDataPart.h +++ b/src/Storages/MergeTree/checkDataPart.h @@ -19,4 +19,7 @@ IMergeTreeDataPart::Checksums checkDataPart( const MergeTreeDataPartType & part_type, bool require_checksums, std::function is_cancelled = []{ return false; }); + +bool isNotEnoughMemoryErrorCode(int code); + } From d99c26a40ce7e6871345fc5a5c77d115a12a5a77 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 05:31:21 +0300 Subject: [PATCH 429/743] Fix infinite loop in dictIsIn function #515 --- src/Core/Defines.h | 3 ++ src/Dictionaries/CacheDictionary.cpp | 6 ++-- src/Dictionaries/FlatDictionary.cpp | 4 ++- src/Dictionaries/HashedDictionary.cpp | 4 ++- .../01251_dict_is_in_infinite_loop.reference | 9 +++++ .../01251_dict_is_in_infinite_loop.sql | 33 +++++++++++++++++++ 6 files changed, 55 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference create mode 100644 tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql diff --git a/src/Core/Defines.h b/src/Core/Defines.h index fe614cec6bd..5552de3b045 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -98,3 +98,6 @@ /// Default limit on recursion depth of recursive descend parser. #define DBMS_DEFAULT_MAX_PARSER_DEPTH 1000 + +/// Max depth of hierarchical dictionary +#define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000 diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 30bd521c0bb..f75b063e4e3 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -17,6 +18,7 @@ #include "DictionaryBlockInputStream.h" #include "DictionaryFactory.h" + namespace ProfileEvents { extern const Event DictCacheKeysRequested; @@ -144,7 +146,7 @@ void CacheDictionary::isInImpl(const PaddedPODArray & child_ids, const Ance PaddedPODArray children(out_size, 0); PaddedPODArray parents(child_ids.begin(), child_ids.end()); - while (true) + for (size_t i = 0; i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) { size_t out_idx = 0; size_t parents_idx = 0; @@ -218,7 +220,7 @@ void CacheDictionary::isInConstantVector(const Key child_id, const PaddedPODArra std::vector ancestors(1, child_id); /// Iteratively find all ancestors for child. - while (true) + for (size_t i = 0; i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) { toParent(child, parent); diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index b70fe95346d..fd2275f384d 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -2,6 +2,8 @@ #include #include "DictionaryBlockInputStream.h" #include "DictionaryFactory.h" +#include + namespace DB { @@ -77,7 +79,7 @@ void FlatDictionary::isInImpl(const ChildType & child_ids, const AncestorType & auto id = getAt(child_ids, row); const auto ancestor_id = getAt(ancestor_ids, row); - while (id < loaded_size && id != null_value && id != ancestor_id) + for (size_t i = 0; id < loaded_size && id != null_value && id != ancestor_id && i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) id = attr[id]; out[row] = id != null_value && id == ancestor_id; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index c401142584c..f889c820770 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -2,6 +2,8 @@ #include #include "DictionaryBlockInputStream.h" #include "DictionaryFactory.h" +#include + namespace { @@ -87,7 +89,7 @@ void HashedDictionary::isInAttrImpl(const AttrType & attr, const ChildType & chi auto id = getAt(child_ids, row); const auto ancestor_id = getAt(ancestor_ids, row); - while (id != null_value && id != ancestor_id) + for (size_t i = 0; id != null_value && id != ancestor_id && i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) { auto it = attr.find(id); if (it != std::end(attr)) diff --git a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference new file mode 100644 index 00000000000..3e9f5d53b15 --- /dev/null +++ b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference @@ -0,0 +1,9 @@ +1 +1 +1 +1 +1 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql new file mode 100644 index 00000000000..5f865f97176 --- /dev/null +++ b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql @@ -0,0 +1,33 @@ +DROP TABLE IF EXISTS dict_source; +CREATE TABLE dict_source (id UInt64, parent_id UInt64, value String) ENGINE = Memory; +INSERT INTO dict_source VALUES (1, 0, 'hello'), (2, 1, 'world'), (3, 2, 'upyachka'), (11, 22, 'a'), (22, 11, 'b'); + +DROP DATABASE IF EXISTS database_for_dict; +CREATE DATABASE database_for_dict Engine = Ordinary; + +DROP DICTIONARY IF EXISTS database_for_dict.dictionary_with_hierarchy; + +CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy +( + id UInt64, parent_id UInt64 HIERARCHICAL, value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' table 'dict_source')) +LAYOUT(HASHED()) +LIFETIME(MIN 1 MAX 1); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(2), toUInt64(1)); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(22), toUInt64(11)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), toUInt64(11)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(11), materialize(toUInt64(22))); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), materialize(toUInt64(11))); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(22), toUInt64(111)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), toUInt64(111)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(11), materialize(toUInt64(222))); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), materialize(toUInt64(111))); + +DROP DICTIONARY database_for_dict.dictionary_with_hierarchy; +DROP TABLE dict_source; +DROP DATABASE database_for_dict; From 7634092536384f166b4c8546b7ca77516f0ed5af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 05:35:19 +0300 Subject: [PATCH 430/743] More tests --- .../01251_dict_is_in_infinite_loop.reference | 30 +++++++++ .../01251_dict_is_in_infinite_loop.sql | 64 +++++++++++++++++++ 2 files changed, 94 insertions(+) diff --git a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference index 3e9f5d53b15..757d2858524 100644 --- a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference +++ b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference @@ -7,3 +7,33 @@ 0 0 0 +[11,22] +[22,11] +[11,22] +[22,11] +1 +1 +1 +1 +1 +0 +0 +0 +0 +[11,22] +[22,11] +[11,22] +[22,11] +1 +1 +1 +1 +1 +255 +255 +0 +255 +[11,22] +[22,11] +[11,22] +[22,11] diff --git a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql index 5f865f97176..d8722d4f535 100644 --- a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql +++ b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql @@ -28,6 +28,70 @@ SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUIn SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(11), materialize(toUInt64(222))); SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), materialize(toUInt64(111))); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(11)); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(22)); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(11))); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22))); + + +DROP DICTIONARY IF EXISTS database_for_dict.dictionary_with_hierarchy; + +CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy +( + id UInt64, parent_id UInt64 HIERARCHICAL, value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' table 'dict_source')) +LAYOUT(FLAT()) +LIFETIME(MIN 1 MAX 1); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(2), toUInt64(1)); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(22), toUInt64(11)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), toUInt64(11)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(11), materialize(toUInt64(22))); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), materialize(toUInt64(11))); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(22), toUInt64(111)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), toUInt64(111)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(11), materialize(toUInt64(222))); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), materialize(toUInt64(111))); + +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(11)); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(22)); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(11))); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22))); + + +DROP DICTIONARY IF EXISTS database_for_dict.dictionary_with_hierarchy; + +CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy +( + id UInt64, parent_id UInt64 HIERARCHICAL, value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' table 'dict_source')) +LAYOUT(CACHE(SIZE_IN_CELLS 10)) +LIFETIME(MIN 1 MAX 1); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(2), toUInt64(1)); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(22), toUInt64(11)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), toUInt64(11)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(11), materialize(toUInt64(22))); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), materialize(toUInt64(11))); + +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(22), toUInt64(111)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), toUInt64(111)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(11), materialize(toUInt64(222))); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22)), materialize(toUInt64(111))); + +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(11)); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(22)); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(11))); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', materialize(toUInt64(22))); + + DROP DICTIONARY database_for_dict.dictionary_with_hierarchy; DROP TABLE dict_source; DROP DATABASE database_for_dict; From 3862085af7b0fe98c07d36ddc5762f9c2edad53f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 05:39:43 +0300 Subject: [PATCH 431/743] Fix for embedded dictionaries --- src/Dictionaries/Embedded/RegionsHierarchy.h | 3 ++- src/Functions/FunctionsEmbeddedDictionaries.h | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/Embedded/RegionsHierarchy.h b/src/Dictionaries/Embedded/RegionsHierarchy.h index 25625284e32..7f42a76ecc2 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchy.h +++ b/src/Dictionaries/Embedded/RegionsHierarchy.h @@ -4,6 +4,7 @@ #include #include #include "GeodataProviders/IHierarchiesProvider.h" +#include class IRegionsHierarchyDataProvider; @@ -59,7 +60,7 @@ public: if (lhs >= parents.size()) return false; - while (lhs != 0 && lhs != rhs) + for (size_t i = 0; lhs != 0 && lhs != rhs && i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) lhs = parents[lhs]; return lhs != 0; diff --git a/src/Functions/FunctionsEmbeddedDictionaries.h b/src/Functions/FunctionsEmbeddedDictionaries.h index a8078bc47b7..d337f9ea363 100644 --- a/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/src/Functions/FunctionsEmbeddedDictionaries.h @@ -16,6 +16,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -450,7 +451,7 @@ public: for (size_t i = 0; i < size; ++i) { T cur = vec_from[i]; - while (cur) + for (size_t depth = 0; cur && depth < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++depth) { res_values.push_back(cur); cur = Transform::toParent(cur, dict); From e06dc573b466984383d5c90cb493782387160461 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 06:15:29 +0300 Subject: [PATCH 432/743] Fix flacky integration test --- tests/integration/test_insert_into_distributed/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index b9767e8b694..e48584bac84 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -132,6 +132,7 @@ def test_inserts_batching(started_cluster): for i in range(10, 13): instance.query("INSERT INTO distributed(d, x) VALUES ('2000-01-01', {})".format(i)) + instance.query("SYSTEM FLUSH DISTRIBUTED distributed") time.sleep(1.0) result = remote.query("SELECT _part, groupArray(x) FROM local2 GROUP BY _part ORDER BY _part") From c893edf46f831068920b425f6249f382a63548cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 06:48:35 +0300 Subject: [PATCH 433/743] Fix style --- src/Common/memcmpSmall.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/memcmpSmall.h b/src/Common/memcmpSmall.h index 8be098fe7bf..bafc08a9cbe 100644 --- a/src/Common/memcmpSmall.h +++ b/src/Common/memcmpSmall.h @@ -138,7 +138,6 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz } - /** Variant when memory regions have same size. * TODO Check if the compiler can optimize previous function when the caller pass identical sizes. */ From be4d20e6908f90627a71b26e87c03388e3ed9208 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 07:02:52 +0300 Subject: [PATCH 434/743] Fixed an error and added a test --- src/Functions/FunctionsComparison.h | 7 ++++++- .../queries/0_stateless/01251_string_comparison.reference | 1 + tests/queries/0_stateless/01251_string_comparison.sql | 1 + 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01251_string_comparison.reference create mode 100644 tests/queries/0_stateless/01251_string_comparison.sql diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 8cc1d5114b8..5671a8674e8 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -748,7 +748,12 @@ private: if (c0_const && c1_const) { - return executeString(block, result, c0_const->convertToFullColumn().get(), c1_const->convertToFullColumn().get()); + auto res = executeString(block, result, &c0_const->getDataColumn(), &c1_const->getDataColumn()); + if (!res) + return false; + + block.getByPosition(result).column = ColumnConst::create(block.getByPosition(result).column, c0_const->size()); + return true; } else { diff --git a/tests/queries/0_stateless/01251_string_comparison.reference b/tests/queries/0_stateless/01251_string_comparison.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01251_string_comparison.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01251_string_comparison.sql b/tests/queries/0_stateless/01251_string_comparison.sql new file mode 100644 index 00000000000..b2923ba4320 --- /dev/null +++ b/tests/queries/0_stateless/01251_string_comparison.sql @@ -0,0 +1 @@ +SELECT isConstant('a' = 'b'); From a3195d56ae5ec346db074c5b7ee0db0a784d2bce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 07:19:50 +0300 Subject: [PATCH 435/743] Fix translation --- 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 3f907541a3c..72f93402d64 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1274,7 +1274,8 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) ++total_fetches; SCOPE_EXIT({--total_fetches;}); - if (storage_settings_ptr->replicated_max_parallel_fetches_for_table && current_table_fetches >= storage_settings_ptr->replicated_max_parallel_fetches_for_table) + if (storage_settings_ptr->replicated_max_parallel_fetches_for_table + && current_table_fetches >= storage_settings_ptr->replicated_max_parallel_fetches_for_table) { throw Exception("Too many fetches from replicas for table, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches_for_table.toString(), ErrorCodes::TOO_MANY_FETCHES); @@ -1416,7 +1417,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) } catch (...) { - /** If you can not download the part you need for some merge, it's better not to try to get other parts for this merge, + /** If we can not download the part we need for some merge, it's better not to try to get other parts for this merge, * but try to get already merged part. To do this, move the action to get the remaining parts * for this merge at the end of the queue. */ From 318ab3b51eaf3efe702dc8b579304b7dd4c3dfb2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 20 Apr 2020 12:49:17 +0800 Subject: [PATCH 436/743] ISSUES-5436 try fix build failure & pvs & style --- programs/server/HTTPHandlerFactory.cpp | 55 +++++++++++++++----------- programs/server/HTTPHandlerFactory.h | 3 +- programs/server/Server.cpp | 23 ++--------- 3 files changed, 36 insertions(+), 45 deletions(-) diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index 8dee745d0e3..55aaa7dc6a2 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -23,13 +23,12 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_COMPILE_REGEXP; - extern const int NO_ELEMENTS_IN_CONFIG; + extern const int LOGICAL_ERROR; extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_) - : server(server_), log(&Logger::get(name_)), name(name_) +HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) + : log(&Logger::get(name_)), name(name_) { } @@ -77,29 +76,37 @@ HTTPRequestHandlerFactoryMain::TThis * HTTPRequestHandlerFactoryMain::addHandler static inline auto createHandlersFactoryFromConfig(IServer & server, const std::string & name, const String & prefix) { - auto main_handler_factory = new HTTPRequestHandlerFactoryMain(server, name); + auto main_handler_factory = new HTTPRequestHandlerFactoryMain(name); - Poco::Util::AbstractConfiguration::Keys keys; - server.config().keys(prefix, keys); - - for (const auto & key : keys) + try { - if (!startsWith(key, "routing_rule")) - throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + Poco::Util::AbstractConfiguration::Keys keys; + server.config().keys(prefix, keys); - const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); + for (const auto & key : keys) + { + if (!startsWith(key, "routing_rule")) + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - if (handler_type == "static") - main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix)); - else if (handler_type == "dynamic_query_handler") - main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix)); - else if (handler_type == "predefine_query_handler") - main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix)); - else - throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); + + if (handler_type == "static") + main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix)); + else if (handler_type == "dynamic_query_handler") + main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix)); + else if (handler_type == "predefine_query_handler") + main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix)); + else + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + + return main_handler_factory; + } + catch (...) + { + delete main_handler_factory; + throw; } - - return main_handler_factory; } static const auto ping_response_expression = "Ok.\n"; @@ -111,7 +118,7 @@ static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IS return createHandlersFactoryFromConfig(server, name, "routing_rules"); else { - return (new HTTPRequestHandlerFactoryMain(server, name)) + return (new HTTPRequestHandlerFactoryMain(name)) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) ->attachStrictPath("/")->allowGetAndHeadRequest()) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) @@ -127,7 +134,7 @@ static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IS static inline Poco::Net::HTTPRequestHandlerFactory * createInterserverHTTPHandlerFactory(IServer & server, const std::string & name) { - return (new HTTPRequestHandlerFactoryMain(server, name)) + return (new HTTPRequestHandlerFactoryMain(name)) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) ->attachStrictPath("/")->allowGetAndHeadRequest()) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) diff --git a/programs/server/HTTPHandlerFactory.h b/programs/server/HTTPHandlerFactory.h index 15f15db9768..c3686d9f479 100644 --- a/programs/server/HTTPHandlerFactory.h +++ b/programs/server/HTTPHandlerFactory.h @@ -16,7 +16,6 @@ class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactor private: using TThis = HTTPRequestHandlerFactoryMain; - IServer & server; Logger * log; std::string name; @@ -25,7 +24,7 @@ public: ~HTTPRequestHandlerFactoryMain(); - HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_); + HTTPRequestHandlerFactoryMain(const std::string & name_); TThis * addHandler(Poco::Net::HTTPRequestHandlerFactory * child_factory); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 96a14dd3d87..5ee7a39d6a0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -767,15 +767,9 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, port); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); - auto handler_factory = createHandlerFactory(*this, "HTTPHandler-factory"); -// if (config().has("prometheus") && config().getInt("prometheus.port", 0) == 0) -// handler_factory->addHandler(async_metrics); servers.emplace_back(std::make_unique( - handler_factory, - server_pool, - socket, - http_params)); + createHandlerFactory(*this, "HTTPHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for http://" + address.toString()); }); @@ -789,10 +783,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "HTTPSHandler-factory"), - server_pool, - socket, - http_params)); + createHandlerFactory(*this, "HTTPSHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for https://" + address.toString()); #else @@ -847,10 +838,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params)); + createHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for replica communication (interserver): http://" + address.toString()); }); @@ -863,10 +851,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "InterserverIOHTTPSHandler-factory"), - server_pool, - socket, - http_params)); + createHandlerFactory(*this, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for secure replica communication (interserver): https://" + address.toString()); #else From 38aaa6206e66fe241d48e8842d9a4a8a6c377e40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 08:29:08 +0300 Subject: [PATCH 437/743] Fix test --- .../01251_dict_is_in_infinite_loop.sql | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql index d8722d4f535..decf65dc8cf 100644 --- a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql +++ b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql @@ -1,10 +1,10 @@ -DROP TABLE IF EXISTS dict_source; -CREATE TABLE dict_source (id UInt64, parent_id UInt64, value String) ENGINE = Memory; -INSERT INTO dict_source VALUES (1, 0, 'hello'), (2, 1, 'world'), (3, 2, 'upyachka'), (11, 22, 'a'), (22, 11, 'b'); - DROP DATABASE IF EXISTS database_for_dict; CREATE DATABASE database_for_dict Engine = Ordinary; +DROP TABLE IF EXISTS database_for_dict.dict_source; +CREATE TABLE database_for_dict.dict_source (id UInt64, parent_id UInt64, value String) ENGINE = Memory; +INSERT INTO database_for_dict.dict_source VALUES (1, 0, 'hello'), (2, 1, 'world'), (3, 2, 'upyachka'), (11, 22, 'a'), (22, 11, 'b'); + DROP DICTIONARY IF EXISTS database_for_dict.dictionary_with_hierarchy; CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy @@ -12,7 +12,7 @@ CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy id UInt64, parent_id UInt64 HIERARCHICAL, value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' table 'dict_source')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'dict_source')) LAYOUT(HASHED()) LIFETIME(MIN 1 MAX 1); @@ -41,7 +41,7 @@ CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy id UInt64, parent_id UInt64 HIERARCHICAL, value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' table 'dict_source')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'dict_source')) LAYOUT(FLAT()) LIFETIME(MIN 1 MAX 1); @@ -70,7 +70,7 @@ CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy id UInt64, parent_id UInt64 HIERARCHICAL, value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' table 'dict_source')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'dict_source')) LAYOUT(CACHE(SIZE_IN_CELLS 10)) LIFETIME(MIN 1 MAX 1); @@ -93,5 +93,5 @@ SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', materiali DROP DICTIONARY database_for_dict.dictionary_with_hierarchy; -DROP TABLE dict_source; +DROP TABLE database_for_dict.dict_source; DROP DATABASE database_for_dict; From 47e4d14eceb9344e7abcaf43e51881fc93b6368d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 09:12:51 +0300 Subject: [PATCH 438/743] Whitespace --- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 72f93402d64..579ccc1bd1e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3307,7 +3307,6 @@ void StorageReplicatedMergeTree::alter( alter_entry->alter_version = new_metadata_version; alter_entry->create_time = time(nullptr); - auto maybe_mutation_commands = params.getMutationCommands(current_metadata); alter_entry->have_mutation = !maybe_mutation_commands.empty(); From c7d0625d0215971359a430d2a48a90db62378e0f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 20 Apr 2020 10:07:05 +0300 Subject: [PATCH 439/743] Restore lost anchor --- website/templates/index/performance.html | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/website/templates/index/performance.html b/website/templates/index/performance.html index de54b05dd2d..facf7003b6a 100644 --- a/website/templates/index/performance.html +++ b/website/templates/index/performance.html @@ -11,12 +11,11 @@

    -

    Independent benchmarks

    +

    Independent benchmarks

    -
    • ClickHouse: New Open Source Columnar Database
      Percona
    • From 860c4a27bd571d2cb45c3d6639c21ab2cb429a10 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 10:10:25 +0300 Subject: [PATCH 440/743] Fix for weird timezones --- base/common/DateLUTImpl.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index ec32d62bcad..a4d4c700d37 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -95,14 +95,14 @@ private: /// UTC offset is from -12 to +14 in all known time zones. This requires checking only three indices. - if ((guess == 0 || t >= lut[guess].date) && t < lut[DayNum(guess + 1)].date) + if (t >= lut[guess].date && t < lut[DayNum(guess + 1)].date) return guess; /// Time zones that have offset 0 from UTC do daylight saving time change (if any) towards increasing UTC offset (example: British Standard Time). - if (offset_at_start_of_epoch >= 0) - return DayNum(guess + 1); + if (t < lut[guess].date) + return DayNum(guess - 1); - return DayNum(guess - 1); + return DayNum(guess + 1); } inline const Values & find(time_t t) const @@ -579,7 +579,7 @@ public: return t / 3600; /// Assume that if offset was fractional, then the fraction is the same as at the beginning of epoch. - /// NOTE This assumption is false for "Pacific/Pitcairn" time zone. + /// NOTE This assumption is false for "Pacific/Pitcairn" and "Pacific/Kiritimati" time zones. return (t + 86400 - offset_at_start_of_epoch) / 3600; } From 1d559b9308148328c79a0f1e772e48ba9b93f654 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 10:21:20 +0300 Subject: [PATCH 441/743] Added a test --- tests/queries/0_stateless/01252_weird_time_zone.reference | 7 +++++++ tests/queries/0_stateless/01252_weird_time_zone.sql | 7 +++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/01252_weird_time_zone.reference create mode 100644 tests/queries/0_stateless/01252_weird_time_zone.sql diff --git a/tests/queries/0_stateless/01252_weird_time_zone.reference b/tests/queries/0_stateless/01252_weird_time_zone.reference new file mode 100644 index 00000000000..7d569731109 --- /dev/null +++ b/tests/queries/0_stateless/01252_weird_time_zone.reference @@ -0,0 +1,7 @@ +2020-01-02 03:04:05 +2020-01-02 03:04:05 +2020-01-02 03:04:05 +2020-01-02 03:04:05 +2020-01-02 03:04:05 +2020-01-02 03:04:05 +2020-01-02 03:04:05 diff --git a/tests/queries/0_stateless/01252_weird_time_zone.sql b/tests/queries/0_stateless/01252_weird_time_zone.sql new file mode 100644 index 00000000000..e2500548530 --- /dev/null +++ b/tests/queries/0_stateless/01252_weird_time_zone.sql @@ -0,0 +1,7 @@ +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Kiritimati'); +SELECT toDateTime('2020-01-02 03:04:05', 'Africa/El_Aaiun'); +SELECT toDateTime('2020-01-02 03:04:05', 'Asia/Pyongyang'); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Kwajalein'); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Apia'); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Enderbury'); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Fakaofo'); From ac31d22572faefaa85ac90fc949d80dcd675f8a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 10:23:17 +0300 Subject: [PATCH 442/743] More tests --- .../0_stateless/01252_weird_time_zone.reference | 14 +++++++------- .../queries/0_stateless/01252_weird_time_zone.sql | 15 ++++++++------- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01252_weird_time_zone.reference b/tests/queries/0_stateless/01252_weird_time_zone.reference index 7d569731109..f2968d4efa6 100644 --- a/tests/queries/0_stateless/01252_weird_time_zone.reference +++ b/tests/queries/0_stateless/01252_weird_time_zone.reference @@ -1,7 +1,7 @@ -2020-01-02 03:04:05 -2020-01-02 03:04:05 -2020-01-02 03:04:05 -2020-01-02 03:04:05 -2020-01-02 03:04:05 -2020-01-02 03:04:05 -2020-01-02 03:04:05 +2020-01-02 03:04:05 2020-01-02 00:00:00 3 +2020-01-02 03:04:05 2020-01-02 00:00:00 3 +2020-01-02 03:04:05 2020-01-02 00:00:00 3 +2020-01-02 03:04:05 2020-01-02 00:00:00 3 +2020-01-02 03:04:05 2020-01-02 00:00:00 3 +2020-01-02 03:04:05 2020-01-02 00:00:00 3 +2020-01-02 03:04:05 2020-01-02 00:00:00 3 diff --git a/tests/queries/0_stateless/01252_weird_time_zone.sql b/tests/queries/0_stateless/01252_weird_time_zone.sql index e2500548530..cd01ca6832e 100644 --- a/tests/queries/0_stateless/01252_weird_time_zone.sql +++ b/tests/queries/0_stateless/01252_weird_time_zone.sql @@ -1,7 +1,8 @@ -SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Kiritimati'); -SELECT toDateTime('2020-01-02 03:04:05', 'Africa/El_Aaiun'); -SELECT toDateTime('2020-01-02 03:04:05', 'Asia/Pyongyang'); -SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Kwajalein'); -SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Apia'); -SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Enderbury'); -SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Fakaofo'); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Kiritimati') AS x, toStartOfDay(x), toHour(x); +SELECT toDateTime('2020-01-02 03:04:05', 'Africa/El_Aaiun') AS x, toStartOfDay(x), toHour(x); +SELECT toDateTime('2020-01-02 03:04:05', 'Asia/Pyongyang') AS x, toStartOfDay(x), toHour(x); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Kwajalein') AS x, toStartOfDay(x), toHour(x); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Apia') AS x, toStartOfDay(x), toHour(x); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Enderbury') AS x, toStartOfDay(x), toHour(x); +SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Fakaofo') AS x, toStartOfDay(x), toHour(x); + From 48d6eb719a6133253553c35c2916bd5f5620d471 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 10:41:03 +0300 Subject: [PATCH 443/743] Fix error --- base/common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index a4d4c700d37..45637199e4f 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -95,7 +95,7 @@ private: /// UTC offset is from -12 to +14 in all known time zones. This requires checking only three indices. - if (t >= lut[guess].date && t < lut[DayNum(guess + 1)].date) + if ((guess == 0 || t >= lut[guess].date) && t < lut[DayNum(guess + 1)].date) return guess; /// Time zones that have offset 0 from UTC do daylight saving time change (if any) towards increasing UTC offset (example: British Standard Time). From 5a56714b6eeab64e2964da50da92566e1825a321 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 10:41:26 +0300 Subject: [PATCH 444/743] Added a test --- tests/queries/0_stateless/01252_weird_time_zone.sql | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/01252_weird_time_zone.sql b/tests/queries/0_stateless/01252_weird_time_zone.sql index cd01ca6832e..68ea903a797 100644 --- a/tests/queries/0_stateless/01252_weird_time_zone.sql +++ b/tests/queries/0_stateless/01252_weird_time_zone.sql @@ -6,3 +6,10 @@ SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Apia') AS x, toStartOfDay(x), SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Enderbury') AS x, toStartOfDay(x), toHour(x); SELECT toDateTime('2020-01-02 03:04:05', 'Pacific/Fakaofo') AS x, toStartOfDay(x), toHour(x); +SELECT toHour(toDateTime(rand(), 'Pacific/Kiritimati') AS t) AS h, t FROM numbers(1000000) WHERE h < 0 OR h > 23 ORDER BY h LIMIT 1 BY h; +SELECT toHour(toDateTime(rand(), 'Africa/El_Aaiun') AS t) AS h, t FROM numbers(1000000) WHERE h < 0 OR h > 23 ORDER BY h LIMIT 1 BY h; +SELECT toHour(toDateTime(rand(), 'Asia/Pyongyang') AS t) AS h, t FROM numbers(1000000) WHERE h < 0 OR h > 23 ORDER BY h LIMIT 1 BY h; +SELECT toHour(toDateTime(rand(), 'Pacific/Kwajalein') AS t) AS h, t FROM numbers(1000000) WHERE h < 0 OR h > 23 ORDER BY h LIMIT 1 BY h; +SELECT toHour(toDateTime(rand(), 'Pacific/Apia') AS t) AS h, t FROM numbers(1000000) WHERE h < 0 OR h > 23 ORDER BY h LIMIT 1 BY h; +SELECT toHour(toDateTime(rand(), 'Pacific/Enderbury') AS t) AS h, t FROM numbers(1000000) WHERE h < 0 OR h > 23 ORDER BY h LIMIT 1 BY h; +SELECT toHour(toDateTime(rand(), 'Pacific/Fakaofo') AS t) AS h, t FROM numbers(1000000) WHERE h < 0 OR h > 23 ORDER BY h LIMIT 1 BY h; From 2d08c8e4ab71fe121dd8cedc13ca24559f7de083 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 20 Apr 2020 15:45:19 +0800 Subject: [PATCH 445/743] clang-10 fix --- base/common/strong_typedef.h | 6 ------ contrib/llvm | 2 +- contrib/rapidjson | 2 +- src/Common/tests/chaos_sanitizer.cpp | 13 +++++++------ src/Core/SettingsCollection.h | 4 ++-- src/Core/SettingsCollectionImpl.h | 6 +++--- 6 files changed, 14 insertions(+), 19 deletions(-) diff --git a/base/common/strong_typedef.h b/base/common/strong_typedef.h index ae502b4ef97..a46eb415e15 100644 --- a/base/common/strong_typedef.h +++ b/base/common/strong_typedef.h @@ -1,15 +1,9 @@ #pragma once -#include #include -/** https://svn.boost.org/trac/boost/ticket/5182 - */ - template struct StrongTypedef - : boost::totally_ordered1< StrongTypedef - , boost::totally_ordered2< StrongTypedef, T> > { private: using Self = StrongTypedef; diff --git a/contrib/llvm b/contrib/llvm index 5dab18f4861..4bb9d5c58d9 160000 --- a/contrib/llvm +++ b/contrib/llvm @@ -1 +1 @@ -Subproject commit 5dab18f4861677548b8f7f6815f49384480ecead +Subproject commit 4bb9d5c58d92baf4f043ee5258bcdfa8ac567545 diff --git a/contrib/rapidjson b/contrib/rapidjson index 01950eb7ace..8f4c021fa2f 160000 --- a/contrib/rapidjson +++ b/contrib/rapidjson @@ -1 +1 @@ -Subproject commit 01950eb7acec78818d68b762efc869bba2420d82 +Subproject commit 8f4c021fa2f1e001d2376095928fc0532adf2ae6 diff --git a/src/Common/tests/chaos_sanitizer.cpp b/src/Common/tests/chaos_sanitizer.cpp index 130d660f7ab..98f28f95b78 100644 --- a/src/Common/tests/chaos_sanitizer.cpp +++ b/src/Common/tests/chaos_sanitizer.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -20,31 +21,31 @@ int main(int argc, char ** argv) std::cerr << (DB::ThreadFuzzer::instance().isEffective() ? "ThreadFuzzer is enabled.\n" : "ThreadFuzzer is not enabled.\n"); - volatile size_t counter1 = 0; - volatile size_t counter2 = 0; + std::atomic counter1 = 0; + std::atomic counter2 = 0; /// These threads are synchronized by sleep (that's intentionally incorrect). std::thread t1([&] { for (size_t i = 0; i < num_iterations; ++i) - ++counter1; + counter1.store(counter1.load(std::memory_order_relaxed) + 1, std::memory_order_relaxed); sleepForNanoseconds(100000000); for (size_t i = 0; i < num_iterations; ++i) - ++counter2; + counter2.store(counter2.load(std::memory_order_relaxed) + 1, std::memory_order_relaxed); }); std::thread t2([&] { for (size_t i = 0; i < num_iterations; ++i) - ++counter2; + counter2.store(counter2.load(std::memory_order_relaxed) + 1, std::memory_order_relaxed); sleepForNanoseconds(100000000); for (size_t i = 0; i < num_iterations; ++i) - ++counter1; + counter1.store(counter1.load(std::memory_order_relaxed) + 1, std::memory_order_relaxed); }); t1.join(); diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index d93772e86ed..9b1813cdb86 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -524,8 +524,8 @@ public: bool tryGet(const StringRef & name, String & value) const; /// Compares two collections of settings. - bool operator ==(const Derived & rhs) const; - bool operator!=(const Derived & rhs) const { return !(*this == rhs); } + bool operator ==(const SettingsCollection & rhs) const; + bool operator!=(const SettingsCollection & rhs) const { return !(*this == rhs); } /// Gathers all changed values (e.g. for applying them later to another collection of settings). SettingsChanges changes() const; diff --git a/src/Core/SettingsCollectionImpl.h b/src/Core/SettingsCollectionImpl.h index d5716c2a80d..8210b04e2da 100644 --- a/src/Core/SettingsCollectionImpl.h +++ b/src/Core/SettingsCollectionImpl.h @@ -173,19 +173,19 @@ bool SettingsCollection::tryGet(const StringRef & name, String & value) template -bool SettingsCollection::operator ==(const Derived & rhs) const +bool SettingsCollection::operator ==(const SettingsCollection & rhs) const { const auto & the_members = members(); for (size_t i = 0; i != the_members.size(); ++i) { const auto & member = the_members[i]; bool left_changed = member.is_changed(castToDerived()); - bool right_changed = member.is_changed(rhs); + bool right_changed = member.is_changed(rhs.castToDerived()); if (left_changed || right_changed) { if (left_changed != right_changed) return false; - if (member.get_field(castToDerived()) != member.get_field(rhs)) + if (member.get_field(castToDerived()) != member.get_field(rhs.castToDerived())) return false; } } From 09dedaeac39def693be54a40274bc856d2bef85e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 10:49:00 +0300 Subject: [PATCH 446/743] Fix error --- base/common/DateLUTImpl.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 45637199e4f..cb41d2ef1c1 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -99,10 +99,10 @@ private: return guess; /// Time zones that have offset 0 from UTC do daylight saving time change (if any) towards increasing UTC offset (example: British Standard Time). - if (t < lut[guess].date) - return DayNum(guess - 1); + if (t >= lut[DayNum(guess + 1)].date) + return DayNum(guess + 1); - return DayNum(guess + 1); + return DayNum(guess - 1); } inline const Values & find(time_t t) const From 861241af9e577ca444ccd39d7db7ce4f6b0ba899 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Mon, 20 Apr 2020 08:10:42 +0000 Subject: [PATCH 447/743] Bump pymdown-extensions from 7.0 to 7.1 in /docs/tools Bumps [pymdown-extensions](https://github.com/facelessuser/pymdown-extensions) from 7.0 to 7.1. - [Release notes](https://github.com/facelessuser/pymdown-extensions/releases) - [Commits](https://github.com/facelessuser/pymdown-extensions/compare/7.0.0...7.1) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index e6265b5e9e2..a57a8b22fec 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -24,7 +24,7 @@ nose==1.3.7 protobuf==3.11.3 numpy==1.18.2 Pygments==2.5.2 -pymdown-extensions==7.0 +pymdown-extensions==7.1 python-slugify==1.2.6 PyYAML==5.3.1 repackage==0.7.3 From 848e4a0541630aef09cd7fb4926c89b7bdd0c060 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Apr 2020 11:15:43 +0300 Subject: [PATCH 448/743] Added a test for #2219 --- ..._aggregate_function_JustStranger.reference | 1 + ...ery_in_aggregate_function_JustStranger.sql | 32 +++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.reference create mode 100644 tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.sql diff --git a/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.reference b/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.reference new file mode 100644 index 00000000000..72c3262a309 --- /dev/null +++ b/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.reference @@ -0,0 +1 @@ +Hello 1 diff --git a/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.sql b/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.sql new file mode 100644 index 00000000000..9659db7973d --- /dev/null +++ b/tests/queries/0_stateless/01253_subquery_in_aggregate_function_JustStranger.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_sharded; + +create table + test_table_sharded( + date Date, + text String, + hash UInt64 + ) +engine=MergeTree(date, (hash, date), 8192); + +create table test_table as test_table_sharded +engine=Distributed(test_cluster_two_shards, currentDatabase(), test_table_sharded, hash); + +SET distributed_product_mode = 'local'; +SET insert_distributed_sync = 1; + +INSERT INTO test_table VALUES ('2020-04-20', 'Hello', 123); + +SELECT + text, + uniqExactIf(hash, hash IN ( + SELECT DISTINCT + hash + FROM test_table AS t1 + )) as counter +FROM test_table AS t2 +GROUP BY text +ORDER BY counter, text; + +DROP TABLE test_table; +DROP TABLE test_table_sharded; From 8efd18dacc028470bf8eea9cf9c196ff8b519baf Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Mon, 20 Apr 2020 08:29:05 +0000 Subject: [PATCH 449/743] Bump numpy from 1.18.2 to 1.18.3 in /docs/tools Bumps [numpy](https://github.com/numpy/numpy) from 1.18.2 to 1.18.3. - [Release notes](https://github.com/numpy/numpy/releases) - [Changelog](https://github.com/numpy/numpy/blob/master/doc/HOWTO_RELEASE.rst.txt) - [Commits](https://github.com/numpy/numpy/compare/v1.18.2...v1.18.3) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index a57a8b22fec..cc63ea71d8c 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -22,7 +22,7 @@ mkdocs-macros-plugin==0.4.6 nltk==3.5 nose==1.3.7 protobuf==3.11.3 -numpy==1.18.2 +numpy==1.18.3 Pygments==2.5.2 pymdown-extensions==7.1 python-slugify==1.2.6 From 9ca86dbf34aa4e667790f9e835aa3e3178068022 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Apr 2020 11:30:01 +0300 Subject: [PATCH 450/743] Drop disks reservation in Aggregator True reservation (IVolume::reserve()) will not takes compression into account, and you aggregate a lot of data then it will likely fail even though it can be completed, but just because you have big max_bytes_before_external_group_by and huge max_threads it will fail. --- src/Interpreters/Aggregator.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 03fc04d2152..b11b22014bd 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -694,11 +694,20 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData && worth_convert_to_two_level) { size_t size = current_memory_usage + params.min_free_disk_space; - auto reservation = params.tmp_volume->reserve(size); - if (!reservation) - throw Exception("Not enough space for external aggregation in temporary storage", ErrorCodes::NOT_ENOUGH_SPACE); + const std::string tmp_path = params.tmp_volume->getNextDisk()->getPath(); + + // enoughSpaceInDirectory() is not enough to make it right, since + // another process (or another thread of aggregator) can consume all + // space. + // + // But true reservation (IVolume::reserve()) cannot be used here since + // current_memory_usage does not takes compression into account and + // will reserve way more that actually will be used. + // + // Hence let's do a simple check. + if (!enoughSpaceInDirectory(tmp_path, size)) + throw Exception("Not enough space for external aggregation in " + tmp_path, ErrorCodes::NOT_ENOUGH_SPACE); - const std::string tmp_path(reservation->getDisk()->getPath()); writeToTemporaryFile(result, tmp_path); } From c5ca4c3b3aedf0f602774dc3e9c7b7fc0900a972 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 20 Apr 2020 12:11:13 +0300 Subject: [PATCH 451/743] Docs build/content fixes (#10374) * fix docs build * some markdown fixes * more fixes --- .../simpleaggregatefunction.md | 36 ------------------- .../data_types/simpleaggregatefunction.md | 34 ++++++++++++++++++ 2 files changed, 34 insertions(+), 36 deletions(-) delete mode 100644 docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md create mode 100644 docs/en/sql_reference/data_types/simpleaggregatefunction.md diff --git a/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md b/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md deleted file mode 100644 index 4e086541053..00000000000 --- a/docs/en/sql_reference/data_types/nested_data_structures/simpleaggregatefunction.md +++ /dev/null @@ -1,36 +0,0 @@ -# SimpleAggregateFunction(name, types\_of\_arguments…) {#data-type-simpleaggregatefunction} - -Unlike [`AggregateFunction`](../aggregatefunction.md), which stores not the value of the aggregate function but it's state: - -- `SimpleAggregateFunction` data type stores current value of the aggregate function, and does not store its full state as [`AggregateFunction`](../aggregatefunction.md) does. This optimization can be applied to functions for which the following property holds: the result of applying a function `f` to a row set `S1 UNION ALL S2` can be obtained by applying `f` to parts of the row set separately, and then again applying `f` to the results: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. This property guarantees that partial aggregation results are enough to compute the combined one, so we don't have to store and process any extra data. - -Currently, the following aggregate functions are supported: - - - [`any`](../../query_language/agg_functions/reference.md#agg_function-any) - - [`anyLast`](../../query_language/agg_functions/reference.md#anylastx) - - [`min`](../../query_language/agg_functions/reference.md#agg_function-min) - - [`max`](../../query_language/agg_functions/reference.md#agg_function-max) - - [`sum`](../../query_language/agg_functions/reference.md#agg_function-sum) - - [`groupBitAnd`](../../query_language/agg_functions/reference.md#groupbitand) - - [`groupBitOr`](../../query_language/agg_functions/reference.md#groupbitor) - - [`groupBitXor`](../../query_language/agg_functions/reference.md#groupbitxor) - -- Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. -- `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function. - -**Parameters** - -- Name of the aggregate function. -- Types of the aggregate function arguments. - -**Example** - -``` sql -CREATE TABLE t -( - column1 SimpleAggregateFunction(sum, UInt64), - column2 SimpleAggregateFunction(any, String) -) ENGINE = ... -``` - -[Original article](https://clickhouse.tech/docs/en/data_types/nested_data_structures/simpleaggregatefunction/) diff --git a/docs/en/sql_reference/data_types/simpleaggregatefunction.md b/docs/en/sql_reference/data_types/simpleaggregatefunction.md new file mode 100644 index 00000000000..81036aec4c8 --- /dev/null +++ b/docs/en/sql_reference/data_types/simpleaggregatefunction.md @@ -0,0 +1,34 @@ +# SimpleAggregateFunction(name, types\_of\_arguments…) {#data-type-simpleaggregatefunction} + +`SimpleAggregateFunction` data type stores current value of the aggregate function, and does not store its full state as [`AggregateFunction`](aggregatefunction.md) does. This optimization can be applied to functions for which the following property holds: the result of applying a function `f` to a row set `S1 UNION ALL S2` can be obtained by applying `f` to parts of the row set separately, and then again applying `f` to the results: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. This property guarantees that partial aggregation results are enough to compute the combined one, so we don't have to store and process any extra data. + +The following aggregate functions are supported: + +- [`any`](../../sql_reference/aggregate_functions/reference.md#agg_function-any) +- [`anyLast`](../../sql_reference/aggregate_functions/reference.md#anylastx) +- [`min`](../../sql_reference/aggregate_functions/reference.md#agg_function-min) +- [`max`](../../sql_reference/aggregate_functions/reference.md#agg_function-max) +- [`sum`](../../sql_reference/aggregate_functions/reference.md#agg_function-sum) +- [`groupBitAnd`](../../sql_reference/aggregate_functions/reference.md#groupbitand) +- [`groupBitOr`](../../sql_reference/aggregate_functions/reference.md#groupbitor) +- [`groupBitXor`](../../sql_reference/aggregate_functions/reference.md#groupbitxor) + + +Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function. + +**Parameters** + +- Name of the aggregate function. +- Types of the aggregate function arguments. + +**Example** + +``` sql +CREATE TABLE t +( + column1 SimpleAggregateFunction(sum, UInt64), + column2 SimpleAggregateFunction(any, String) +) ENGINE = ... +``` + +[Original article](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) From 052598534add6f3d8355cf0c6d501aa33293afa9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Apr 2020 12:42:56 +0300 Subject: [PATCH 452/743] Try fix unit tests. --- src/Common/tests/gtest_global_context.h | 27 +++++++----- src/Storages/tests/gtest_storage_log.cpp | 21 ++++----- ..._transform_query_for_external_database.cpp | 43 +++++++++++-------- 3 files changed, 54 insertions(+), 37 deletions(-) diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 6f5a3c0e1a1..05f60e01774 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -2,17 +2,24 @@ #include -inline DB::Context createContext() +struct ContextHolder { - static DB::SharedContextHolder shared_context = DB::Context::createShared(); - auto context = DB::Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - context.setPath("./"); - return context; -} + DB::SharedContextHolder shared_context; + DB::Context context; -inline const DB::Context & getContext() + ContextHolder() + : shared_context(DB::Context::createShared()) + , context(DB::Context::createGlobal(shared_context.get())) + { + } + + ContextHolder(ContextHolder &&) = default; +}; + +inline ContextHolder getContext() { - static DB::Context global_context = createContext(); - return global_context; + ContextHolder holder; + holder.context.makeGlobalContext(); + holder.context.setPath("./"); + return holder; } diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 3fd79079c8f..7f5340ab59a 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -68,7 +68,7 @@ using DiskImplementations = testing::Types; TYPED_TEST_SUITE(StorageLogTest, DiskImplementations); // Returns data written to table in Values format. -std::string writeData(int rows, DB::StoragePtr & table) +std::string writeData(int rows, DB::StoragePtr & table, DB::Context & context) { using namespace DB; @@ -96,23 +96,23 @@ std::string writeData(int rows, DB::StoragePtr & table) block.insert(column); } - BlockOutputStreamPtr out = table->write({}, getContext()); + BlockOutputStreamPtr out = table->write({}, context); out->write(block); return data; } // Returns all table data in Values format. -std::string readData(DB::StoragePtr & table) +std::string readData(DB::StoragePtr & table, DB::Context & context) { using namespace DB; Names column_names; column_names.push_back("a"); - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(getContext()); + QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); - BlockInputStreamPtr in = std::make_shared(std::move(table->read(column_names, {}, getContext(), stage, 8192, 1)[0])); + BlockInputStreamPtr in = std::make_shared(std::move(table->read(column_names, {}, context, stage, 8192, 1)[0])); Block sample; { @@ -123,7 +123,7 @@ std::string readData(DB::StoragePtr & table) std::ostringstream ss; WriteBufferFromOStream out_buf(ss); - BlockOutputStreamPtr output = FormatFactory::instance().getOutput("Values", out_buf, sample, getContext()); + BlockOutputStreamPtr output = FormatFactory::instance().getOutput("Values", out_buf, sample, context); copyData(*in, *output); @@ -135,15 +135,16 @@ std::string readData(DB::StoragePtr & table) TYPED_TEST(StorageLogTest, testReadWrite) { using namespace DB; + auto context_holder = getContext(); std::string data; // Write several chunks of data. - data += writeData(10, this->getTable()); + data += writeData(10, this->getTable(), context_holder.context); data += ","; - data += writeData(20, this->getTable()); + data += writeData(20, this->getTable(), context_holder.context); data += ","; - data += writeData(10, this->getTable()); + data += writeData(10, this->getTable(), context_holder.context); - ASSERT_EQ(data, readData(this->getTable())); + ASSERT_EQ(data, readData(this->getTable(), context_holder.context)); } diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 22407ca1bd9..303950258cb 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -18,7 +18,7 @@ using namespace DB; /// NOTE How to do better? struct State { - Context context = getContext(); + Context & context; NamesAndTypesList columns{ {"column", std::make_shared()}, {"apply_id", std::make_shared()}, @@ -27,7 +27,7 @@ struct State {"create_time", std::make_shared()}, }; - State() + State(Context & context_) : context(context_) { registerFunctions(); DatabasePtr database = std::make_shared("test"); @@ -38,12 +38,6 @@ struct State } }; -static State & state() -{ - static State res; - return res; -} - static void check(const std::string & query, const std::string & expected, const Context & context, const NamesAndTypesList & columns) { @@ -60,47 +54,62 @@ static void check(const std::string & query, const std::string & expected, const TEST(TransformQueryForExternalDatabase, InWithSingleElement) { + auto context_holder = getContext(); + State state(context_holder.context); + check("SELECT column FROM test.table WHERE 1 IN (1)", R"(SELECT "column" FROM "test"."table" WHERE 1)", - state().context, state().columns); + state.context, state.columns); check("SELECT column FROM test.table WHERE column IN (1, 2)", R"(SELECT "column" FROM "test"."table" WHERE "column" IN (1, 2))", - state().context, state().columns); + state.context, state.columns); check("SELECT column FROM test.table WHERE column NOT IN ('hello', 'world')", R"(SELECT "column" FROM "test"."table" WHERE "column" NOT IN ('hello', 'world'))", - state().context, state().columns); + state.context, state.columns); } TEST(TransformQueryForExternalDatabase, Like) { + auto context_holder = getContext(); + State state(context_holder.context); + check("SELECT column FROM test.table WHERE column LIKE '%hello%'", R"(SELECT "column" FROM "test"."table" WHERE "column" LIKE '%hello%')", - state().context, state().columns); + state.context, state.columns); check("SELECT column FROM test.table WHERE column NOT LIKE 'w%rld'", R"(SELECT "column" FROM "test"."table" WHERE "column" NOT LIKE 'w%rld')", - state().context, state().columns); + state.context, state.columns); } TEST(TransformQueryForExternalDatabase, Substring) { + auto context_holder = getContext(); + State state(context_holder.context); + check("SELECT column FROM test.table WHERE left(column, 10) = RIGHT(column, 10) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello'", R"(SELECT "column" FROM "test"."table")", - state().context, state().columns); + state.context, state.columns); } TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) { + auto context_holder = getContext(); + State state(context_holder.context); + check("SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(column, 10) = RIGHT(column, 10) AND column IN (1, 42) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello' AND column != 4", R"(SELECT "column" FROM "test"."table" WHERE 1 AND ("column" = 42) AND ("column" IN (1, 42)) AND ("column" != 4))", - state().context, state().columns); + state.context, state.columns); check("SELECT column FROM test.table WHERE toString(column) = '42' AND left(column, 10) = RIGHT(column, 10) AND column = 42", R"(SELECT "column" FROM "test"."table" WHERE ("column" = 42))", - state().context, state().columns); + state.context, state.columns); } TEST(TransformQueryForExternalDatabase, Issue7245) { + auto context_holder = getContext(); + State state(context_holder.context); + check("select apply_id from test.table where apply_type = 2 and create_time > addDays(toDateTime('2019-01-01 01:02:03'),-7) and apply_status in (3,4)", R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > '2018-12-25 01:02:03') AND ("apply_status" IN (3, 4)))", - state().context, state().columns); + state.context, state.columns); } From 32573c05909993f1ca40738c55eab0eebf472608 Mon Sep 17 00:00:00 2001 From: hcz Date: Mon, 20 Apr 2020 18:08:22 +0800 Subject: [PATCH 453/743] Add JSONExtractKeysAndValuesRaw --- src/Functions/FunctionsJSON.cpp | 1 + src/Functions/FunctionsJSON.h | 115 +++++++++++++++++++++++--------- 2 files changed, 83 insertions(+), 33 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 79dea768f61..e7598104102 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -21,6 +21,7 @@ void registerFunctionsJSON(FunctionFactory & factory) factory.registerFunction>(); factory.registerFunction>(); factory.registerFunction>(); + factory.registerFunction>(); } } diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index acd7c42732c..d456ad5cbe6 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -295,6 +295,7 @@ struct NameJSONExtract { static constexpr auto name{"JSONExtract"}; }; struct NameJSONExtractKeysAndValues { static constexpr auto name{"JSONExtractKeysAndValues"}; }; struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; struct NameJSONExtractArrayRaw { static constexpr auto name{"JSONExtractArrayRaw"}; }; +struct NameJSONExtractKeysAndValuesRaw { static constexpr auto name{"JSONExtractKeysAndValuesRaw"}; }; template @@ -699,8 +700,8 @@ struct JSONExtractTree if (!JSONParser::isArray(it)) return false; - Iterator it2 = it; - if (!JSONParser::firstArrayElement(it2)) + Iterator array_it = it; + if (!JSONParser::firstArrayElement(array_it)) return false; ColumnArray & col_arr = assert_cast(dest); @@ -710,12 +711,12 @@ struct JSONExtractTree do { - if (nested->addValueToColumn(data, it2)) + if (nested->addValueToColumn(data, array_it)) were_valid_elements = true; else data.insertDefault(); } - while (JSONParser::nextArrayElement(it2)); + while (JSONParser::nextArrayElement(array_it)); if (!were_valid_elements) { @@ -764,17 +765,17 @@ struct JSONExtractTree if (JSONParser::isArray(it)) { - Iterator it2 = it; - if (!JSONParser::firstArrayElement(it2)) + Iterator array_it = it; + if (!JSONParser::firstArrayElement(array_it)) return false; for (size_t index = 0; index != nested.size(); ++index) { - if (nested[index]->addValueToColumn(tuple.getColumn(index), it2)) + if (nested[index]->addValueToColumn(tuple.getColumn(index), array_it)) were_valid_elements = true; else tuple.getColumn(index).insertDefault(); - if (!JSONParser::nextArrayElement(it2)) + if (!JSONParser::nextArrayElement(array_it)) break; } @@ -786,25 +787,25 @@ struct JSONExtractTree { if (name_to_index_map.empty()) { - Iterator it2 = it; - if (!JSONParser::firstObjectMember(it2)) + Iterator object_it = it; + if (!JSONParser::firstObjectMember(object_it)) return false; for (size_t index = 0; index != nested.size(); ++index) { - if (nested[index]->addValueToColumn(tuple.getColumn(index), it2)) + if (nested[index]->addValueToColumn(tuple.getColumn(index), object_it)) were_valid_elements = true; else tuple.getColumn(index).insertDefault(); - if (!JSONParser::nextObjectMember(it2)) + if (!JSONParser::nextObjectMember(object_it)) break; } } else { - Iterator it2 = it; + Iterator object_it = it; StringRef key; - if (!JSONParser::firstObjectMember(it2, key)) + if (!JSONParser::firstObjectMember(object_it, key)) return false; do @@ -812,11 +813,11 @@ struct JSONExtractTree auto index = name_to_index_map.find(key); if (index != name_to_index_map.end()) { - if (nested[index->second]->addValueToColumn(tuple.getColumn(index->second), it2)) + if (nested[index->second]->addValueToColumn(tuple.getColumn(index->second), object_it)) were_valid_elements = true; } } - while (JSONParser::nextObjectMember(it2, key)); + while (JSONParser::nextObjectMember(object_it, key)); } set_size(old_size + static_cast(were_valid_elements)); @@ -875,6 +876,7 @@ struct JSONExtractTree } }; + template class JSONExtractImpl { @@ -930,8 +932,8 @@ public: + " should be a constant string specifying the values' data type, illegal value: " + col.column->getName(), ErrorCodes::ILLEGAL_COLUMN}; - DataTypePtr value_type = DataTypeFactory::instance().get(col_type_const->getValue()); DataTypePtr key_type = std::make_unique(); + DataTypePtr value_type = DataTypeFactory::instance().get(col_type_const->getValue()); DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, value_type}); return std::make_unique(tuple_type); } @@ -957,16 +959,16 @@ public: auto & col_value = col_tuple.getColumn(1); StringRef key; - Iterator it2 = it; - if (!JSONParser::firstObjectMember(it2, key)) + Iterator object_it = it; + if (!JSONParser::firstObjectMember(object_it, key)) return false; do { - if (extract_tree->addValueToColumn(col_value, it2)) + if (extract_tree->addValueToColumn(col_value, object_it)) col_key.insertData(key.data, key.size); } - while (JSONParser::nextObjectMember(it2, key)); + while (JSONParser::nextObjectMember(object_it, key)); if (col_tuple.size() == old_size) return false; @@ -1039,14 +1041,14 @@ private: if (JSONParser::isArray(it)) { writeChar('[', buf); - Iterator it2 = it; - if (JSONParser::firstArrayElement(it2)) + Iterator array_it = it; + if (JSONParser::firstArrayElement(array_it)) { - traverse(it2, buf); - while (JSONParser::nextArrayElement(it2)) + traverse(array_it, buf); + while (JSONParser::nextArrayElement(array_it)) { writeChar(',', buf); - traverse(it2, buf); + traverse(array_it, buf); } } writeChar(']', buf); @@ -1055,19 +1057,19 @@ private: if (JSONParser::isObject(it)) { writeChar('{', buf); - Iterator it2 = it; + Iterator object_it = it; StringRef key; - if (JSONParser::firstObjectMember(it2, key)) + if (JSONParser::firstObjectMember(object_it, key)) { writeJSONString(key, buf, format_settings()); writeChar(':', buf); - traverse(it2, buf); - while (JSONParser::nextObjectMember(it2, key)) + traverse(object_it, buf); + while (JSONParser::nextObjectMember(object_it, key)) { writeChar(',', buf); writeJSONString(key, buf, format_settings()); writeChar(':', buf); - traverse(it2, buf); + traverse(object_it, buf); } } writeChar('}', buf); @@ -1092,6 +1094,7 @@ private: } }; + template class JSONExtractArrayRawImpl { @@ -1105,9 +1108,8 @@ public: static bool addValueToColumn(IColumn & dest, const Iterator & it) { if (!JSONParser::isArray(it)) - { return false; - } + ColumnArray & col_res = assert_cast(dest); Iterator array_it = it; size_t size = 0; @@ -1127,4 +1129,51 @@ public: static constexpr size_t num_extra_arguments = 0; static void prepare(const char *, const Block &, const ColumnNumbers &, size_t) {} }; + + +template +class JSONExtractKeysAndValuesRawImpl +{ +public: + + static DataTypePtr getType(const char *, const ColumnsWithTypeAndName &) + { + DataTypePtr key_type = std::make_unique(); + DataTypePtr value_type = std::make_unique(); + DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, value_type}); + return std::make_unique(tuple_type); + } + + using Iterator = typename JSONParser::Iterator; + bool addValueToColumn(IColumn & dest, const Iterator & it) + { + if (!JSONParser::isObject(it)) + return false; + + auto & col_arr = assert_cast(dest); + auto & col_tuple = assert_cast(col_arr.getData()); + auto & col_key = assert_cast(col_tuple.getColumn(0)); + auto & col_value = assert_cast(col_tuple.getColumn(1)); + + Iterator object_it = it; + StringRef key; + size_t size = 0; + if (JSONParser::firstObjectMember(object_it, key)) + { + do + { + col_key.insertData(key.data, key.size); + JSONExtractRawImpl::addValueToColumn(col_value, object_it); + ++size; + } while (JSONParser::nextObjectMember(object_it, key)); + } + + col_arr.getOffsets().push_back(col_arr.getOffsets().back() + size); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const char *, const Block &, const ColumnNumbers &, size_t) {} +}; + } From 309a20caf0589b64ba1866e7bb8dcf1b7c2470bc Mon Sep 17 00:00:00 2001 From: hcz Date: Mon, 20 Apr 2020 18:08:39 +0800 Subject: [PATCH 454/743] Add tests and docs --- .../sql_reference/functions/json_functions.md | 18 ++++++-- .../00918_json_functions.reference | 41 +++++++++++++++---- .../0_stateless/00918_json_functions.sql | 36 ++++++++++++++-- 3 files changed, 79 insertions(+), 16 deletions(-) diff --git a/docs/en/sql_reference/functions/json_functions.md b/docs/en/sql_reference/functions/json_functions.md index f1a6b6cc44a..ca1690130e6 100644 --- a/docs/en/sql_reference/functions/json_functions.md +++ b/docs/en/sql_reference/functions/json_functions.md @@ -194,17 +194,17 @@ SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \ ## JSONExtractKeysAndValues(json\[, indices\_or\_keys…\], value\_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} -Parse key-value pairs from a JSON where the values are of the given ClickHouse data type. +Parses key-value pairs from a JSON where the values are of the given ClickHouse data type. Example: ``` sql -SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)]; +SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)] ``` ## JSONExtractRaw(json\[, indices\_or\_keys\]…) {#jsonextractrawjson-indices-or-keys} -Returns a part of JSON. +Returns a part of JSON as unparsed string. If the part does not exist or has a wrong type, an empty string will be returned. @@ -214,7 +214,7 @@ Example: SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' ``` -## JSONExtractArrayRaw(json\[, indices\_or\_keys\]…) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices\_or\_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} Returns an array with elements of JSON array, each represented as unparsed string. @@ -227,3 +227,13 @@ SELECT JSONExtractArrayRaw('{"a": "hello", "b": [-100, 200.0, "hello"]}', 'b') = ``` [Original article](https://clickhouse.tech/docs/en/query_language/functions/json_functions/) + +## JSONExtractKeysAndValuesRaw(json\[, indices\_or\_keys…\]) {#jsonextractkeysandvaluesrawjson-indices-or-keys} + +Parses key-value pairs from a JSON and returns an array of such pairs, each value represented as unparsed string. + +Example: + +``` sql +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}') = [('a','"hello"'),('b','[-100,200,300]')] +``` diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index 32cde7bbfb4..b83cbe17a6e 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -76,6 +76,21 @@ Friday {} "\\n\\u0000" "☺" +--JSONExtractArrayRaw-- +[] +[] +[] +['[]','[]'] +['-100','200','300'] +['1','2','3','4','5','"hello"'] +['1','2','3'] +['4','5','6'] +--JSONExtractKeysAndValuesRaw-- +[] +[] +[('a','"hello"'),('b','[-100,200,300]')] +[('a','"hello"'),('b','[-100,200,300]'),('c','{"d":[121,144]}')] +[('d','[121,144]')] --const/non-const mixed-- a b @@ -93,6 +108,10 @@ v 1 1 0 +--isValidJSON-- +1 +0 +0 --JSONKey-- a b @@ -158,14 +177,6 @@ Friday {} "\\n\\u0000" "☺" ---const/non-const mixed-- -a -b -c -d -e -u -v --JSONExtractArrayRaw-- [] [] @@ -175,3 +186,17 @@ v ['1','2','3','4','5','"hello"'] ['1','2','3'] ['4','5','6'] +--JSONExtractKeysAndValuesRaw-- +[] +[] +[('a','"hello"'),('b','[-100,200,300]')] +[('a','"hello"'),('b','[-100,200,300]'),('c','{"d":[121,144]}')] +[('d','[121,144]')] +--const/non-const mixed-- +a +b +c +d +e +u +v diff --git a/tests/queries/0_stateless/00918_json_functions.sql b/tests/queries/0_stateless/00918_json_functions.sql index 0db9540377e..7a030d3dab6 100644 --- a/tests/queries/0_stateless/00918_json_functions.sql +++ b/tests/queries/0_stateless/00918_json_functions.sql @@ -88,6 +88,22 @@ SELECT JSONExtractRaw('{}'); SELECT JSONExtractRaw('{"abc":"\\n\\u0000"}', 'abc'); SELECT JSONExtractRaw('{"abc":"\\u263a"}', 'abc'); +SELECT '--JSONExtractArrayRaw--'; +SELECT JSONExtractArrayRaw(''); +SELECT JSONExtractArrayRaw('{"a": "hello", "b": "not_array"}'); +SELECT JSONExtractArrayRaw('[]'); +SELECT JSONExtractArrayRaw('[[],[]]'); +SELECT JSONExtractArrayRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +SELECT JSONExtractArrayRaw('[1,2,3,4,5,"hello"]'); +SELECT JSONExtractArrayRaw(arrayJoin(JSONExtractArrayRaw('[[1,2,3],[4,5,6]]'))); + +SELECT '--JSONExtractKeysAndValuesRaw--'; +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c'); + SELECT '--const/non-const mixed--'; SELECT JSONExtractString('["a", "b", "c", "d", "e"]', idx) FROM (SELECT arrayJoin([1,2,3,4,5]) AS idx); SELECT JSONExtractString(json, 's') FROM (SELECT arrayJoin(['{"s":"u"}', '{"s":"v"}']) AS json); @@ -107,6 +123,11 @@ SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'c'); +SELECT '--isValidJSON--'; +SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}'); +SELECT isValidJSON('not a json'); +SELECT isValidJSON('"HX-='); + SELECT '--JSONKey--'; SELECT JSONKey('{"a": "hello", "b": [-100, 200.0, 300]}', 1); SELECT JSONKey('{"a": "hello", "b": [-100, 200.0, 300]}', 2); @@ -179,10 +200,6 @@ SELECT JSONExtractRaw('{}'); SELECT JSONExtractRaw('{"abc":"\\n\\u0000"}', 'abc'); SELECT JSONExtractRaw('{"abc":"\\u263a"}', 'abc'); -SELECT '--const/non-const mixed--'; -SELECT JSONExtractString('["a", "b", "c", "d", "e"]', idx) FROM (SELECT arrayJoin([1,2,3,4,5]) AS idx); -SELECT JSONExtractString(json, 's') FROM (SELECT arrayJoin(['{"s":"u"}', '{"s":"v"}']) AS json); - SELECT '--JSONExtractArrayRaw--'; SELECT JSONExtractArrayRaw(''); SELECT JSONExtractArrayRaw('{"a": "hello", "b": "not_array"}'); @@ -191,3 +208,14 @@ SELECT JSONExtractArrayRaw('[[],[]]'); SELECT JSONExtractArrayRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); SELECT JSONExtractArrayRaw('[1,2,3,4,5,"hello"]'); SELECT JSONExtractArrayRaw(arrayJoin(JSONExtractArrayRaw('[[1,2,3],[4,5,6]]'))); + +SELECT '--JSONExtractKeysAndValuesRaw--'; +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c'); + +SELECT '--const/non-const mixed--'; +SELECT JSONExtractString('["a", "b", "c", "d", "e"]', idx) FROM (SELECT arrayJoin([1,2,3,4,5]) AS idx); +SELECT JSONExtractString(json, 's') FROM (SELECT arrayJoin(['{"s":"u"}', '{"s":"v"}']) AS json); From c2c99b02bce0911c2b4b4981c4246cfdaa9e0f11 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Apr 2020 13:36:38 +0300 Subject: [PATCH 455/743] Fix prepare for ConcatProcessor. --- src/Processors/ConcatProcessor.cpp | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Processors/ConcatProcessor.cpp b/src/Processors/ConcatProcessor.cpp index d3333c320c0..97215701511 100644 --- a/src/Processors/ConcatProcessor.cpp +++ b/src/Processors/ConcatProcessor.cpp @@ -31,17 +31,13 @@ ConcatProcessor::Status ConcatProcessor::prepare() /// Check can input. - if (current_input == inputs.end()) - return Status::Finished; - - if (current_input->isFinished()) - { + while (current_input != inputs.end() && !current_input->isFinished()) ++current_input; - if (current_input == inputs.end()) - { - output.finish(); - return Status::Finished; - } + + if (current_input == inputs.end()) + { + output.finish(); + return Status::Finished; } auto & input = *current_input; From 8767ea7efa8a02cfd0a0055c590319649bc4315b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 Apr 2020 13:53:29 +0300 Subject: [PATCH 456/743] Add setting which restrict replica to merge parts by itself and always prefer dowloading from other replicas. --- src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 9 ++- .../test_always_fetch_merged/__init__.py | 0 .../test_always_fetch_merged/test.py | 64 +++++++++++++++++++ 4 files changed, 73 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_always_fetch_merged/__init__.py create mode 100644 tests/integration/test_always_fetch_merged/test.py diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 02c852b4f4b..dfe3643e60c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -57,6 +57,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, min_replicated_logs_to_keep, 100, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ M(SettingSeconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ M(SettingUInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + M(SettingBool, always_fetch_merged_part, 0, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \ M(SettingUInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \ M(SettingUInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \ M(SettingUInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 579ccc1bd1e..61ca11a5eac 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -985,6 +985,14 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) LOG_TRACE(log, log_message.rdbuf()); } + const auto storage_settings_ptr = getSettings(); + + if (storage_settings_ptr->always_fetch_merged_part) + { + LOG_INFO(log, "Will fetch part " << entry.new_part_name << " because setting always_fetch_merged_part is set to 1"); + return false; + } + DataPartsVector parts; bool have_all_parts = true; for (const String & name : entry.source_parts) @@ -1005,7 +1013,6 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) parts.push_back(part); } - const auto storage_settings_ptr = getSettings(); if (!have_all_parts) { /// If you do not have all the necessary parts, try to take some already merged part from someone. diff --git a/tests/integration/test_always_fetch_merged/__init__.py b/tests/integration/test_always_fetch_merged/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_always_fetch_merged/test.py b/tests/integration/test_always_fetch_merged/test.py new file mode 100644 index 00000000000..846f0c29a20 --- /dev/null +++ b/tests/integration/test_always_fetch_merged/test.py @@ -0,0 +1,64 @@ +import pytest +import time +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', with_zookeeper=True) +node2 = cluster.add_instance('node2', with_zookeeper=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_replica_always_download(started_cluster): + node1.query(""" + CREATE TABLE test_table( + key UInt64, + value String + ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_table/replicated', '1') + ORDER BY tuple() + """) + node2.query(""" + CREATE TABLE test_table( + key UInt64, + value String + ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_table/replicated', '2') + ORDER BY tuple() + SETTINGS always_fetch_merged_part=1 + """) + + # Stop merges on single node + node1.query("SYSTEM STOP MERGES") + + for i in range(0, 10): + node1.query("INSERT INTO test_table VALUES ({}, '{}')".format(i, i)) + + assert node1.query("SELECT COUNT() FROM test_table") == "10\n" + assert_eq_with_retry(node2, "SELECT COUNT() FROM test_table", "10\n") + + time.sleep(3) + + # Nothing is merged + assert node1.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1") == "10\n" + + assert node2.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1") == "10\n" + + node1.query("SYSTEM START MERGES") + + time.sleep(3) + + # all merged + node1_parts = node1.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1").strip() + node2_parts = node2.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1").strip() + assert node1_parts < '10' # something merged + assert node2_parts < '10' From 1bc84ec8b418d4b2053ec452c0ec47ebaea4d82a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 Apr 2020 13:56:23 +0300 Subject: [PATCH 457/743] Fix comments --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- tests/integration/test_always_fetch_merged/test.py | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 61ca11a5eac..2a0dac0fe4c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -989,7 +989,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) if (storage_settings_ptr->always_fetch_merged_part) { - LOG_INFO(log, "Will fetch part " << entry.new_part_name << " because setting always_fetch_merged_part is set to 1"); + LOG_INFO(log, "Will fetch part " << entry.new_part_name << " because setting 'always_fetch_merged_part' is true"); return false; } diff --git a/tests/integration/test_always_fetch_merged/test.py b/tests/integration/test_always_fetch_merged/test.py index 846f0c29a20..8301110f401 100644 --- a/tests/integration/test_always_fetch_merged/test.py +++ b/tests/integration/test_always_fetch_merged/test.py @@ -57,7 +57,6 @@ def test_replica_always_download(started_cluster): time.sleep(3) - # all merged node1_parts = node1.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1").strip() node2_parts = node2.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1").strip() assert node1_parts < '10' # something merged From 05206df88cb635010200c4f21eecd87ff97ad696 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 20 Apr 2020 13:56:59 +0300 Subject: [PATCH 458/743] better --- src/Storages/StorageReplicatedMergeTree.cpp | 7 ++----- src/Storages/StorageReplicatedMergeTree.h | 3 +-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 82f0c113109..c52540cefd3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2695,7 +2695,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } -void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id, const String & part_name) +void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) { auto zookeeper = getZooKeeper(); @@ -2721,10 +2721,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id, break; } - if (part_name.empty() || parts_with_quorum.added_parts[partition_id] == part_name) - parts_with_quorum.added_parts.erase(partition_id); - else - break; + parts_with_quorum.added_parts.erase(partition_id); String new_added_parts = parts_with_quorum.toString(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2846e731926..17b454f2757 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -465,8 +465,7 @@ private: void updateQuorum(const String & part_name); /// Deletes info from quorum/last_part node for particular partition_id. - /// If part_name if specified, deletes only if last_part == part_name. - void cleanLastPartNode(const String & partition_id, const String & part_name = ""); + void cleanLastPartNode(const String & partition_id); /// Creates new block number if block with such block_id does not exist std::optional allocateBlockNumber( From 3dbc9b556216786874a29fbcf937e9da5dff7911 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Apr 2020 14:03:29 +0300 Subject: [PATCH 459/743] Fix prepare for ConcatProcessor. --- src/Processors/ConcatProcessor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/ConcatProcessor.cpp b/src/Processors/ConcatProcessor.cpp index 97215701511..27338c7c879 100644 --- a/src/Processors/ConcatProcessor.cpp +++ b/src/Processors/ConcatProcessor.cpp @@ -31,7 +31,7 @@ ConcatProcessor::Status ConcatProcessor::prepare() /// Check can input. - while (current_input != inputs.end() && !current_input->isFinished()) + while (current_input != inputs.end() && current_input->isFinished()) ++current_input; if (current_input == inputs.end()) From 6c319850ae2a2d6e95fd52e167e96b1a2aea6b91 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Apr 2020 14:05:24 +0300 Subject: [PATCH 460/743] Update tests. --- .../00184_shard_distributed_group_by_no_merge.reference | 4 ++++ .../0_stateless/00184_shard_distributed_group_by_no_merge.sql | 1 + 2 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference index 1957f3a9604..e3dbeb81c7c 100644 --- a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference +++ b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.reference @@ -1,2 +1,6 @@ 1 1 1 1 +1 1 +1 1 +1 1 +1 1 diff --git a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql index 7b7a0914c0c..78f1fb68385 100644 --- a/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql +++ b/tests/queries/0_stateless/00184_shard_distributed_group_by_no_merge.sql @@ -1 +1,2 @@ SELECT count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one) SETTINGS distributed_group_by_no_merge = 1; +SELECT count(), uniq(dummy) FROM remote('127.0.0.{2,3,4,5}', system.one) SETTINGS distributed_group_by_no_merge = 1; From 82985ff594a9b6d88260b1c96f84833aa999601b Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 20 Apr 2020 14:15:06 +0300 Subject: [PATCH 461/743] Allowed to alter column in non-modifying data mode when the same type is specified. --- src/Storages/AlterCommands.cpp | 19 +++- src/Storages/AlterCommands.h | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../integration/test_alter_codec/__init__.py | 0 .../configs/config.d/cluster.xml | 16 ++++ .../test_alter_codec/configs/logs_config.xml | 17 ++++ tests/integration/test_alter_codec/test.py | 88 +++++++++++++++++++ 7 files changed, 139 insertions(+), 7 deletions(-) create mode 100644 tests/integration/test_alter_codec/__init__.py create mode 100644 tests/integration/test_alter_codec/configs/config.d/cluster.xml create mode 100644 tests/integration/test_alter_codec/configs/logs_config.xml create mode 100644 tests/integration/test_alter_codec/test.py diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 8eebaabf342..c194d14fd65 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -472,11 +472,22 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } -bool AlterCommand::isModifyingData() const +bool AlterCommand::isModifyingData(const StorageInMemoryMetadata & metadata) const { /// Possible change data representation on disk if (type == MODIFY_COLUMN) - return data_type != nullptr; + { + if (data_type == nullptr) + return false; + + for (const auto & column : metadata.columns.getAllPhysical()) + { + if (column.name == column_name) + return column.type->getName() != data_type->getName(); + } + + return true; + } return type == ADD_COLUMN /// We need to change columns.txt in each part for MergeTree || type == DROP_COLUMN /// We need to change columns.txt in each part for MergeTree @@ -888,11 +899,11 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context); } -bool AlterCommands::isModifyingData() const +bool AlterCommands::isModifyingData(const StorageInMemoryMetadata & metadata) const { for (const auto & param : *this) { - if (param.isModifyingData()) + if (param.isModifyingData(metadata)) return true; } diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index be27ba6ac2b..4e40c1ee190 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -108,7 +108,7 @@ struct AlterCommand /// * column files (data and marks) /// * each part meta (columns.txt) /// in each part on disk (it's not lightweight alter). - bool isModifyingData() const; + bool isModifyingData(const StorageInMemoryMetadata & metadata) const; bool isRequireMutationStage(const StorageInMemoryMetadata & metadata) const; @@ -151,7 +151,7 @@ public: void apply(StorageInMemoryMetadata & metadata) const; /// At least one command modify data on disk. - bool isModifyingData() const; + bool isModifyingData(const StorageInMemoryMetadata & metadata) const; /// At least one command modify settings. bool isSettingsAlter() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index aaacea6f1e7..0795e020a35 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1484,7 +1484,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S ErrorCodes::ILLEGAL_COLUMN); } } - else if (command.isModifyingData()) + else if (command.isModifyingData(getInMemoryMetadata())) { if (columns_alter_type_forbidden.count(command.column_name)) throw Exception("Trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN); diff --git a/tests/integration/test_alter_codec/__init__.py b/tests/integration/test_alter_codec/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_alter_codec/configs/config.d/cluster.xml b/tests/integration/test_alter_codec/configs/config.d/cluster.xml new file mode 100644 index 00000000000..ec7c9b8e4f8 --- /dev/null +++ b/tests/integration/test_alter_codec/configs/config.d/cluster.xml @@ -0,0 +1,16 @@ + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + \ No newline at end of file diff --git a/tests/integration/test_alter_codec/configs/logs_config.xml b/tests/integration/test_alter_codec/configs/logs_config.xml new file mode 100644 index 00000000000..bdf1bbc11c1 --- /dev/null +++ b/tests/integration/test_alter_codec/configs/logs_config.xml @@ -0,0 +1,17 @@ + + 3 + + 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 + + + system + part_log
      + 500 +
      +
      diff --git a/tests/integration/test_alter_codec/test.py b/tests/integration/test_alter_codec/test.py new file mode 100644 index 00000000000..573fe7f0961 --- /dev/null +++ b/tests/integration/test_alter_codec/test.py @@ -0,0 +1,88 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', + config_dir='configs', + main_configs=['configs/logs_config.xml'], + with_zookeeper=True, + macros={"shard": 0, "replica": 1} ) + +node2 = cluster.add_instance('node2', + config_dir='configs', + main_configs=['configs/logs_config.xml'], + with_zookeeper=True, + macros={"shard": 0, "replica": 2} ) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_alter_codec_pk(started_cluster): + try: + name = "test_alter_codec_pk" + node1.query(""" + CREATE TABLE {name} (id UInt64) Engine=MergeTree() ORDER BY id + """.format(name=name)) + + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(NONE)".format(name=name)) + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(Delta, LZ4)".format(name=name)) + + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt32 CODEC(Delta, LZ4)".format(name=name)) + + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 DEFAULT 3 CODEC(Delta, LZ4)".format(name=name)) + + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 ALIAS 3 CODEC(Delta, LZ4)".format(name=name)) + + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name)) + + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64".format(name=name)) + + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {name} MODIFY COLUMN id Int64".format(name=name)) + + finally: + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + + +def test_alter_codec_index(started_cluster): + try: + name = "test_alter_codec_index" + node1.query(""" + CREATE TABLE {name} (`id` UInt64, INDEX id_index id TYPE minmax GRANULARITY 1) Engine=MergeTree() ORDER BY tuple() + """.format(name=name)) + + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(NONE)".format(name=name)) + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(Delta, LZ4)".format(name=name)) + + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt32 CODEC(Delta, LZ4)".format(name=name)) + + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 DEFAULT 3 CODEC(Delta, LZ4)".format(name=name)) + + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 ALIAS 3 CODEC(Delta, LZ4)".format(name=name)) + + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name)) + + node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64".format(name=name)) + + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {name} MODIFY COLUMN id Int64".format(name=name)) + + finally: + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) From d5b0a7e691afd4ddd4dcd56a4a494d7abed89e25 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Mon, 20 Apr 2020 15:21:25 +0300 Subject: [PATCH 462/743] Fix Arcadia build (#10380) --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 5 ++++- src/Common/ya.make | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 2fba10b20e9..9c2167fa49e 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -11,7 +11,10 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_NETSSL #include #endif diff --git a/src/Common/ya.make b/src/Common/ya.make index eece911d2b8..4dcce48487d 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -13,6 +13,7 @@ PEERDIR( clickhouse/base/widechar_width contrib/libs/libcpuid/libcpuid contrib/libs/openssl + contrib/libs/poco/NetSSL_OpenSSL contrib/libs/re2 contrib/restricted/ryu ) From 52dfd107853b62cad28d7928f570f4f9cbf11893 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 20 Apr 2020 15:23:33 +0300 Subject: [PATCH 463/743] Delete convert_toc.py --- docs/tools/convert_toc.py | 142 -------------------------------------- 1 file changed, 142 deletions(-) delete mode 100755 docs/tools/convert_toc.py diff --git a/docs/tools/convert_toc.py b/docs/tools/convert_toc.py deleted file mode 100755 index 5e3fe97de44..00000000000 --- a/docs/tools/convert_toc.py +++ /dev/null @@ -1,142 +0,0 @@ -#!/usr/bin/env python3 - -import os -import sys -import subprocess - -import yaml - -import util - -lang = 'ru' -base_dir = os.path.join(os.path.dirname(__file__), '..') -en_dir = os.path.join(base_dir, 'en') -docs_dir = os.path.join(base_dir, lang) -redirects_file = os.path.join(base_dir, 'redirects.txt') -redirects = {} - - -def make_key(path): - path = path.lower().replace(' ', '_').replace('c++', 'cpp') - if path.startswith('clickhouse_'): - path = path.replace('clickhouse_', '', 1) - if path != 'data_types' and path.startswith('data_'): - path = path.replace('data_', '', 1) - to_remove = [ - '\'', '"', '.', ',', '(', ')', - 'how_to_', '_of_clickhouse' - ] - for token in to_remove: - path = path.replace(token, '') - return path - - -def process_md_file(title, idx, original_path, proper_path): - proper_md_path = '/'.join(proper_path + [original_path.rsplit('/', 1)[-1]]) - if proper_md_path == 'introduction/index.md': - proper_md_path = 'index.md' - print(locals()) - if original_path != proper_md_path: - redirects[original_path] = proper_md_path - original_path = os.path.join(docs_dir, original_path) - proper_md_path = os.path.join(docs_dir, proper_md_path) - if os.path.exists(original_path): - meta, content = util.read_md_file(original_path) - else: - meta, content = util.read_md_file(proper_md_path) - meta['toc_title'] = title - meta['toc_priority'] = idx - if title == 'hidden': - meta['toc_hidden'] = True - - for src, dst in redirects.items(): - content = content.replace('(' + src, '(' + dst) - content = content.replace('../' + src, '../' + dst) - - util.write_md_file(proper_md_path, meta, content) - if original_path != proper_md_path: - subprocess.check_call(f'git add {proper_md_path}', shell=True) - if os.path.exists(original_path): - subprocess.check_call(f'rm {original_path}', shell=True) - - -def process_toc_entry(entry, path, idx): - if isinstance(entry, list): - for e in entry: - idx = process_toc_entry(e, path, idx) - elif isinstance(entry, dict): - for key, value in entry.items(): - next_path = path + [make_key(key)] - index_md_idx = idx - idx += 1 - - if isinstance(value, list): - for v in value: - process_toc_entry(v, next_path, idx) - idx += 1 - else: - process_md_file(key, idx, value, path) - idx += 1 - - index_md_path = os.path.join(docs_dir, '/'.join(next_path), 'index.md') - if os.path.exists(os.path.dirname(index_md_path)): - index_meta, index_content = util.read_md_file(index_md_path) - if not index_meta.get('toc_folder_title'): - index_meta['toc_folder_title'] = key - index_meta['toc_priority'] = index_md_idx - util.write_md_file(index_md_path, index_meta, index_content) - subprocess.check_call(f'git add {index_md_path}', shell=True) - return idx - - -def process_toc_yaml(path): - with util.cd(docs_dir): - init_redirects() - with open(path, 'r') as f: - data = yaml.full_load(f.read()) - process_toc_entry(data['nav'], [], 1) - update_redirects() - - -def init_redirects(): - with open(redirects_file, 'r') as f: - for line in f: - src, dst = line.strip().split(' ', 1) - redirects[src] = dst - - -def update_redirects(): - with open(redirects_file, 'w') as f: - for src, dst in sorted(redirects.items()): - print(f'{src} {dst}', file=f) - - -def sync_translation(): - init_redirects() - for src, dst in redirects.items(): - en_src = os.path.join(en_dir, src) - lang_src = os.path.join(docs_dir, src) - lang_dst = os.path.join(docs_dir, dst) - if os.path.exists(lang_src): - if os.path.islink(lang_src): - pass - else: - en_meta, en_content = util.read_md_file(en_src) - lang_meta, lang_content = util.read_md_file(lang_src) - en_meta.update(lang_meta) - - for src_link, dst_link in redirects.items(): - lang_content = lang_content.replace('(' + src_link, '(' + dst) - lang_content = lang_content.replace('../' + src_link, '../' + dst) - - util.write_md_file(lang_dst, en_meta, lang_content) - subprocess.check_call(f'git add {lang_dst}', shell=True) - subprocess.check_call(f'rm {lang_src}', shell=True) - - -if __name__ == '__main__': - sync_translation() - # if len(sys.argv) == 1: - # process_toc_yaml(os.path.join(base_dir, f'toc_{lang}.yml')) - # else: - # process_toc_yaml(sys.argv[1]) From b58fa8585cda86978d63a15514b755bd92897f89 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 20 Apr 2020 15:52:12 +0300 Subject: [PATCH 464/743] Update website.py --- docs/tools/website.py | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/docs/tools/website.py b/docs/tools/website.py index 83eef270fc5..bf579aa28bb 100644 --- a/docs/tools/website.py +++ b/docs/tools/website.py @@ -13,21 +13,6 @@ import jsmin import mdx_clickhouse -def copy_icons(args): - logging.info('Copying icons') - icons_dir = os.path.join(args.output_dir, 'images', 'icons') - os.makedirs(icons_dir) - for icon in [ - 'github', - 'edit', - 'external-link' - ]: - icon = '%s.svg' % icon - icon_src = os.path.join(args.website_dir, 'images', 'feathericons', 'icons', icon) - icon_dst = os.path.join(icons_dir, icon) - shutil.copy2(icon_src, icon_dst) - - def build_website(args): logging.info('Building website') env = jinja2.Environment( From 0afd36193387fcb7e8d57a1a8d0160d1859a5f91 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 Apr 2020 17:19:55 +0300 Subject: [PATCH 465/743] Fix subprocess call in clickhouse-test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 81a3275c218..26fef8cb476 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -110,7 +110,7 @@ def get_stacktraces_from_gdb(server_pid): # collect server stacktraces from system.stack_trace table def get_stacktraces_from_clickhouse(client): try: - return subprocess.check_call("{} --allow_introspection_functions=1 --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), shell=True) + return subprocess.check_output("{} --allow_introspection_functions=1 --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), shell=True) except Exception as ex: return "Error occured while receiving stack traces from client: {}".format(str(ex)) From 672c952d39786fa1a97865e2df01baec8bd762ba Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 20 Apr 2020 17:44:21 +0300 Subject: [PATCH 466/743] Update test.py --- tests/integration/test_row_policy/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 3a5b7340528..e5b05c24a55 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -101,9 +101,9 @@ def test_join(): def test_cannot_trick_row_policy_with_keyword_with(): - assert instance.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1") == "1\t0\n1\t1\n" - assert instance.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1") == "1\t0\n1\t1\n" - assert instance.query("WITH 0 AS a SELECT a FROM mydb.filtered_table1") == "1\n1\n" + assert instance.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1") == "0\t0\n0\t1\n" + assert instance.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1") == "0\t0\n0\t1\n" + assert instance.query("WITH 0 AS a SELECT a FROM mydb.filtered_table1") == "0\n0\n" assert instance.query("WITH 0 AS a SELECT b FROM mydb.filtered_table1") == "0\n1\n" From e1fa39d801e21a80bcf63ddffebb1fb8b119d3c0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 Apr 2020 18:17:34 +0300 Subject: [PATCH 467/743] fix test --- tests/integration/test_always_fetch_merged/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_always_fetch_merged/test.py b/tests/integration/test_always_fetch_merged/test.py index 8301110f401..63ab6f6b5ea 100644 --- a/tests/integration/test_always_fetch_merged/test.py +++ b/tests/integration/test_always_fetch_merged/test.py @@ -59,5 +59,5 @@ def test_replica_always_download(started_cluster): node1_parts = node1.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1").strip() node2_parts = node2.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1").strip() - assert node1_parts < '10' # something merged - assert node2_parts < '10' + assert int(node1_parts) < 10 # something merged + assert int(node2_parts) < 10 From 9d83a9dc3d6d762d306295a81d9ee33997ffa369 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 20 Apr 2020 18:31:56 +0300 Subject: [PATCH 468/743] Fix "en" changelog build (#10377) * fix docs build * some markdown fixes * more fixes * fix changelog build --- CHANGELOG.md | 6 + docs/en/whats_new/changelog/index.md | 651 +-------------------------- 2 files changed, 7 insertions(+), 650 deletions(-) mode change 100644 => 120000 docs/en/whats_new/changelog/index.md diff --git a/CHANGELOG.md b/CHANGELOG.md index e5d1c90bf22..8375f4ba152 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,9 @@ +--- +toc_folder_title: Changelog +toc_priority: 74 +toc_title: '2020' +--- + ## ClickHouse release v20.3 ### ClickHouse release v20.3.7.46, 2020-04-17 diff --git a/docs/en/whats_new/changelog/index.md b/docs/en/whats_new/changelog/index.md deleted file mode 100644 index 969e71fbf42..00000000000 --- a/docs/en/whats_new/changelog/index.md +++ /dev/null @@ -1,650 +0,0 @@ ---- -toc_folder_title: Changelog -toc_priority: 74 -toc_title: '2020' ---- - -## ClickHouse release v20.3 - -### ClickHouse release v20.3.4.10, 2020-03-20 - -#### Bug Fix -* This release also contains all bug fixes from 20.1.8.41 -* Fix missing `rows_before_limit_at_least` for queries over http (with processors pipeline). This fixes [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730). [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - - -### ClickHouse release v20.3.3.6, 2020-03-17 - -#### Bug Fix -* This release also contains all bug fixes from 20.1.7.38 -* Fix bug in a replication that doesn't allow replication to work if the user has executed mutations on the previous version. This fixes [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again. -* Add setting `use_compact_format_in_distributed_parts_names` which allows to write files for `INSERT` queries into `Distributed` table with more compact format. This fixes [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again. - -### ClickHouse release v20.3.2.1, 2020-03-12 - -#### Backward Incompatible Change - -* Fixed the issue `file name too long` when sending data for `Distributed` tables for a large number of replicas. Fixed the issue that replica credentials were exposed in the server log. The format of directory name on disk was changed to `[shard{shard_index}[_replica{replica_index}]]`. [#8911](https://github.com/ClickHouse/ClickHouse/pull/8911) ([Mikhail Korotov](https://github.com/millb)) After you upgrade to the new version, you will not be able to downgrade without manual intervention, because old server version does not recognize the new directory format. If you want to downgrade, you have to manually rename the corresponding directories to the old format. This change is relevant only if you have used asynchronous `INSERT`s to `Distributed` tables. In the version 20.3.3 we will introduce a setting that will allow you to enable the new format gradually. -* Changed the format of replication log entries for mutation commands. You have to wait for old mutations to process before installing the new version. -* Implement simple memory profiler that dumps stacktraces to `system.trace_log` every N bytes over soft allocation limit [#8765](https://github.com/ClickHouse/ClickHouse/pull/8765) ([Ivan](https://github.com/abyss7)) [#9472](https://github.com/ClickHouse/ClickHouse/pull/9472) ([alexey-milovidov](https://github.com/alexey-milovidov)) The column of `system.trace_log` was renamed from `timer_type` to `trace_type`. This will require changes in third-party performance analysis and flamegraph processing tools. -* Use OS thread id everywhere instead of internal thread number. This fixes [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477) Old `clickhouse-client` cannot receive logs that are send from the server when the setting `send_logs_level` is enabled, because the names and types of the structured log messages were changed. On the other hand, different server versions can send logs with different types to each other. When you don't use the `send_logs_level` setting, you should not care. [#8954](https://github.com/ClickHouse/ClickHouse/pull/8954) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove `indexHint` function [#9542](https://github.com/ClickHouse/ClickHouse/pull/9542) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove `findClusterIndex`, `findClusterValue` functions. This fixes [#8641](https://github.com/ClickHouse/ClickHouse/issues/8641). If you were using these functions, send an email to `clickhouse-feedback@yandex-team.com` [#9543](https://github.com/ClickHouse/ClickHouse/pull/9543) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now it's not allowed to create columns or add columns with `SELECT` subquery as default expression. [#9481](https://github.com/ClickHouse/ClickHouse/pull/9481) ([alesapin](https://github.com/alesapin)) -* Require aliases for subqueries in JOIN. [#9274](https://github.com/ClickHouse/ClickHouse/pull/9274) ([Artem Zuikov](https://github.com/4ertus2)) -* Improved `ALTER MODIFY/ADD` queries logic. Now you cannot `ADD` column without type, `MODIFY` default expression doesn't change type of column and `MODIFY` type doesn't loose default expression value. Fixes [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). [#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) ([alesapin](https://github.com/alesapin)) -* Require server to be restarted to apply the changes in logging configuration. This is a temporary workaround to avoid the bug where the server logs to a deleted log file (see [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* The setting `experimental_use_processors` is enabled by default. This setting enables usage of the new query pipeline. This is internal refactoring and we expect no visible changes. If you will see any issues, set it to back zero. [#8768](https://github.com/ClickHouse/ClickHouse/pull/8768) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### New Feature -* Add `Avro` and `AvroConfluent` input/output formats [#8571](https://github.com/ClickHouse/ClickHouse/pull/8571) ([Andrew Onyshchuk](https://github.com/oandrew)) [#8957](https://github.com/ClickHouse/ClickHouse/pull/8957) ([Andrew Onyshchuk](https://github.com/oandrew)) [#8717](https://github.com/ClickHouse/ClickHouse/pull/8717) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Multi-threaded and non-blocking updates of expired keys in `cache` dictionaries (with optional permission to read old ones). [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Add query `ALTER ... MATERIALIZE TTL`. It runs mutation that forces to remove expired data by TTL and recalculates meta-information about TTL in all parts. [#8775](https://github.com/ClickHouse/ClickHouse/pull/8775) ([Anton Popov](https://github.com/CurtizJ)) -* Switch from HashJoin to MergeJoin (on disk) if needed [#9082](https://github.com/ClickHouse/ClickHouse/pull/9082) ([Artem Zuikov](https://github.com/4ertus2)) -* Added `MOVE PARTITION` command for `ALTER TABLE` [#4729](https://github.com/ClickHouse/ClickHouse/issues/4729) [#6168](https://github.com/ClickHouse/ClickHouse/pull/6168) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Reloading storage configuration from configuration file on the fly. [#8594](https://github.com/ClickHouse/ClickHouse/pull/8594) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Allowed to change `storage_policy` to not less rich one. [#8107](https://github.com/ClickHouse/ClickHouse/pull/8107) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Added support for globs/wildcards for S3 storage and table function. [#8851](https://github.com/ClickHouse/ClickHouse/pull/8851) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Implement `bitAnd`, `bitOr`, `bitXor`, `bitNot` for `FixedString(N)` datatype. [#9091](https://github.com/ClickHouse/ClickHouse/pull/9091) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Added function `bitCount`. This fixes [#8702](https://github.com/ClickHouse/ClickHouse/issues/8702). [#8708](https://github.com/ClickHouse/ClickHouse/pull/8708) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#8749](https://github.com/ClickHouse/ClickHouse/pull/8749) ([ikopylov](https://github.com/ikopylov)) -* Add `generateRandom` table function to generate random rows with given schema. Allows to populate arbitrary test table with data. [#8994](https://github.com/ClickHouse/ClickHouse/pull/8994) ([Ilya Yatsishin](https://github.com/qoega)) -* `JSONEachRowFormat`: support special case when objects enclosed in top-level array. [#8860](https://github.com/ClickHouse/ClickHouse/pull/8860) ([Kruglov Pavel](https://github.com/Avogar)) -* Now it's possible to create a column with `DEFAULT` expression which depends on a column with default `ALIAS` expression. [#9489](https://github.com/ClickHouse/ClickHouse/pull/9489) ([alesapin](https://github.com/alesapin)) -* Allow to specify `--limit` more than the source data size in `clickhouse-obfuscator`. The data will repeat itself with different random seed. [#9155](https://github.com/ClickHouse/ClickHouse/pull/9155) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added `groupArraySample` function (similar to `groupArray`) with reservior sampling algorithm. [#8286](https://github.com/ClickHouse/ClickHouse/pull/8286) ([Amos Bird](https://github.com/amosbird)) -* Now you can monitor the size of update queue in `cache`/`complex_key_cache` dictionaries via system metrics. [#9413](https://github.com/ClickHouse/ClickHouse/pull/9413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Allow to use CRLF as a line separator in CSV output format with setting `output_format_csv_crlf_end_of_line` is set to 1 [#8934](https://github.com/ClickHouse/ClickHouse/pull/8934) [#8935](https://github.com/ClickHouse/ClickHouse/pull/8935) [#8963](https://github.com/ClickHouse/ClickHouse/pull/8963) ([Mikhail Korotov](https://github.com/millb)) -* Implement more functions of the [H3](https://github.com/uber/h3) API: `h3GetBaseCell`, `h3HexAreaM2`, `h3IndexesAreNeighbors`, `h3ToChildren`, `h3ToString` and `stringToH3` [#8938](https://github.com/ClickHouse/ClickHouse/pull/8938) ([Nico Mandery](https://github.com/nmandery)) -* New setting introduced: `max_parser_depth` to control maximum stack size and allow large complex queries. This fixes [#6681](https://github.com/ClickHouse/ClickHouse/issues/6681) and [#7668](https://github.com/ClickHouse/ClickHouse/issues/7668). [#8647](https://github.com/ClickHouse/ClickHouse/pull/8647) ([Maxim Smirnov](https://github.com/qMBQx8GH)) -* Add a setting `force_optimize_skip_unused_shards` setting to throw if skipping of unused shards is not possible [#8805](https://github.com/ClickHouse/ClickHouse/pull/8805) ([Azat Khuzhin](https://github.com/azat)) -* Allow to configure multiple disks/volumes for storing data for send in `Distributed` engine [#8756](https://github.com/ClickHouse/ClickHouse/pull/8756) ([Azat Khuzhin](https://github.com/azat)) -* Support storage policy (``) for storing temporary data. [#8750](https://github.com/ClickHouse/ClickHouse/pull/8750) ([Azat Khuzhin](https://github.com/azat)) -* Added `X-ClickHouse-Exception-Code` HTTP header that is set if exception was thrown before sending data. This implements [#4971](https://github.com/ClickHouse/ClickHouse/issues/4971). [#8786](https://github.com/ClickHouse/ClickHouse/pull/8786) ([Mikhail Korotov](https://github.com/millb)) -* Added function `ifNotFinite`. It is just a syntactic sugar: `ifNotFinite(x, y) = isFinite(x) ? x : y`. [#8710](https://github.com/ClickHouse/ClickHouse/pull/8710) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added `last_successful_update_time` column in `system.dictionaries` table [#9394](https://github.com/ClickHouse/ClickHouse/pull/9394) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Add `blockSerializedSize` function (size on disk without compression) [#8952](https://github.com/ClickHouse/ClickHouse/pull/8952) ([Azat Khuzhin](https://github.com/azat)) -* Add function `moduloOrZero` [#9358](https://github.com/ClickHouse/ClickHouse/pull/9358) ([hcz](https://github.com/hczhcz)) -* Added system tables `system.zeros` and `system.zeros_mt` as well as tale functions `zeros()` and `zeros_mt()`. Tables (and table functions) contain single column with name `zero` and type `UInt8`. This column contains zeros. It is needed for test purposes as the fastest method to generate many rows. This fixes [#6604](https://github.com/ClickHouse/ClickHouse/issues/6604) [#9593](https://github.com/ClickHouse/ClickHouse/pull/9593) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### Experimental Feature -* Add new compact format of parts in `MergeTree`-family tables in which all columns are stored in one file. It helps to increase performance of small and frequent inserts. The old format (one file per column) is now called wide. Data storing format is controlled by settings `min_bytes_for_wide_part` and `min_rows_for_wide_part`. [#8290](https://github.com/ClickHouse/ClickHouse/pull/8290) ([Anton Popov](https://github.com/CurtizJ)) -* Support for S3 storage for `Log`, `TinyLog` and `StripeLog` tables. [#8862](https://github.com/ClickHouse/ClickHouse/pull/8862) ([Pavel Kovalenko](https://github.com/Jokser)) - -#### Bug Fix -* Fixed inconsistent whitespaces in log messages. [#9322](https://github.com/ClickHouse/ClickHouse/pull/9322) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix bug in which arrays of unnamed tuples were flattened as Nested structures on table creation. [#8866](https://github.com/ClickHouse/ClickHouse/pull/8866) ([achulkov2](https://github.com/achulkov2)) -* Fixed the issue when "Too many open files" error may happen if there are too many files matching glob pattern in `File` table or `file` table function. Now files are opened lazily. This fixes [#8857](https://github.com/ClickHouse/ClickHouse/issues/8857) [#8861](https://github.com/ClickHouse/ClickHouse/pull/8861) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* DROP TEMPORARY TABLE now drops only temporary table. [#8907](https://github.com/ClickHouse/ClickHouse/pull/8907) ([Vitaly Baranov](https://github.com/vitlibar)) -* Remove outdated partition when we shutdown the server or DETACH/ATTACH a table. [#8602](https://github.com/ClickHouse/ClickHouse/pull/8602) ([Guillaume Tassery](https://github.com/YiuRULE)) -* For how the default disk calculates the free space from `data` subdirectory. Fixed the issue when the amount of free space is not calculated correctly if the `data` directory is mounted to a separate device (rare case). This fixes [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) [#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) ([Mikhail Korotov](https://github.com/millb)) -* Allow comma (cross) join with IN () inside. [#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) ([Artem Zuikov](https://github.com/4ertus2)) -* Allow to rewrite CROSS to INNER JOIN if there's [NOT] LIKE operator in WHERE section. [#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix possible incorrect result after `GROUP BY` with enabled setting `distributed_aggregation_memory_efficient`. Fixes [#9134](https://github.com/ClickHouse/ClickHouse/issues/9134). [#9289](https://github.com/ClickHouse/ClickHouse/pull/9289) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Found keys were counted as missed in metrics of cache dictionaries. [#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix replication protocol incompatibility introduced in [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) -* Fixed race condition on `queue_task_handle` at the startup of `ReplicatedMergeTree` tables. [#9552](https://github.com/ClickHouse/ClickHouse/pull/9552) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* The token `NOT` didn't work in `SHOW TABLES NOT LIKE` query [#8727](https://github.com/ClickHouse/ClickHouse/issues/8727) [#8940](https://github.com/ClickHouse/ClickHouse/pull/8940) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added range check to function `h3EdgeLengthM`. Without this check, buffer overflow is possible. [#8945](https://github.com/ClickHouse/ClickHouse/pull/8945) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed up a bug in batched calculations of ternary logical OPs on multiple arguments (more than 10). [#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([Alexander Kazakov](https://github.com/Akazz)) -* Fix error of PREWHERE optimization, which could lead to segfaults or `Inconsistent number of columns got from MergeTreeRangeReader` exception. [#9024](https://github.com/ClickHouse/ClickHouse/pull/9024) ([Anton Popov](https://github.com/CurtizJ)) -* Fix unexpected `Timeout exceeded while reading from socket` exception, which randomly happens on secure connection before timeout actually exceeded and when query profiler is enabled. Also add `connect_timeout_with_failover_secure_ms` settings (default 100ms), which is similar to `connect_timeout_with_failover_ms`, but is used for secure connections (because SSL handshake is slower, than ordinary TCP connection) [#9026](https://github.com/ClickHouse/ClickHouse/pull/9026) ([tavplubix](https://github.com/tavplubix)) -* Fix bug with mutations finalization, when mutation may hang in state with `parts_to_do=0` and `is_done=0`. [#9022](https://github.com/ClickHouse/ClickHouse/pull/9022) ([alesapin](https://github.com/alesapin)) -* Use new ANY JOIN logic with `partial_merge_join` setting. It's possible to make `ANY|ALL|SEMI LEFT` and `ALL INNER` joins with `partial_merge_join=1` now. [#8932](https://github.com/ClickHouse/ClickHouse/pull/8932) ([Artem Zuikov](https://github.com/4ertus2)) -* Shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send queries to a shard with another constraints. [#9447](https://github.com/ClickHouse/ClickHouse/pull/9447) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fixed memory management problem in `MergeTreeReadPool`. [#8791](https://github.com/ClickHouse/ClickHouse/pull/8791) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix `toDecimal*OrNull()` functions family when called with string `e`. Fixes [#8312](https://github.com/ClickHouse/ClickHouse/issues/8312) [#8764](https://github.com/ClickHouse/ClickHouse/pull/8764) ([Artem Zuikov](https://github.com/4ertus2)) -* Make sure that `FORMAT Null` sends no data to the client. [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Fix bug that timestamp in `LiveViewBlockInputStream` will not updated. `LIVE VIEW` is an experimental feature. [#8644](https://github.com/ClickHouse/ClickHouse/pull/8644) ([vxider](https://github.com/Vxider)) [#8625](https://github.com/ClickHouse/ClickHouse/pull/8625) ([vxider](https://github.com/Vxider)) -* Fixed `ALTER MODIFY TTL` wrong behavior which did not allow to delete old TTL expressions. [#8422](https://github.com/ClickHouse/ClickHouse/pull/8422) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fixed UBSan report in MergeTreeIndexSet. This fixes [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) [#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed the behaviour of `match` and `extract` functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. This fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Avoid throwing from destructor in Apache Avro 3rd-party library. [#9066](https://github.com/ClickHouse/ClickHouse/pull/9066) ([Andrew Onyshchuk](https://github.com/oandrew)) -* Don't commit a batch polled from `Kafka` partially as it can lead to holes in data. [#8876](https://github.com/ClickHouse/ClickHouse/pull/8876) ([filimonov](https://github.com/filimonov)) -* Fix `joinGet` with nullable return types. https://github.com/ClickHouse/ClickHouse/issues/8919 [#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) ([Amos Bird](https://github.com/amosbird)) -* Fix data incompatibility when compressed with `T64` codec. [#9016](https://github.com/ClickHouse/ClickHouse/pull/9016) ([Artem Zuikov](https://github.com/4ertus2)) Fix data type ids in `T64` compression codec that leads to wrong (de)compression in affected versions. [#9033](https://github.com/ClickHouse/ClickHouse/pull/9033) ([Artem Zuikov](https://github.com/4ertus2)) -* Add setting `enable_early_constant_folding` and disable it in some cases that leads to errors. [#9010](https://github.com/ClickHouse/ClickHouse/pull/9010) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix pushdown predicate optimizer with VIEW and enable the test [#9011](https://github.com/ClickHouse/ClickHouse/pull/9011) ([Winter Zhang](https://github.com/zhang2014)) -* Fix segfault in `Merge` tables, that can happen when reading from `File` storages [#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) ([tavplubix](https://github.com/tavplubix)) -* Added a check for storage policy in `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE`. Otherwise it could make data of part inaccessible after restart and prevent ClickHouse to start. [#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix alters if there is TTL set for table. [#8800](https://github.com/ClickHouse/ClickHouse/pull/8800) ([Anton Popov](https://github.com/CurtizJ)) -* Fix race condition that can happen when `SYSTEM RELOAD ALL DICTIONARIES` is executed while some dictionary is being modified/added/removed. [#8801](https://github.com/ClickHouse/ClickHouse/pull/8801) ([Vitaly Baranov](https://github.com/vitlibar)) -* In previous versions `Memory` database engine use empty data path, so tables are created in `path` directory (e.g. `/var/lib/clickhouse/`), not in data directory of database (e.g. `/var/lib/clickhouse/db_name`). [#8753](https://github.com/ClickHouse/ClickHouse/pull/8753) ([tavplubix](https://github.com/tavplubix)) -* Fixed wrong log messages about missing default disk or policy. [#9530](https://github.com/ClickHouse/ClickHouse/pull/9530) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix not(has()) for the bloom_filter index of array types. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) -* Allow first column(s) in a table with `Log` engine be an alias [#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) ([Ivan](https://github.com/abyss7)) -* Fix order of ranges while reading from `MergeTree` table in one thread. It could lead to exceptions from `MergeTreeRangeReader` or wrong query results. [#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) ([Anton Popov](https://github.com/CurtizJ)) -* Make `reinterpretAsFixedString` to return `FixedString` instead of `String`. [#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) ([Andrew Onyshchuk](https://github.com/oandrew)) -* Avoid extremely rare cases when the user can get wrong error message (`Success` instead of detailed error description). [#9457](https://github.com/ClickHouse/ClickHouse/pull/9457) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Do not crash when using `Template` format with empty row template. [#8785](https://github.com/ClickHouse/ClickHouse/pull/8785) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Metadata files for system tables could be created in wrong place [#8653](https://github.com/ClickHouse/ClickHouse/pull/8653) ([tavplubix](https://github.com/tavplubix)) Fixes [#8581](https://github.com/ClickHouse/ClickHouse/issues/8581). -* Fix data race on exception_ptr in cache dictionary [#8303](https://github.com/ClickHouse/ClickHouse/issues/8303). [#9379](https://github.com/ClickHouse/ClickHouse/pull/9379) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Do not throw an exception for query `ATTACH TABLE IF NOT EXISTS`. Previously it was thrown if table already exists, despite the `IF NOT EXISTS` clause. [#8967](https://github.com/ClickHouse/ClickHouse/pull/8967) ([Anton Popov](https://github.com/CurtizJ)) -* Fixed missing closing paren in exception message. [#8811](https://github.com/ClickHouse/ClickHouse/pull/8811) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Avoid message `Possible deadlock avoided` at the startup of clickhouse-client in interactive mode. [#9455](https://github.com/ClickHouse/ClickHouse/pull/9455) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed the issue when padding at the end of base64 encoded value can be malformed. Update base64 library. This fixes [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Prevent losing data in `Kafka` in rare cases when exception happens after reading suffix but before commit. Fixes [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378) [#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) ([filimonov](https://github.com/filimonov)) -* Fixed exception in `DROP TABLE IF EXISTS` [#8663](https://github.com/ClickHouse/ClickHouse/pull/8663) ([Nikita Vasilev](https://github.com/nikvas0)) -* Fix crash when a user tries to `ALTER MODIFY SETTING` for old-formated `MergeTree` table engines family. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) -* Support for UInt64 numbers that don't fit in Int64 in JSON-related functions. Update SIMDJSON to master. This fixes [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) [#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed execution of inversed predicates when non-strictly monotinic functional index is used. [#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) ([Alexander Kazakov](https://github.com/Akazz)) -* Don't try to fold `IN` constant in `GROUP BY` [#8868](https://github.com/ClickHouse/ClickHouse/pull/8868) ([Amos Bird](https://github.com/amosbird)) -* Fix bug in `ALTER DELETE` mutations which leads to index corruption. This fixes [#9019](https://github.com/ClickHouse/ClickHouse/issues/9019) and [#8982](https://github.com/ClickHouse/ClickHouse/issues/8982). Additionally fix extremely rare race conditions in `ReplicatedMergeTree` `ALTER` queries. [#9048](https://github.com/ClickHouse/ClickHouse/pull/9048) ([alesapin](https://github.com/alesapin)) -* When the setting `compile_expressions` is enabled, you can get `unexpected column` in `LLVMExecutableFunction` when we use `Nullable` type [#8910](https://github.com/ClickHouse/ClickHouse/pull/8910) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Multiple fixes for `Kafka` engine: 1) fix duplicates that were appearing during consumer group rebalance. 2) Fix rare 'holes' appeared when data were polled from several partitions with one poll and committed partially (now we always process / commit the whole polled block of messages). 3) Fix flushes by block size (before that only flushing by timeout was working properly). 4) better subscription procedure (with assignment feedback). 5) Make tests work faster (with default intervals and timeouts). Due to the fact that data was not flushed by block size before (as it should according to documentation), that PR may lead to some performance degradation with default settings (due to more often & tinier flushes which are less optimal). If you encounter the performance issue after that change - please increase `kafka_max_block_size` in the table to the bigger value ( for example `CREATE TABLE ...Engine=Kafka ... SETTINGS ... kafka_max_block_size=524288`). Fixes [#7259](https://github.com/ClickHouse/ClickHouse/issues/7259) [#8917](https://github.com/ClickHouse/ClickHouse/pull/8917) ([filimonov](https://github.com/filimonov)) -* Fix `Parameter out of bound` exception in some queries after PREWHERE optimizations. [#8914](https://github.com/ClickHouse/ClickHouse/pull/8914) ([Baudouin Giard](https://github.com/bgiard)) -* Fixed the case of mixed-constness of arguments of function `arrayZip`. [#8705](https://github.com/ClickHouse/ClickHouse/pull/8705) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* When executing `CREATE` query, fold constant expressions in storage engine arguments. Replace empty database name with current database. Fixes [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492) [#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) ([tavplubix](https://github.com/tavplubix)) -* Now it's not possible to create or add columns with simple cyclic aliases like `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) -* Fixed a bug with double move which may corrupt original part. This is relevant if you use `ALTER TABLE MOVE` [#8680](https://github.com/ClickHouse/ClickHouse/pull/8680) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Allow `interval` identifier to correctly parse without backticks. Fixed issue when a query cannot be executed even if the `interval` identifier is enclosed in backticks or double quotes. This fixes [#9124](https://github.com/ClickHouse/ClickHouse/issues/9124). [#9142](https://github.com/ClickHouse/ClickHouse/pull/9142) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed fuzz test and incorrect behaviour of `bitTestAll`/`bitTestAny` functions. [#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix possible crash/wrong number of rows in `LIMIT n WITH TIES` when there are a lot of rows equal to n'th row. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) -* Fix mutations with parts written with enabled `insert_quorum`. [#9463](https://github.com/ClickHouse/ClickHouse/pull/9463) ([alesapin](https://github.com/alesapin)) -* Fix data race at destruction of `Poco::HTTPServer`. It could happen when server is started and immediately shut down. [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) -* Fix bug in which a misleading error message was shown when running `SHOW CREATE TABLE a_table_that_does_not_exist`. [#8899](https://github.com/ClickHouse/ClickHouse/pull/8899) ([achulkov2](https://github.com/achulkov2)) -* Fixed `Parameters are out of bound` exception in some rare cases when we have a constant in the `SELECT` clause when we have an `ORDER BY` and a `LIMIT` clause. [#8892](https://github.com/ClickHouse/ClickHouse/pull/8892) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Fix mutations finalization, when already done mutation can have status `is_done=0`. [#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) ([alesapin](https://github.com/alesapin)) -* Prevent from executing `ALTER ADD INDEX` for MergeTree tables with old syntax, because it doesn't work. [#8822](https://github.com/ClickHouse/ClickHouse/pull/8822) ([Mikhail Korotov](https://github.com/millb)) -* During server startup do not access table, which `LIVE VIEW` depends on, so server will be able to start. Also remove `LIVE VIEW` dependencies when detaching `LIVE VIEW`. `LIVE VIEW` is an experimental feature. [#8824](https://github.com/ClickHouse/ClickHouse/pull/8824) ([tavplubix](https://github.com/tavplubix)) -* Fix possible segfault in `MergeTreeRangeReader`, while executing `PREWHERE`. [#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) ([Anton Popov](https://github.com/CurtizJ)) -* Fix possible mismatched checksums with column TTLs. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) -* Fixed a bug when parts were not being moved in background by TTL rules in case when there is only one volume. [#8672](https://github.com/ClickHouse/ClickHouse/pull/8672) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fixed the issue `Method createColumn() is not implemented for data type Set`. This fixes [#7799](https://github.com/ClickHouse/ClickHouse/issues/7799). [#8674](https://github.com/ClickHouse/ClickHouse/pull/8674) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now we will try finalize mutations more frequently. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) -* Fix `intDiv` by minus one constant [#9351](https://github.com/ClickHouse/ClickHouse/pull/9351) ([hcz](https://github.com/hczhcz)) -* Fix possible race condition in `BlockIO`. [#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix bug leading to server termination when trying to use / drop `Kafka` table created with wrong parameters. [#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) ([filimonov](https://github.com/filimonov)) -* Added workaround if OS returns wrong result for `timer_create` function. [#8837](https://github.com/ClickHouse/ClickHouse/pull/8837) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed error in usage of `min_marks_for_seek` parameter. Fixed the error message when there is no sharding key in Distributed table and we try to skip unused shards. [#8908](https://github.com/ClickHouse/ClickHouse/pull/8908) ([Azat Khuzhin](https://github.com/azat)) - -#### Improvement -* Implement `ALTER MODIFY/DROP` queries on top of mutations for `ReplicatedMergeTree*` engines family. Now `ALTERS` blocks only at the metadata update stage, and don't block after that. [#8701](https://github.com/ClickHouse/ClickHouse/pull/8701) ([alesapin](https://github.com/alesapin)) -* Add ability to rewrite CROSS to INNER JOINs with `WHERE` section containing unqialified names. [#9512](https://github.com/ClickHouse/ClickHouse/pull/9512) ([Artem Zuikov](https://github.com/4ertus2)) -* Make `SHOW TABLES` and `SHOW DATABASES` queries support the `WHERE` expressions and `FROM`/`IN` [#9076](https://github.com/ClickHouse/ClickHouse/pull/9076) ([sundyli](https://github.com/sundy-li)) -* Added a setting `deduplicate_blocks_in_dependent_materialized_views`. [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) ([urykhy](https://github.com/urykhy)) -* After recent changes MySQL client started to print binary strings in hex thereby making them not readable ([#9032](https://github.com/ClickHouse/ClickHouse/issues/9032)). The workaround in ClickHouse is to mark string columns as UTF-8, which is not always, but usually the case. [#9079](https://github.com/ClickHouse/ClickHouse/pull/9079) ([Yuriy Baranov](https://github.com/yurriy)) -* Add support of String and FixedString keys for `sumMap` [#8903](https://github.com/ClickHouse/ClickHouse/pull/8903) ([Baudouin Giard](https://github.com/bgiard)) -* Support string keys in SummingMergeTree maps [#8933](https://github.com/ClickHouse/ClickHouse/pull/8933) ([Baudouin Giard](https://github.com/bgiard)) -* Signal termination of thread to the thread pool even if the thread has thrown exception [#8736](https://github.com/ClickHouse/ClickHouse/pull/8736) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* Allow to set `query_id` in `clickhouse-benchmark` [#9416](https://github.com/ClickHouse/ClickHouse/pull/9416) ([Anton Popov](https://github.com/CurtizJ)) -* Don't allow strange expressions in `ALTER TABLE ... PARTITION partition` query. This addresses [#7192](https://github.com/ClickHouse/ClickHouse/issues/7192) [#8835](https://github.com/ClickHouse/ClickHouse/pull/8835) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* The table `system.table_engines` now provides information about feature support (like `supports_ttl` or `supports_sort_order`). [#8830](https://github.com/ClickHouse/ClickHouse/pull/8830) ([Max Akhmedov](https://github.com/zlobober)) -* Enable `system.metric_log` by default. It will contain rows with values of ProfileEvents, CurrentMetrics collected with "collect_interval_milliseconds" interval (one second by default). The table is very small (usually in order of megabytes) and collecting this data by default is reasonable. [#9225](https://github.com/ClickHouse/ClickHouse/pull/9225) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries. Fixes [#6964](https://github.com/ClickHouse/ClickHouse/issues/6964) [#8874](https://github.com/ClickHouse/ClickHouse/pull/8874) ([Ivan](https://github.com/abyss7)) -* Now temporary `LIVE VIEW` is created by `CREATE LIVE VIEW name WITH TIMEOUT [42] ...` instead of `CREATE TEMPORARY LIVE VIEW ...`, because the previous syntax was not consistent with `CREATE TEMPORARY TABLE ...` [#9131](https://github.com/ClickHouse/ClickHouse/pull/9131) ([tavplubix](https://github.com/tavplubix)) -* Add text_log.level configuration parameter to limit entries that goes to `system.text_log` table [#8809](https://github.com/ClickHouse/ClickHouse/pull/8809) ([Azat Khuzhin](https://github.com/azat)) -* Allow to put downloaded part to a disks/volumes according to TTL rules [#8598](https://github.com/ClickHouse/ClickHouse/pull/8598) ([Vladimir Chebotarev](https://github.com/excitoon)) -* For external MySQL dictionaries, allow to mutualize MySQL connection pool to "share" them among dictionaries. This option significantly reduces the number of connections to MySQL servers. [#9409](https://github.com/ClickHouse/ClickHouse/pull/9409) ([Clément Rodriguez](https://github.com/clemrodriguez)) -* Show nearest query execution time for quantiles in `clickhouse-benchmark` output instead of interpolated values. It's better to show values that correspond to the execution time of some queries. [#8712](https://github.com/ClickHouse/ClickHouse/pull/8712) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Possibility to add key & timestamp for the message when inserting data to Kafka. Fixes [#7198](https://github.com/ClickHouse/ClickHouse/issues/7198) [#8969](https://github.com/ClickHouse/ClickHouse/pull/8969) ([filimonov](https://github.com/filimonov)) -* If server is run from terminal, highlight thread number, query id and log priority by colors. This is for improved readability of correlated log messages for developers. [#8961](https://github.com/ClickHouse/ClickHouse/pull/8961) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better exception message while loading tables for `Ordinary` database. [#9527](https://github.com/ClickHouse/ClickHouse/pull/9527) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Implement `arraySlice` for arrays with aggregate function states. This fixes [#9388](https://github.com/ClickHouse/ClickHouse/issues/9388) [#9391](https://github.com/ClickHouse/ClickHouse/pull/9391) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Allow constant functions and constant arrays to be used on the right side of IN operator. [#8813](https://github.com/ClickHouse/ClickHouse/pull/8813) ([Anton Popov](https://github.com/CurtizJ)) -* If zookeeper exception has happened while fetching data for system.replicas, display it in a separate column. This implements [#9137](https://github.com/ClickHouse/ClickHouse/issues/9137) [#9138](https://github.com/ClickHouse/ClickHouse/pull/9138) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Atomically remove MergeTree data parts on destroy. [#8402](https://github.com/ClickHouse/ClickHouse/pull/8402) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Support row-level security for Distributed tables. [#8926](https://github.com/ClickHouse/ClickHouse/pull/8926) ([Ivan](https://github.com/abyss7)) -* Now we recognize suffix (like KB, KiB...) in settings values. [#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([Mikhail Korotov](https://github.com/millb)) -* Prevent out of memory while constructing result of a large JOIN. [#8637](https://github.com/ClickHouse/ClickHouse/pull/8637) ([Artem Zuikov](https://github.com/4ertus2)) -* Added names of clusters to suggestions in interactive mode in `clickhouse-client`. [#8709](https://github.com/ClickHouse/ClickHouse/pull/8709) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries [#8820](https://github.com/ClickHouse/ClickHouse/pull/8820) ([Ivan](https://github.com/abyss7)) -* Added column `exception_code` in `system.query_log` table. [#8770](https://github.com/ClickHouse/ClickHouse/pull/8770) ([Mikhail Korotov](https://github.com/millb)) -* Enabled MySQL compatibility server on port `9004` in the default server configuration file. Fixed password generation command in the example in configuration. [#8771](https://github.com/ClickHouse/ClickHouse/pull/8771) ([Yuriy Baranov](https://github.com/yurriy)) -* Prevent abort on shutdown if the filesystem is readonly. This fixes [#9094](https://github.com/ClickHouse/ClickHouse/issues/9094) [#9100](https://github.com/ClickHouse/ClickHouse/pull/9100) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better exception message when length is required in HTTP POST query. [#9453](https://github.com/ClickHouse/ClickHouse/pull/9453) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add `_path` and `_file` virtual columns to `HDFS` and `File` engines and `hdfs` and `file` table functions [#8489](https://github.com/ClickHouse/ClickHouse/pull/8489) ([Olga Khvostikova](https://github.com/stavrolia)) -* Fix error `Cannot find column` while inserting into `MATERIALIZED VIEW` in case if new column was added to view's internal table. [#8766](https://github.com/ClickHouse/ClickHouse/pull/8766) [#8788](https://github.com/ClickHouse/ClickHouse/pull/8788) ([vzakaznikov](https://github.com/vzakaznikov)) [#8788](https://github.com/ClickHouse/ClickHouse/issues/8788) [#8806](https://github.com/ClickHouse/ClickHouse/pull/8806) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#8803](https://github.com/ClickHouse/ClickHouse/pull/8803) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix progress over native client-server protocol, by send progress after final update (like logs). This may be relevant only to some third-party tools that are using native protocol. [#9495](https://github.com/ClickHouse/ClickHouse/pull/9495) ([Azat Khuzhin](https://github.com/azat)) -* Add a system metric tracking the number of client connections using MySQL protocol ([#9013](https://github.com/ClickHouse/ClickHouse/issues/9013)). [#9015](https://github.com/ClickHouse/ClickHouse/pull/9015) ([Eugene Klimov](https://github.com/Slach)) -* From now on, HTTP responses will have `X-ClickHouse-Timezone` header set to the same timezone value that `SELECT timezone()` would report. [#9493](https://github.com/ClickHouse/ClickHouse/pull/9493) ([Denis Glazachev](https://github.com/traceon)) - -#### Performance Improvement -* Improve performance of analysing index with IN [#9261](https://github.com/ClickHouse/ClickHouse/pull/9261) ([Anton Popov](https://github.com/CurtizJ)) -* Simpler and more efficient code in Logical Functions + code cleanups. A followup to [#8718](https://github.com/ClickHouse/ClickHouse/issues/8718) [#8728](https://github.com/ClickHouse/ClickHouse/pull/8728) ([Alexander Kazakov](https://github.com/Akazz)) -* Overall performance improvement (in range of 5%..200% for affected queries) by ensuring even more strict aliasing with C++20 features. [#9304](https://github.com/ClickHouse/ClickHouse/pull/9304) ([Amos Bird](https://github.com/amosbird)) -* More strict aliasing for inner loops of comparison functions. [#9327](https://github.com/ClickHouse/ClickHouse/pull/9327) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* More strict aliasing for inner loops of arithmetic functions. [#9325](https://github.com/ClickHouse/ClickHouse/pull/9325) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* A ~3 times faster implementation for ColumnVector::replicate(), via which ColumnConst::convertToFullColumn() is implemented. Also will be useful in tests when materializing constants. [#9293](https://github.com/ClickHouse/ClickHouse/pull/9293) ([Alexander Kazakov](https://github.com/Akazz)) -* Another minor performance improvement to `ColumnVector::replicate()` (this speeds up the `materialize` function and higher order functions) an even further improvement to [#9293](https://github.com/ClickHouse/ClickHouse/issues/9293) [#9442](https://github.com/ClickHouse/ClickHouse/pull/9442) ([Alexander Kazakov](https://github.com/Akazz)) -* Improved performance of `stochasticLinearRegression` aggregate function. This patch is contributed by Intel. [#8652](https://github.com/ClickHouse/ClickHouse/pull/8652) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improve performance of `reinterpretAsFixedString` function. [#9342](https://github.com/ClickHouse/ClickHouse/pull/9342) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Do not send blocks to client for `Null` format in processors pipeline. [#8797](https://github.com/ClickHouse/ClickHouse/pull/8797) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) - -#### Build/Testing/Packaging Improvement -* Exception handling now works correctly on Windows Subsystem for Linux. See https://github.com/ClickHouse-Extras/libunwind/pull/3 This fixes [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) -* Replace `readline` with `replxx` for interactive line editing in `clickhouse-client` [#8416](https://github.com/ClickHouse/ClickHouse/pull/8416) ([Ivan](https://github.com/abyss7)) -* Better build time and less template instantiations in FunctionsComparison. [#9324](https://github.com/ClickHouse/ClickHouse/pull/9324) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added integration with `clang-tidy` in CI. See also [#6044](https://github.com/ClickHouse/ClickHouse/issues/6044) [#9566](https://github.com/ClickHouse/ClickHouse/pull/9566) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now we link ClickHouse in CI using `lld` even for `gcc`. [#9049](https://github.com/ClickHouse/ClickHouse/pull/9049) ([alesapin](https://github.com/alesapin)) -* Allow to randomize thread scheduling and insert glitches when `THREAD_FUZZER_*` environment variables are set. This helps testing. [#9459](https://github.com/ClickHouse/ClickHouse/pull/9459) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Enable secure sockets in stateless tests [#9288](https://github.com/ClickHouse/ClickHouse/pull/9288) ([tavplubix](https://github.com/tavplubix)) -* Make SPLIT_SHARED_LIBRARIES=OFF more robust [#9156](https://github.com/ClickHouse/ClickHouse/pull/9156) ([Azat Khuzhin](https://github.com/azat)) -* Make "performance_introspection_and_logging" test reliable to random server stuck. This may happen in CI environment. See also [#9515](https://github.com/ClickHouse/ClickHouse/issues/9515) [#9528](https://github.com/ClickHouse/ClickHouse/pull/9528) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Validate XML in style check. [#9550](https://github.com/ClickHouse/ClickHouse/pull/9550) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed race condition in test `00738_lock_for_inner_table`. This test relied on sleep. [#9555](https://github.com/ClickHouse/ClickHouse/pull/9555) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove performance tests of type `once`. This is needed to run all performance tests in statistical comparison mode (more reliable). [#9557](https://github.com/ClickHouse/ClickHouse/pull/9557) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added performance test for arithmetic functions. [#9326](https://github.com/ClickHouse/ClickHouse/pull/9326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added performance test for `sumMap` and `sumMapWithOverflow` aggregate functions. Follow-up for [#8933](https://github.com/ClickHouse/ClickHouse/issues/8933) [#8947](https://github.com/ClickHouse/ClickHouse/pull/8947) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Ensure style of ErrorCodes by style check. [#9370](https://github.com/ClickHouse/ClickHouse/pull/9370) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add script for tests history. [#8796](https://github.com/ClickHouse/ClickHouse/pull/8796) ([alesapin](https://github.com/alesapin)) -* Add GCC warning `-Wsuggest-override` to locate and fix all places where `override` keyword must be used. [#8760](https://github.com/ClickHouse/ClickHouse/pull/8760) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* Ignore weak symbol under Mac OS X because it must be defined [#9538](https://github.com/ClickHouse/ClickHouse/pull/9538) ([Deleted user](https://github.com/ghost)) -* Normalize running time of some queries in performance tests. This is done in preparation to run all the performance tests in comparison mode. [#9565](https://github.com/ClickHouse/ClickHouse/pull/9565) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix some tests to support pytest with query tests [#9062](https://github.com/ClickHouse/ClickHouse/pull/9062) ([Ivan](https://github.com/abyss7)) -* Enable SSL in build with MSan, so server will not fail at startup when running stateless tests [#9531](https://github.com/ClickHouse/ClickHouse/pull/9531) ([tavplubix](https://github.com/tavplubix)) -* Fix database substitution in test results [#9384](https://github.com/ClickHouse/ClickHouse/pull/9384) ([Ilya Yatsishin](https://github.com/qoega)) -* Build fixes for miscellaneous platforms [#9381](https://github.com/ClickHouse/ClickHouse/pull/9381) ([proller](https://github.com/proller)) [#8755](https://github.com/ClickHouse/ClickHouse/pull/8755) ([proller](https://github.com/proller)) [#8631](https://github.com/ClickHouse/ClickHouse/pull/8631) ([proller](https://github.com/proller)) -* Added disks section to stateless-with-coverage test docker image [#9213](https://github.com/ClickHouse/ClickHouse/pull/9213) ([Pavel Kovalenko](https://github.com/Jokser)) -* Get rid of in-source-tree files when building with GRPC [#9588](https://github.com/ClickHouse/ClickHouse/pull/9588) ([Amos Bird](https://github.com/amosbird)) -* Slightly faster build time by removing SessionCleaner from Context. Make the code of SessionCleaner more simple. [#9232](https://github.com/ClickHouse/ClickHouse/pull/9232) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Updated checking for hung queries in clickhouse-test script [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alexander Kazakov](https://github.com/Akazz)) -* Removed some useless files from repository. [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Changed type of math perftests from `once` to `loop`. [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Add docker image which allows to build interactive code browser HTML report for our codebase. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse.tech/codebrowser/html_report///ClickHouse/dbms/src/index.html) -* Suppress some test failures under MSan. [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Speedup "exception while insert" test. This test often time out in debug-with-coverage build. [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Updated `libcxx` and `libcxxabi` to master. In preparation to [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix flacky test `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Clean up duplicated linker flags. Make sure the linker won't look up an unexpected symbol. [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos Bird](https://github.com/amosbird)) -* Add `clickhouse-odbc` driver into test images. This allows to test interaction of ClickHouse with ClickHouse via its own ODBC driver. [#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) -* Fix several bugs in unit tests. [#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([alesapin](https://github.com/alesapin)) -* Enable `-Wmissing-include-dirs` GCC warning to eliminate all non-existing includes - mostly as a result of CMake scripting errors [#8704](https://github.com/ClickHouse/ClickHouse/pull/8704) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* Describe reasons if query profiler cannot work. This is intended for [#9049](https://github.com/ClickHouse/ClickHouse/issues/9049) [#9144](https://github.com/ClickHouse/ClickHouse/pull/9144) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update OpenSSL to upstream master. Fixed the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` and `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. The issue was present in version 20.1. [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update Dockerfile for server [#8893](https://github.com/ClickHouse/ClickHouse/pull/8893) ([Ilya Mazaev](https://github.com/ne-ray)) -* Minor fixes in build-gcc-from-sources script [#8774](https://github.com/ClickHouse/ClickHouse/pull/8774) ([Michael Nacharov](https://github.com/mnach)) -* Replace `numbers` to `zeros` in perftests where `number` column is not used. This will lead to more clean test results. [#9600](https://github.com/ClickHouse/ClickHouse/pull/9600) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix stack overflow issue when using initializer_list in Column constructors. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)) -* Upgrade librdkafka to v1.3.0. Enable bundled `rdkafka` and `gsasl` libraries on Mac OS X. [#9000](https://github.com/ClickHouse/ClickHouse/pull/9000) ([Andrew Onyshchuk](https://github.com/oandrew)) -* build fix on GCC 9.2.0 [#9306](https://github.com/ClickHouse/ClickHouse/pull/9306) ([vxider](https://github.com/Vxider)) - - -## ClickHouse release v20.1 - -### ClickHouse release v20.1.8.41, 2020-03-20 - -#### Bug Fix -* Fix possible permanent `Cannot schedule a task` error (due to unhandled exception in `ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread`). This fixes [#6833](https://github.com/ClickHouse/ClickHouse/issues/6833). [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) -* Fix excessive memory consumption in `ALTER` queries (mutations). This fixes [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin)) -* Fix bug in backquoting in external dictionaries DDL. This fixes [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin)) - -### ClickHouse release v20.1.7.38, 2020-03-18 - -#### Bug Fix -* Fixed incorrect internal function names for `sumKahan` and `sumWithOverflow`. I lead to exception while using this functions in remote queries. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). This issue was in all ClickHouse releases. -* Allow `ALTER ON CLUSTER` of `Distributed` tables with internal replication. This fixes [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). This issue was in all ClickHouse releases. -* Fix possible exceptions `Size of filter doesn't match size of column` and `Invalid number of rows in Chunk` in `MergeTreeRangeReader`. They could appear while executing `PREWHERE` in some cases. Fixes [#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) -* Fixed the issue: timezone was not preserved if you write a simple arithmetic expression like `time + 1` (in contrast to an expression like `time + INTERVAL 1 SECOND`). This fixes [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)). This issue was in all ClickHouse releases. -* Now it's not possible to create or add columns with simple cyclic aliases like `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) -* Fixed the issue when padding at the end of base64 encoded value can be malformed. Update base64 library. This fixes [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix data race at destruction of `Poco::HTTPServer`. It could happen when server is started and immediately shut down. [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) -* Fix possible crash/wrong number of rows in `LIMIT n WITH TIES` when there are a lot of rows equal to n'th row. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) -* Fix possible mismatched checksums with column TTLs. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) -* Fix crash when a user tries to `ALTER MODIFY SETTING` for old-formated `MergeTree` table engines family. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) -* Now we will try finalize mutations more frequently. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) -* Fix replication protocol incompatibility introduced in [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) -* Fix not(has()) for the bloom_filter index of array types. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) -* Fixed the behaviour of `match` and `extract` functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. This fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### Build/Testing/Packaging Improvement - -* Exception handling now works correctly on Windows Subsystem for Linux. See https://github.com/ClickHouse-Extras/libunwind/pull/3 This fixes [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) - - -### ClickHouse release v20.1.6.30, 2020-03-05 - -#### Bug Fix - -* Fix data incompatibility when compressed with `T64` codec. -[#9039](https://github.com/ClickHouse/ClickHouse/pull/9039) [(abyss7)](https://github.com/abyss7) -* Fix order of ranges while reading from MergeTree table in one thread. Fixes [#8964](https://github.com/ClickHouse/ClickHouse/issues/8964). -[#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) [(CurtizJ)](https://github.com/CurtizJ) -* Fix possible segfault in `MergeTreeRangeReader`, while executing `PREWHERE`. Fixes [#9064](https://github.com/ClickHouse/ClickHouse/issues/9064). -[#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) [(CurtizJ)](https://github.com/CurtizJ) -* Fix `reinterpretAsFixedString` to return `FixedString` instead of `String`. -[#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) [(oandrew)](https://github.com/oandrew) -* Fix `joinGet` with nullable return types. Fixes [#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) -[#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) [(amosbird)](https://github.com/amosbird) -* Fix fuzz test and incorrect behaviour of bitTestAll/bitTestAny functions. -[#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* Fix the behaviour of match and extract functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. Fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) -[#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* Fixed execution of inversed predicates when non-strictly monotinic functional index is used. Fixes [#9034](https://github.com/ClickHouse/ClickHouse/issues/9034) -[#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) [(Akazz)](https://github.com/Akazz) -* Allow to rewrite `CROSS` to `INNER JOIN` if there's `[NOT] LIKE` operator in `WHERE` section. Fixes [#9191](https://github.com/ClickHouse/ClickHouse/issues/9191) -[#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) [(4ertus2)](https://github.com/4ertus2) -* Allow first column(s) in a table with Log engine be an alias. -[#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) [(abyss7)](https://github.com/abyss7) -* Allow comma join with `IN()` inside. Fixes [#7314](https://github.com/ClickHouse/ClickHouse/issues/7314). -[#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) [(4ertus2)](https://github.com/4ertus2) -* Improve `ALTER MODIFY/ADD` queries logic. Now you cannot `ADD` column without type, `MODIFY` default expression doesn't change type of column and `MODIFY` type doesn't loose default expression value. Fixes [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). -[#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) [(alesapin)](https://github.com/alesapin) -* Fix mutations finalization, when already done mutation can have status is_done=0. -[#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) [(alesapin)](https://github.com/alesapin) -* Support "Processors" pipeline for system.numbers and system.numbers_mt. This also fixes the bug when `max_execution_time` is not respected. -[#7796](https://github.com/ClickHouse/ClickHouse/pull/7796) [(KochetovNicolai)](https://github.com/KochetovNicolai) -* Fix wrong counting of `DictCacheKeysRequestedFound` metric. -[#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) [(nikitamikhaylov)](https://github.com/nikitamikhaylov) -* Added a check for storage policy in `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE` which otherwise could make data of part inaccessible after restart and prevent ClickHouse to start. -[#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) [(excitoon)](https://github.com/excitoon) -* Fixed UBSan report in `MergeTreeIndexSet`. This fixes [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) -[#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* Fix possible datarace in BlockIO. -[#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) [(KochetovNicolai)](https://github.com/KochetovNicolai) -* Support for `UInt64` numbers that don't fit in Int64 in JSON-related functions. Update `SIMDJSON` to master. This fixes [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) -[#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* Fix the issue when the amount of free space is not calculated correctly if the data directory is mounted to a separate device. For default disk calculate the free space from data subdirectory. This fixes [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) -[#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) [(millb)](https://github.com/millb) -* Fix the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error and SSL Exception: error:2400006E:random number generator::error retrieving entropy.` Update OpenSSL to upstream master. -[#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* When executing `CREATE` query, fold constant expressions in storage engine arguments. Replace empty database name with current database. Fixes [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). Also fix check for local address in ClickHouseDictionarySource. -[#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) -* Fix segfault in `StorageMerge`, which can happen when reading from StorageFile. -[#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) [(tabplubix)](https://github.com/tavplubix) -* Prevent losing data in `Kafka` in rare cases when exception happens after reading suffix but before commit. Fixes [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). Related: [#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) -[#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) -* Fix bug leading to server termination when trying to use / drop `Kafka` table created with wrong parameters. Fixes [#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporates [#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). -[#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) - -#### New Feature -* Add `deduplicate_blocks_in_dependent_materialized_views` option to control the behaviour of idempotent inserts into tables with materialized views. This new feature was added to the bugfix release by a special request from Altinity. -[#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) - -### ClickHouse release v20.1.2.4, 2020-01-22 - -#### Backward Incompatible Change -* Make the setting `merge_tree_uniform_read_distribution` obsolete. The server still recognizes this setting but it has no effect. [#8308](https://github.com/ClickHouse/ClickHouse/pull/8308) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Changed return type of the function `greatCircleDistance` to `Float32` because now the result of calculation is `Float32`. [#7993](https://github.com/ClickHouse/ClickHouse/pull/7993) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now it's expected that query parameters are represented in "escaped" format. For example, to pass string `ab` you have to write `a\tb` or `a\b` and respectively, `a%5Ctb` or `a%5C%09b` in URL. This is needed to add the possibility to pass NULL as `\N`. This fixes [#7488](https://github.com/ClickHouse/ClickHouse/issues/7488). [#8517](https://github.com/ClickHouse/ClickHouse/pull/8517) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Enable `use_minimalistic_part_header_in_zookeeper` setting for `ReplicatedMergeTree` by default. This will significantly reduce amount of data stored in ZooKeeper. This setting is supported since version 19.1 and we already use it in production in multiple services without any issues for more than half a year. Disable this setting if you have a chance to downgrade to versions older than 19.1. [#6850](https://github.com/ClickHouse/ClickHouse/pull/6850) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Data skipping indices are production ready and enabled by default. The settings `allow_experimental_data_skipping_indices`, `allow_experimental_cross_to_join_conversion` and `allow_experimental_multiple_joins_emulation` are now obsolete and do nothing. [#7974](https://github.com/ClickHouse/ClickHouse/pull/7974) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add new `ANY JOIN` logic for `StorageJoin` consistent with `JOIN` operation. To upgrade without changes in behaviour you need add `SETTINGS any_join_distinct_right_table_keys = 1` to Engine Join tables metadata or recreate these tables after upgrade. [#8400](https://github.com/ClickHouse/ClickHouse/pull/8400) ([Artem Zuikov](https://github.com/4ertus2)) -* Require server to be restarted to apply the changes in logging configuration. This is a temporary workaround to avoid the bug where the server logs to a deleted log file (see [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) - -#### New Feature -* Added information about part paths to `system.merges`. [#8043](https://github.com/ClickHouse/ClickHouse/pull/8043) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Add ability to execute `SYSTEM RELOAD DICTIONARY` query in `ON CLUSTER` mode. [#8288](https://github.com/ClickHouse/ClickHouse/pull/8288) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Add ability to execute `CREATE DICTIONARY` queries in `ON CLUSTER` mode. [#8163](https://github.com/ClickHouse/ClickHouse/pull/8163) ([alesapin](https://github.com/alesapin)) -* Now user's profile in `users.xml` can inherit multiple profiles. [#8343](https://github.com/ClickHouse/ClickHouse/pull/8343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -* Added `system.stack_trace` table that allows to look at stack traces of all server threads. This is useful for developers to introspect server state. This fixes [#7576](https://github.com/ClickHouse/ClickHouse/issues/7576). [#8344](https://github.com/ClickHouse/ClickHouse/pull/8344) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add `DateTime64` datatype with configurable sub-second precision. [#7170](https://github.com/ClickHouse/ClickHouse/pull/7170) ([Vasily Nemkov](https://github.com/Enmk)) -* Add table function `clusterAllReplicas` which allows to query all the nodes in the cluster. [#8493](https://github.com/ClickHouse/ClickHouse/pull/8493) ([kiran sunkari](https://github.com/kiransunkari)) -* Add aggregate function `categoricalInformationValue` which calculates the information value of a discrete feature. [#8117](https://github.com/ClickHouse/ClickHouse/pull/8117) ([hcz](https://github.com/hczhcz)) -* Speed up parsing of data files in `CSV`, `TSV` and `JSONEachRow` format by doing it in parallel. [#7780](https://github.com/ClickHouse/ClickHouse/pull/7780) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Add function `bankerRound` which performs banker's rounding. [#8112](https://github.com/ClickHouse/ClickHouse/pull/8112) ([hcz](https://github.com/hczhcz)) -* Support more languages in embedded dictionary for region names: 'ru', 'en', 'ua', 'uk', 'by', 'kz', 'tr', 'de', 'uz', 'lv', 'lt', 'et', 'pt', 'he', 'vi'. [#8189](https://github.com/ClickHouse/ClickHouse/pull/8189) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improvements in consistency of `ANY JOIN` logic. Now `t1 ANY LEFT JOIN t2` equals `t2 ANY RIGHT JOIN t1`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* Add setting `any_join_distinct_right_table_keys` which enables old behaviour for `ANY INNER JOIN`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* Add new `SEMI` and `ANTI JOIN`. Old `ANY INNER JOIN` behaviour now available as `SEMI LEFT JOIN`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* Added `Distributed` format for `File` engine and `file` table function which allows to read from `.bin` files generated by asynchronous inserts into `Distributed` table. [#8535](https://github.com/ClickHouse/ClickHouse/pull/8535) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Add optional reset column argument for `runningAccumulate` which allows to reset aggregation results for each new key value. [#8326](https://github.com/ClickHouse/ClickHouse/pull/8326) ([Sergey Kononenko](https://github.com/kononencheg)) -* Add ability to use ClickHouse as Prometheus endpoint. [#7900](https://github.com/ClickHouse/ClickHouse/pull/7900) ([vdimir](https://github.com/Vdimir)) -* Add section `` in `config.xml` which restricts allowed hosts for remote table engines and table functions `URL`, `S3`, `HDFS`. [#7154](https://github.com/ClickHouse/ClickHouse/pull/7154) ([Mikhail Korotov](https://github.com/millb)) -* Added function `greatCircleAngle` which calculates the distance on a sphere in degrees. [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Changed Earth radius to be consistent with H3 library. [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added `JSONCompactEachRow` and `JSONCompactEachRowWithNamesAndTypes` formats for input and output. [#7841](https://github.com/ClickHouse/ClickHouse/pull/7841) ([Mikhail Korotov](https://github.com/millb)) -* Added feature for file-related table engines and table functions (`File`, `S3`, `URL`, `HDFS`) which allows to read and write `gzip` files based on additional engine parameter or file extension. [#7840](https://github.com/ClickHouse/ClickHouse/pull/7840) ([Andrey Bodrov](https://github.com/apbodrov)) -* Added the `randomASCII(length)` function, generating a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. [#8401](https://github.com/ClickHouse/ClickHouse/pull/8401) ([BayoNet](https://github.com/BayoNet)) -* Added function `JSONExtractArrayRaw` which returns an array on unparsed json array elements from `JSON` string. [#8081](https://github.com/ClickHouse/ClickHouse/pull/8081) ([Oleg Matrokhin](https://github.com/errx)) -* Add `arrayZip` function which allows to combine multiple arrays of equal lengths into one array of tuples. [#8149](https://github.com/ClickHouse/ClickHouse/pull/8149) ([Winter Zhang](https://github.com/zhang2014)) -* Add ability to move data between disks according to configured `TTL`-expressions for `*MergeTree` table engines family. [#8140](https://github.com/ClickHouse/ClickHouse/pull/8140) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Added new aggregate function `avgWeighted` which allows to calculate weighted average. [#7898](https://github.com/ClickHouse/ClickHouse/pull/7898) ([Andrey Bodrov](https://github.com/apbodrov)) -* Now parallel parsing is enabled by default for `TSV`, `TSKV`, `CSV` and `JSONEachRow` formats. [#7894](https://github.com/ClickHouse/ClickHouse/pull/7894) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Add several geo functions from `H3` library: `h3GetResolution`, `h3EdgeAngle`, `h3EdgeLength`, `h3IsValid` and `h3kRing`. [#8034](https://github.com/ClickHouse/ClickHouse/pull/8034) ([Konstantin Malanchev](https://github.com/hombit)) -* Added support for brotli (`br`) compression in file-related storages and table functions. This fixes [#8156](https://github.com/ClickHouse/ClickHouse/issues/8156). [#8526](https://github.com/ClickHouse/ClickHouse/pull/8526) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add `groupBit*` functions for the `SimpleAggregationFunction` type. [#8485](https://github.com/ClickHouse/ClickHouse/pull/8485) ([Guillaume Tassery](https://github.com/YiuRULE)) - -#### Bug Fix -* Fix rename of tables with `Distributed` engine. Fixes issue [#7868](https://github.com/ClickHouse/ClickHouse/issues/7868). [#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) -* Now dictionaries support `EXPRESSION` for attributes in arbitrary string in non-ClickHouse SQL dialect. [#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) -* Fix broken `INSERT SELECT FROM mysql(...)` query. This fixes [#8070](https://github.com/ClickHouse/ClickHouse/issues/8070) and [#7960](https://github.com/ClickHouse/ClickHouse/issues/7960). [#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) -* Fix error "Mismatch column sizes" when inserting default `Tuple` from `JSONEachRow`. This fixes [#5653](https://github.com/ClickHouse/ClickHouse/issues/5653). [#8606](https://github.com/ClickHouse/ClickHouse/pull/8606) ([tavplubix](https://github.com/tavplubix)) -* Now an exception will be thrown in case of using `WITH TIES` alongside `LIMIT BY`. Also add ability to use `TOP` with `LIMIT BY`. This fixes [#7472](https://github.com/ClickHouse/ClickHouse/issues/7472). [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix unintendent dependency from fresh glibc version in `clickhouse-odbc-bridge` binary. [#8046](https://github.com/ClickHouse/ClickHouse/pull/8046) ([Amos Bird](https://github.com/amosbird)) -* Fix bug in check function of `*MergeTree` engines family. Now it doesn't fail in case when we have equal amount of rows in last granule and last mark (non-final). [#8047](https://github.com/ClickHouse/ClickHouse/pull/8047) ([alesapin](https://github.com/alesapin)) -* Fix insert into `Enum*` columns after `ALTER` query, when underlying numeric type is equal to table specified type. This fixes [#7836](https://github.com/ClickHouse/ClickHouse/issues/7836). [#7908](https://github.com/ClickHouse/ClickHouse/pull/7908) ([Anton Popov](https://github.com/CurtizJ)) -* Allowed non-constant negative "size" argument for function `substring`. It was not allowed by mistake. This fixes [#4832](https://github.com/ClickHouse/ClickHouse/issues/4832). [#7703](https://github.com/ClickHouse/ClickHouse/pull/7703) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix parsing bug when wrong number of arguments passed to `(O|J)DBC` table engine. [#7709](https://github.com/ClickHouse/ClickHouse/pull/7709) ([alesapin](https://github.com/alesapin)) -* Using command name of the running clickhouse process when sending logs to syslog. In previous versions, empty string was used instead of command name. [#8460](https://github.com/ClickHouse/ClickHouse/pull/8460) ([Michael Nacharov](https://github.com/mnach)) -* Fix check of allowed hosts for `localhost`. This PR fixes the solution provided in [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241). [#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix rare crash in `argMin` and `argMax` functions for long string arguments, when result is used in `runningAccumulate` function. This fixes [#8325](https://github.com/ClickHouse/ClickHouse/issues/8325) [#8341](https://github.com/ClickHouse/ClickHouse/pull/8341) ([dinosaur](https://github.com/769344359)) -* Fix memory overcommit for tables with `Buffer` engine. [#8345](https://github.com/ClickHouse/ClickHouse/pull/8345) ([Azat Khuzhin](https://github.com/azat)) -* Fixed potential bug in functions that can take `NULL` as one of the arguments and return non-NULL. [#8196](https://github.com/ClickHouse/ClickHouse/pull/8196) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better metrics calculations in thread pool for background processes for `MergeTree` table engines. [#8194](https://github.com/ClickHouse/ClickHouse/pull/8194) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix function `IN` inside `WHERE` statement when row-level table filter is present. Fixes [#6687](https://github.com/ClickHouse/ClickHouse/issues/6687) [#8357](https://github.com/ClickHouse/ClickHouse/pull/8357) ([Ivan](https://github.com/abyss7)) -* Now an exception is thrown if the integral value is not parsed completely for settings values. [#7678](https://github.com/ClickHouse/ClickHouse/pull/7678) ([Mikhail Korotov](https://github.com/millb)) -* Fix exception when aggregate function is used in query to distributed table with more than two local shards. [#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) -* Now bloom filter can handle zero length arrays and doesn't perform redundant calculations. [#8242](https://github.com/ClickHouse/ClickHouse/pull/8242) ([achimbab](https://github.com/achimbab)) -* Fixed checking if a client host is allowed by matching the client host to `host_regexp` specified in `users.xml`. [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241) ([Vitaly Baranov](https://github.com/vitlibar)) -* Relax ambiguous column check that leads to false positives in multiple `JOIN ON` section. [#8385](https://github.com/ClickHouse/ClickHouse/pull/8385) ([Artem Zuikov](https://github.com/4ertus2)) -* Fixed possible server crash (`std::terminate`) when the server cannot send or write data in `JSON` or `XML` format with values of `String` data type (that require `UTF-8` validation) or when compressing result data with Brotli algorithm or in some other rare cases. This fixes [#7603](https://github.com/ClickHouse/ClickHouse/issues/7603) [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix race condition in `StorageDistributedDirectoryMonitor` found by CI. This fixes [#8364](https://github.com/ClickHouse/ClickHouse/issues/8364). [#8383](https://github.com/ClickHouse/ClickHouse/pull/8383) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now background merges in `*MergeTree` table engines family preserve storage policy volume order more accurately. [#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Now table engine `Kafka` works properly with `Native` format. This fixes [#6731](https://github.com/ClickHouse/ClickHouse/issues/6731) [#7337](https://github.com/ClickHouse/ClickHouse/issues/7337) [#8003](https://github.com/ClickHouse/ClickHouse/issues/8003). [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) -* Fixed formats with headers (like `CSVWithNames`) which were throwing exception about EOF for table engine `Kafka`. [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) -* Fixed a bug with making set from subquery in right part of `IN` section. This fixes [#5767](https://github.com/ClickHouse/ClickHouse/issues/5767) and [#2542](https://github.com/ClickHouse/ClickHouse/issues/2542). [#7755](https://github.com/ClickHouse/ClickHouse/pull/7755) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix possible crash while reading from storage `File`. [#7756](https://github.com/ClickHouse/ClickHouse/pull/7756) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed reading of the files in `Parquet` format containing columns of type `list`. [#8334](https://github.com/ClickHouse/ClickHouse/pull/8334) ([maxulan](https://github.com/maxulan)) -* Fix error `Not found column` for distributed queries with `PREWHERE` condition dependent on sampling key if `max_parallel_replicas > 1`. [#7913](https://github.com/ClickHouse/ClickHouse/pull/7913) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix error `Not found column` if query used `PREWHERE` dependent on table's alias and the result set was empty because of primary key condition. [#7911](https://github.com/ClickHouse/ClickHouse/pull/7911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed return type for functions `rand` and `randConstant` in case of `Nullable` argument. Now functions always return `UInt32` and never `Nullable(UInt32)`. [#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Disabled predicate push-down for `WITH FILL` expression. This fixes [#7784](https://github.com/ClickHouse/ClickHouse/issues/7784). [#7789](https://github.com/ClickHouse/ClickHouse/pull/7789) ([Winter Zhang](https://github.com/zhang2014)) -* Fixed incorrect `count()` result for `SummingMergeTree` when `FINAL` section is used. [#3280](https://github.com/ClickHouse/ClickHouse/issues/3280) [#7786](https://github.com/ClickHouse/ClickHouse/pull/7786) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix possible incorrect result for constant functions from remote servers. It happened for queries with functions like `version()`, `uptime()`, etc. which returns different constant values for different servers. This fixes [#7666](https://github.com/ClickHouse/ClickHouse/issues/7666). [#7689](https://github.com/ClickHouse/ClickHouse/pull/7689) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix complicated bug in push-down predicate optimization which leads to wrong results. This fixes a lot of issues on push-down predicate optimization. [#8503](https://github.com/ClickHouse/ClickHouse/pull/8503) ([Winter Zhang](https://github.com/zhang2014)) -* Fix crash in `CREATE TABLE .. AS dictionary` query. [#8508](https://github.com/ClickHouse/ClickHouse/pull/8508) ([Azat Khuzhin](https://github.com/azat)) -* Several improvements ClickHouse grammar in `.g4` file. [#8294](https://github.com/ClickHouse/ClickHouse/pull/8294) ([taiyang-li](https://github.com/taiyang-li)) -* Fix bug that leads to crashes in `JOIN`s with tables with engine `Join`. This fixes [#7556](https://github.com/ClickHouse/ClickHouse/issues/7556) [#8254](https://github.com/ClickHouse/ClickHouse/issues/8254) [#7915](https://github.com/ClickHouse/ClickHouse/issues/7915) [#8100](https://github.com/ClickHouse/ClickHouse/issues/8100). [#8298](https://github.com/ClickHouse/ClickHouse/pull/8298) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix redundant dictionaries reload on `CREATE DATABASE`. [#7916](https://github.com/ClickHouse/ClickHouse/pull/7916) ([Azat Khuzhin](https://github.com/azat)) -* Limit maximum number of streams for read from `StorageFile` and `StorageHDFS`. Fixes https://github.com/ClickHouse/ClickHouse/issues/7650. [#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) -* Fix bug in `ALTER ... MODIFY ... CODEC` query, when user specify both default expression and codec. Fixes [8593](https://github.com/ClickHouse/ClickHouse/issues/8593). [#8614](https://github.com/ClickHouse/ClickHouse/pull/8614) ([alesapin](https://github.com/alesapin)) -* Fix error in background merge of columns with `SimpleAggregateFunction(LowCardinality)` type. [#8613](https://github.com/ClickHouse/ClickHouse/pull/8613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed type check in function `toDateTime64`. [#8375](https://github.com/ClickHouse/ClickHouse/pull/8375) ([Vasily Nemkov](https://github.com/Enmk)) -* Now server do not crash on `LEFT` or `FULL JOIN` with and Join engine and unsupported `join_use_nulls` settings. [#8479](https://github.com/ClickHouse/ClickHouse/pull/8479) ([Artem Zuikov](https://github.com/4ertus2)) -* Now `DROP DICTIONARY IF EXISTS db.dict` query doesn't throw exception if `db` doesn't exist. [#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix possible crashes in table functions (`file`, `mysql`, `remote`) caused by usage of reference to removed `IStorage` object. Fix incorrect parsing of columns specified at insertion into table function. [#7762](https://github.com/ClickHouse/ClickHouse/pull/7762) ([tavplubix](https://github.com/tavplubix)) -* Ensure network be up before starting `clickhouse-server`. This fixes [#7507](https://github.com/ClickHouse/ClickHouse/issues/7507). [#8570](https://github.com/ClickHouse/ClickHouse/pull/8570) ([Zhichang Yu](https://github.com/yuzhichang)) -* Fix timeouts handling for secure connections, so queries doesn't hang indefenitely. This fixes [#8126](https://github.com/ClickHouse/ClickHouse/issues/8126). [#8128](https://github.com/ClickHouse/ClickHouse/pull/8128) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix `clickhouse-copier`'s redundant contention between concurrent workers. [#7816](https://github.com/ClickHouse/ClickHouse/pull/7816) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* Now mutations doesn't skip attached parts, even if their mutation version were larger than current mutation version. [#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) [#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) -* Ignore redundant copies of `*MergeTree` data parts after move to another disk and server restart. [#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix crash in `FULL JOIN` with `LowCardinality` in `JOIN` key. [#8252](https://github.com/ClickHouse/ClickHouse/pull/8252) ([Artem Zuikov](https://github.com/4ertus2)) -* Forbidden to use column name more than once in insert query like `INSERT INTO tbl (x, y, x)`. This fixes [#5465](https://github.com/ClickHouse/ClickHouse/issues/5465), [#7681](https://github.com/ClickHouse/ClickHouse/issues/7681). [#7685](https://github.com/ClickHouse/ClickHouse/pull/7685) ([alesapin](https://github.com/alesapin)) -* Added fallback for detection the number of physical CPU cores for unknown CPUs (using the number of logical CPU cores). This fixes [#5239](https://github.com/ClickHouse/ClickHouse/issues/5239). [#7726](https://github.com/ClickHouse/ClickHouse/pull/7726) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix `There's no column` error for materialized and alias columns. [#8210](https://github.com/ClickHouse/ClickHouse/pull/8210) ([Artem Zuikov](https://github.com/4ertus2)) -* Fixed sever crash when `EXISTS` query was used without `TABLE` or `DICTIONARY` qualifier. Just like `EXISTS t`. This fixes [#8172](https://github.com/ClickHouse/ClickHouse/issues/8172). This bug was introduced in version 19.17. [#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix rare bug with error `"Sizes of columns doesn't match"` that might appear when using `SimpleAggregateFunction` column. [#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) -* Fix bug where user with empty `allow_databases` got access to all databases (and same for `allow_dictionaries`). [#7793](https://github.com/ClickHouse/ClickHouse/pull/7793) ([DeifyTheGod](https://github.com/DeifyTheGod)) -* Fix client crash when server already disconnected from client. [#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) -* Fix `ORDER BY` behaviour in case of sorting by primary key prefix and non primary key suffix. [#7759](https://github.com/ClickHouse/ClickHouse/pull/7759) ([Anton Popov](https://github.com/CurtizJ)) -* Check if qualified column present in the table. This fixes [#6836](https://github.com/ClickHouse/ClickHouse/issues/6836). [#7758](https://github.com/ClickHouse/ClickHouse/pull/7758) ([Artem Zuikov](https://github.com/4ertus2)) -* Fixed behavior with `ALTER MOVE` ran immediately after merge finish moves superpart of specified. Fixes [#8103](https://github.com/ClickHouse/ClickHouse/issues/8103). [#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix possible server crash while using `UNION` with different number of columns. Fixes [#7279](https://github.com/ClickHouse/ClickHouse/issues/7279). [#7929](https://github.com/ClickHouse/ClickHouse/pull/7929) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix size of result substring for function `substr` with negative size. [#8589](https://github.com/ClickHouse/ClickHouse/pull/8589) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now server does not execute part mutation in `MergeTree` if there are not enough free threads in background pool. [#8588](https://github.com/ClickHouse/ClickHouse/pull/8588) ([tavplubix](https://github.com/tavplubix)) -* Fix a minor typo on formatting `UNION ALL` AST. [#7999](https://github.com/ClickHouse/ClickHouse/pull/7999) ([litao91](https://github.com/litao91)) -* Fixed incorrect bloom filter results for negative numbers. This fixes [#8317](https://github.com/ClickHouse/ClickHouse/issues/8317). [#8566](https://github.com/ClickHouse/ClickHouse/pull/8566) ([Winter Zhang](https://github.com/zhang2014)) -* Fixed potential buffer overflow in decompress. Malicious user can pass fabricated compressed data that will cause read after buffer. This issue was found by Eldar Zaitov from Yandex information security team. [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix incorrect result because of integers overflow in `arrayIntersect`. [#7777](https://github.com/ClickHouse/ClickHouse/pull/7777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now `OPTIMIZE TABLE` query will not wait for offline replicas to perform the operation. [#8314](https://github.com/ClickHouse/ClickHouse/pull/8314) ([javi santana](https://github.com/javisantana)) -* Fixed `ALTER TTL` parser for `Replicated*MergeTree` tables. [#8318](https://github.com/ClickHouse/ClickHouse/pull/8318) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix communication between server and client, so server read temporary tables info after query failure. [#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) -* Fix `bitmapAnd` function error when intersecting an aggregated bitmap and a scalar bitmap. [#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) -* Refine the definition of `ZXid` according to the ZooKeeper Programmer's Guide which fixes bug in `clickhouse-cluster-copier`. [#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* `odbc` table function now respects `external_table_functions_use_nulls` setting. [#7506](https://github.com/ClickHouse/ClickHouse/pull/7506) ([Vasily Nemkov](https://github.com/Enmk)) -* Fixed bug that lead to a rare data race. [#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alexander Kazakov](https://github.com/Akazz)) -* Now `SYSTEM RELOAD DICTIONARY` reloads a dictionary completely, ignoring `update_field`. This fixes [#7440](https://github.com/ClickHouse/ClickHouse/issues/7440). [#8037](https://github.com/ClickHouse/ClickHouse/pull/8037) ([Vitaly Baranov](https://github.com/vitlibar)) -* Add ability to check if dictionary exists in create query. [#8032](https://github.com/ClickHouse/ClickHouse/pull/8032) ([alesapin](https://github.com/alesapin)) -* Fix `Float*` parsing in `Values` format. This fixes [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817). [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) -* Fix crash when we cannot reserve space in some background operations of `*MergeTree` table engines family. [#7873](https://github.com/ClickHouse/ClickHouse/pull/7873) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix crash of merge operation when table contains `SimpleAggregateFunction(LowCardinality)` column. This fixes [#8515](https://github.com/ClickHouse/ClickHouse/issues/8515). [#8522](https://github.com/ClickHouse/ClickHouse/pull/8522) ([Azat Khuzhin](https://github.com/azat)) -* Restore support of all ICU locales and add the ability to apply collations for constant expressions. Also add language name to `system.collations` table. [#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) -* Fix bug when external dictionaries with zero minimal lifetime (`LIFETIME(MIN 0 MAX N)`, `LIFETIME(N)`) don't update in background. [#7983](https://github.com/ClickHouse/ClickHouse/pull/7983) ([alesapin](https://github.com/alesapin)) -* Fix crash when external dictionary with ClickHouse source has subquery in query. [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix incorrect parsing of file extension in table with engine `URL`. This fixes [#8157](https://github.com/ClickHouse/ClickHouse/issues/8157). [#8419](https://github.com/ClickHouse/ClickHouse/pull/8419) ([Andrey Bodrov](https://github.com/apbodrov)) -* Fix `CHECK TABLE` query for `*MergeTree` tables without key. Fixes [#7543](https://github.com/ClickHouse/ClickHouse/issues/7543). [#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) -* Fixed conversion of `Float64` to MySQL type. [#8079](https://github.com/ClickHouse/ClickHouse/pull/8079) ([Yuriy Baranov](https://github.com/yurriy)) -* Now if table was not completely dropped because of server crash, server will try to restore and load it. [#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) -* Fixed crash in table function `file` while inserting into file that doesn't exist. Now in this case file would be created and then insert would be processed. [#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) -* Fix rare deadlock which can happen when `trace_log` is in enabled. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) -* Add ability to work with different types besides `Date` in `RangeHashed` external dictionary created from DDL query. Fixes [7899](https://github.com/ClickHouse/ClickHouse/issues/7899). [#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) -* Fixes crash when `now64()` is called with result of another function. [#8270](https://github.com/ClickHouse/ClickHouse/pull/8270) ([Vasily Nemkov](https://github.com/Enmk)) -* Fixed bug with detecting client IP for connections through mysql wire protocol. [#7743](https://github.com/ClickHouse/ClickHouse/pull/7743) ([Dmitry Muzyka](https://github.com/dmitriy-myz)) -* Fix empty array handling in `arraySplit` function. This fixes [#7708](https://github.com/ClickHouse/ClickHouse/issues/7708). [#7747](https://github.com/ClickHouse/ClickHouse/pull/7747) ([hcz](https://github.com/hczhcz)) -* Fixed the issue when `pid-file` of another running `clickhouse-server` may be deleted. [#8487](https://github.com/ClickHouse/ClickHouse/pull/8487) ([Weiqing Xu](https://github.com/weiqxu)) -* Fix dictionary reload if it has `invalidate_query`, which stopped updates and some exception on previous update tries. [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) -* Fixed error in function `arrayReduce` that may lead to "double free" and error in aggregate function combinator `Resample` that may lead to memory leak. Added aggregate function `aggThrow`. This function can be used for testing purposes. [#8446](https://github.com/ClickHouse/ClickHouse/pull/8446) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### Improvement -* Improved logging when working with `S3` table engine. [#8251](https://github.com/ClickHouse/ClickHouse/pull/8251) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) -* Printed help message when no arguments are passed when calling `clickhouse-local`. This fixes [#5335](https://github.com/ClickHouse/ClickHouse/issues/5335). [#8230](https://github.com/ClickHouse/ClickHouse/pull/8230) ([Andrey Nagorny](https://github.com/Melancholic)) -* Add setting `mutations_sync` which allows to wait `ALTER UPDATE/DELETE` queries synchronously. [#8237](https://github.com/ClickHouse/ClickHouse/pull/8237) ([alesapin](https://github.com/alesapin)) -* Allow to set up relative `user_files_path` in `config.xml` (in the way similar to `format_schema_path`). [#7632](https://github.com/ClickHouse/ClickHouse/pull/7632) ([hcz](https://github.com/hczhcz)) -* Add exception for illegal types for conversion functions with `-OrZero` postfix. [#7880](https://github.com/ClickHouse/ClickHouse/pull/7880) ([Andrey Konyaev](https://github.com/akonyaev90)) -* Simplify format of the header of data sending to a shard in a distributed query. [#8044](https://github.com/ClickHouse/ClickHouse/pull/8044) ([Vitaly Baranov](https://github.com/vitlibar)) -* `Live View` table engine refactoring. [#8519](https://github.com/ClickHouse/ClickHouse/pull/8519) ([vzakaznikov](https://github.com/vzakaznikov)) -* Add additional checks for external dictionaries created from DDL-queries. [#8127](https://github.com/ClickHouse/ClickHouse/pull/8127) ([alesapin](https://github.com/alesapin)) -* Fix error `Column ... already exists` while using `FINAL` and `SAMPLE` together, e.g. `select count() from table final sample 1/2`. Fixes [#5186](https://github.com/ClickHouse/ClickHouse/issues/5186). [#7907](https://github.com/ClickHouse/ClickHouse/pull/7907) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now table the first argument of `joinGet` function can be table indentifier. [#7707](https://github.com/ClickHouse/ClickHouse/pull/7707) ([Amos Bird](https://github.com/amosbird)) -* Allow using `MaterializedView` with subqueries above `Kafka` tables. [#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) -* Now background moves between disks run it the seprate thread pool. [#7670](https://github.com/ClickHouse/ClickHouse/pull/7670) ([Vladimir Chebotarev](https://github.com/excitoon)) -* `SYSTEM RELOAD DICTIONARY` now executes synchronously. [#8240](https://github.com/ClickHouse/ClickHouse/pull/8240) ([Vitaly Baranov](https://github.com/vitlibar)) -* Stack traces now display physical addresses (offsets in object file) instead of virtual memory addresses (where the object file was loaded). That allows the use of `addr2line` when binary is position independent and ASLR is active. This fixes [#8360](https://github.com/ClickHouse/ClickHouse/issues/8360). [#8387](https://github.com/ClickHouse/ClickHouse/pull/8387) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Support new syntax for row-level security filters: `…
      `. Fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([Ivan](https://github.com/abyss7)) -* Now `cityHash` function can work with `Decimal` and `UUID` types. Fixes [#5184](https://github.com/ClickHouse/ClickHouse/issues/5184). [#7693](https://github.com/ClickHouse/ClickHouse/pull/7693) ([Mikhail Korotov](https://github.com/millb)) -* Removed fixed index granularity (it was 1024) from system logs because it's obsolete after implementation of adaptive granularity. [#7698](https://github.com/ClickHouse/ClickHouse/pull/7698) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Enabled MySQL compatibility server when ClickHouse is compiled without SSL. [#7852](https://github.com/ClickHouse/ClickHouse/pull/7852) ([Yuriy Baranov](https://github.com/yurriy)) -* Now server checksums distributed batches, which gives more verbose errors in case of corrupted data in batch. [#7914](https://github.com/ClickHouse/ClickHouse/pull/7914) ([Azat Khuzhin](https://github.com/azat)) -* Support `DROP DATABASE`, `DETACH TABLE`, `DROP TABLE` and `ATTACH TABLE` for `MySQL` database engine. [#8202](https://github.com/ClickHouse/ClickHouse/pull/8202) ([Winter Zhang](https://github.com/zhang2014)) -* Add authentication in S3 table function and table engine. [#7623](https://github.com/ClickHouse/ClickHouse/pull/7623) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Added check for extra parts of `MergeTree` at different disks, in order to not allow to miss data parts at undefined disks. [#8118](https://github.com/ClickHouse/ClickHouse/pull/8118) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Enable SSL support for Mac client and server. [#8297](https://github.com/ClickHouse/ClickHouse/pull/8297) ([Ivan](https://github.com/abyss7)) -* Now ClickHouse can work as MySQL federated server (see https://dev.mysql.com/doc/refman/5.7/en/federated-create-server.html). [#7717](https://github.com/ClickHouse/ClickHouse/pull/7717) ([Maxim Fedotov](https://github.com/MaxFedotov)) -* `clickhouse-client` now only enable `bracketed-paste` when multiquery is on and multiline is off. This fixes (#7757)[https://github.com/ClickHouse/ClickHouse/issues/7757]. [#7761](https://github.com/ClickHouse/ClickHouse/pull/7761) ([Amos Bird](https://github.com/amosbird)) -* Support `Array(Decimal)` in `if` function. [#7721](https://github.com/ClickHouse/ClickHouse/pull/7721) ([Artem Zuikov](https://github.com/4ertus2)) -* Support Decimals in `arrayDifference`, `arrayCumSum` and `arrayCumSumNegative` functions. [#7724](https://github.com/ClickHouse/ClickHouse/pull/7724) ([Artem Zuikov](https://github.com/4ertus2)) -* Added `lifetime` column to `system.dictionaries` table. [#6820](https://github.com/ClickHouse/ClickHouse/issues/6820) [#7727](https://github.com/ClickHouse/ClickHouse/pull/7727) ([kekekekule](https://github.com/kekekekule)) -* Improved check for existing parts on different disks for `*MergeTree` table engines. Addresses [#7660](https://github.com/ClickHouse/ClickHouse/issues/7660). [#8440](https://github.com/ClickHouse/ClickHouse/pull/8440) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Integration with `AWS SDK` for `S3` interactions which allows to use all S3 features out of the box. [#8011](https://github.com/ClickHouse/ClickHouse/pull/8011) ([Pavel Kovalenko](https://github.com/Jokser)) -* Added support for subqueries in `Live View` tables. [#7792](https://github.com/ClickHouse/ClickHouse/pull/7792) ([vzakaznikov](https://github.com/vzakaznikov)) -* Check for using `Date` or `DateTime` column from `TTL` expressions was removed. [#7920](https://github.com/ClickHouse/ClickHouse/pull/7920) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Information about disk was added to `system.detached_parts` table. [#7833](https://github.com/ClickHouse/ClickHouse/pull/7833) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Now settings `max_(table|partition)_size_to_drop` can be changed without a restart. [#7779](https://github.com/ClickHouse/ClickHouse/pull/7779) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) -* Slightly better usability of error messages. Ask user not to remove the lines below `Stack trace:`. [#7897](https://github.com/ClickHouse/ClickHouse/pull/7897) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better reading messages from `Kafka` engine in various formats after [#7935](https://github.com/ClickHouse/ClickHouse/issues/7935). [#8035](https://github.com/ClickHouse/ClickHouse/pull/8035) ([Ivan](https://github.com/abyss7)) -* Better compatibility with MySQL clients which don't support `sha2_password` auth plugin. [#8036](https://github.com/ClickHouse/ClickHouse/pull/8036) ([Yuriy Baranov](https://github.com/yurriy)) -* Support more column types in MySQL compatibility server. [#7975](https://github.com/ClickHouse/ClickHouse/pull/7975) ([Yuriy Baranov](https://github.com/yurriy)) -* Implement `ORDER BY` optimization for `Merge`, `Buffer` and `Materilized View` storages with underlying `MergeTree` tables. [#8130](https://github.com/ClickHouse/ClickHouse/pull/8130) ([Anton Popov](https://github.com/CurtizJ)) -* Now we always use POSIX implementation of `getrandom` to have better compatibility with old kernels (< 3.17). [#7940](https://github.com/ClickHouse/ClickHouse/pull/7940) ([Amos Bird](https://github.com/amosbird)) -* Better check for valid destination in a move TTL rule. [#8410](https://github.com/ClickHouse/ClickHouse/pull/8410) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Better checks for broken insert batches for `Distributed` table engine. [#7933](https://github.com/ClickHouse/ClickHouse/pull/7933) ([Azat Khuzhin](https://github.com/azat)) -* Add column with array of parts name which mutations must process in future to `system.mutations` table. [#8179](https://github.com/ClickHouse/ClickHouse/pull/8179) ([alesapin](https://github.com/alesapin)) -* Parallel merge sort optimization for processors. [#8552](https://github.com/ClickHouse/ClickHouse/pull/8552) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* The settings `mark_cache_min_lifetime` is now obsolete and does nothing. In previous versions, mark cache can grow in memory larger than `mark_cache_size` to accomodate data within `mark_cache_min_lifetime` seconds. That was leading to confusion and higher memory usage than expected, that is especially bad on memory constrained systems. If you will see performance degradation after installing this release, you should increase the `mark_cache_size`. [#8484](https://github.com/ClickHouse/ClickHouse/pull/8484) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Preparation to use `tid` everywhere. This is needed for [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477). [#8276](https://github.com/ClickHouse/ClickHouse/pull/8276) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### Performance Improvement -* Performance optimizations in processors pipeline. [#7988](https://github.com/ClickHouse/ClickHouse/pull/7988) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Non-blocking updates of expired keys in cache dictionaries (with permission to read old ones). [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Compile ClickHouse without `-fno-omit-frame-pointer` globally to spare one more register. [#8097](https://github.com/ClickHouse/ClickHouse/pull/8097) ([Amos Bird](https://github.com/amosbird)) -* Speedup `greatCircleDistance` function and add performance tests for it. [#7307](https://github.com/ClickHouse/ClickHouse/pull/7307) ([Olga Khvostikova](https://github.com/stavrolia)) -* Improved performance of function `roundDown`. [#8465](https://github.com/ClickHouse/ClickHouse/pull/8465) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improved performance of `max`, `min`, `argMin`, `argMax` for `DateTime64` data type. [#8199](https://github.com/ClickHouse/ClickHouse/pull/8199) ([Vasily Nemkov](https://github.com/Enmk)) -* Improved performance of sorting without a limit or with big limit and external sorting. [#8545](https://github.com/ClickHouse/ClickHouse/pull/8545) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improved performance of formatting floating point numbers up to 6 times. [#8542](https://github.com/ClickHouse/ClickHouse/pull/8542) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improved performance of `modulo` function. [#7750](https://github.com/ClickHouse/ClickHouse/pull/7750) ([Amos Bird](https://github.com/amosbird)) -* Optimized `ORDER BY` and merging with single column key. [#8335](https://github.com/ClickHouse/ClickHouse/pull/8335) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better implementation for `arrayReduce`, `-Array` and `-State` combinators. [#7710](https://github.com/ClickHouse/ClickHouse/pull/7710) ([Amos Bird](https://github.com/amosbird)) -* Now `PREWHERE` should be optimized to be at least as efficient as `WHERE`. [#7769](https://github.com/ClickHouse/ClickHouse/pull/7769) ([Amos Bird](https://github.com/amosbird)) -* Improve the way `round` and `roundBankers` handling negative numbers. [#8229](https://github.com/ClickHouse/ClickHouse/pull/8229) ([hcz](https://github.com/hczhcz)) -* Improved decoding performance of `DoubleDelta` and `Gorilla` codecs by roughly 30-40%. This fixes [#7082](https://github.com/ClickHouse/ClickHouse/issues/7082). [#8019](https://github.com/ClickHouse/ClickHouse/pull/8019) ([Vasily Nemkov](https://github.com/Enmk)) -* Improved performance of `base64` related functions. [#8444](https://github.com/ClickHouse/ClickHouse/pull/8444) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added a function `geoDistance`. It is similar to `greatCircleDistance` but uses approximation to WGS-84 ellipsoid model. The performance of both functions are near the same. [#8086](https://github.com/ClickHouse/ClickHouse/pull/8086) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Faster `min` and `max` aggregation functions for `Decimal` data type. [#8144](https://github.com/ClickHouse/ClickHouse/pull/8144) ([Artem Zuikov](https://github.com/4ertus2)) -* Vectorize processing `arrayReduce`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird)) -* `if` chains are now optimized as `multiIf`. [#8355](https://github.com/ClickHouse/ClickHouse/pull/8355) ([kamalov-ruslan](https://github.com/kamalov-ruslan)) -* Fix performance regression of `Kafka` table engine introduced in 19.15. This fixes [#7261](https://github.com/ClickHouse/ClickHouse/issues/7261). [#7935](https://github.com/ClickHouse/ClickHouse/pull/7935) ([filimonov](https://github.com/filimonov)) -* Removed "pie" code generation that `gcc` from Debian packages occasionally brings by default. [#8483](https://github.com/ClickHouse/ClickHouse/pull/8483) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Parallel parsing data formats [#6553](https://github.com/ClickHouse/ClickHouse/pull/6553) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Enable optimized parser of `Values` with expressions by default (`input_format_values_deduce_templates_of_expressions=1`). [#8231](https://github.com/ClickHouse/ClickHouse/pull/8231) ([tavplubix](https://github.com/tavplubix)) - -#### Build/Testing/Packaging Improvement -* Build fixes for `ARM` and in minimal mode. [#8304](https://github.com/ClickHouse/ClickHouse/pull/8304) ([proller](https://github.com/proller)) -* Add coverage file flush for `clickhouse-server` when std::atexit is not called. Also slightly improved logging in stateless tests with coverage. [#8267](https://github.com/ClickHouse/ClickHouse/pull/8267) ([alesapin](https://github.com/alesapin)) -* Update LLVM library in contrib. Avoid using LLVM from OS packages. [#8258](https://github.com/ClickHouse/ClickHouse/pull/8258) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Make bundled `curl` build fully quiet. [#8232](https://github.com/ClickHouse/ClickHouse/pull/8232) [#8203](https://github.com/ClickHouse/ClickHouse/pull/8203) ([Pavel Kovalenko](https://github.com/Jokser)) -* Fix some `MemorySanitizer` warnings. [#8235](https://github.com/ClickHouse/ClickHouse/pull/8235) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Use `add_warning` and `no_warning` macros in `CMakeLists.txt`. [#8604](https://github.com/ClickHouse/ClickHouse/pull/8604) ([Ivan](https://github.com/abyss7)) -* Add support of Minio S3 Compatible object (https://min.io/) for better integration tests. [#7863](https://github.com/ClickHouse/ClickHouse/pull/7863) [#7875](https://github.com/ClickHouse/ClickHouse/pull/7875) ([Pavel Kovalenko](https://github.com/Jokser)) -* Imported `libc` headers to contrib. It allows to make builds more consistent across various systems (only for `x86_64-linux-gnu`). [#5773](https://github.com/ClickHouse/ClickHouse/pull/5773) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove `-fPIC` from some libraries. [#8464](https://github.com/ClickHouse/ClickHouse/pull/8464) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Clean `CMakeLists.txt` for curl. See https://github.com/ClickHouse/ClickHouse/pull/8011#issuecomment-569478910 [#8459](https://github.com/ClickHouse/ClickHouse/pull/8459) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Silent warnings in `CapNProto` library. [#8220](https://github.com/ClickHouse/ClickHouse/pull/8220) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add performance tests for short string optimized hash tables. [#7679](https://github.com/ClickHouse/ClickHouse/pull/7679) ([Amos Bird](https://github.com/amosbird)) -* Now ClickHouse will build on `AArch64` even if `MADV_FREE` is not available. This fixes [#8027](https://github.com/ClickHouse/ClickHouse/issues/8027). [#8243](https://github.com/ClickHouse/ClickHouse/pull/8243) ([Amos Bird](https://github.com/amosbird)) -* Update `zlib-ng` to fix memory sanitizer problems. [#7182](https://github.com/ClickHouse/ClickHouse/pull/7182) [#8206](https://github.com/ClickHouse/ClickHouse/pull/8206) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Enable internal MySQL library on non-Linux system, because usage of OS packages is very fragile and usually doesn't work at all. This fixes [#5765](https://github.com/ClickHouse/ClickHouse/issues/5765). [#8426](https://github.com/ClickHouse/ClickHouse/pull/8426) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed build on some systems after enabling `libc++`. This supersedes [#8374](https://github.com/ClickHouse/ClickHouse/issues/8374). [#8380](https://github.com/ClickHouse/ClickHouse/pull/8380) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Make `Field` methods more type-safe to find more errors. [#7386](https://github.com/ClickHouse/ClickHouse/pull/7386) [#8209](https://github.com/ClickHouse/ClickHouse/pull/8209) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Added missing files to the `libc-headers` submodule. [#8507](https://github.com/ClickHouse/ClickHouse/pull/8507) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix wrong `JSON` quoting in performance test output. [#8497](https://github.com/ClickHouse/ClickHouse/pull/8497) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now stack trace is displayed for `std::exception` and `Poco::Exception`. In previous versions it was available only for `DB::Exception`. This improves diagnostics. [#8501](https://github.com/ClickHouse/ClickHouse/pull/8501) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Porting `clock_gettime` and `clock_nanosleep` for fresh glibc versions. [#8054](https://github.com/ClickHouse/ClickHouse/pull/8054) ([Amos Bird](https://github.com/amosbird)) -* Enable `part_log` in example config for developers. [#8609](https://github.com/ClickHouse/ClickHouse/pull/8609) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix async nature of reload in `01036_no_superfluous_dict_reload_on_create_database*`. [#8111](https://github.com/ClickHouse/ClickHouse/pull/8111) ([Azat Khuzhin](https://github.com/azat)) -* Fixed codec performance tests. [#8615](https://github.com/ClickHouse/ClickHouse/pull/8615) ([Vasily Nemkov](https://github.com/Enmk)) -* Add install scripts for `.tgz` build and documentation for them. [#8612](https://github.com/ClickHouse/ClickHouse/pull/8612) [#8591](https://github.com/ClickHouse/ClickHouse/pull/8591) ([alesapin](https://github.com/alesapin)) -* Removed old `ZSTD` test (it was created in year 2016 to reproduce the bug that pre 1.0 version of ZSTD has had). This fixes [#8618](https://github.com/ClickHouse/ClickHouse/issues/8618). [#8619](https://github.com/ClickHouse/ClickHouse/pull/8619) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed build on Mac OS Catalina. [#8600](https://github.com/ClickHouse/ClickHouse/pull/8600) ([meo](https://github.com/meob)) -* Increased number of rows in codec performance tests to make results noticeable. [#8574](https://github.com/ClickHouse/ClickHouse/pull/8574) ([Vasily Nemkov](https://github.com/Enmk)) -* In debug builds, treat `LOGICAL_ERROR` exceptions as assertion failures, so that they are easier to notice. [#8475](https://github.com/ClickHouse/ClickHouse/pull/8475) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Make formats-related performance test more deterministic. [#8477](https://github.com/ClickHouse/ClickHouse/pull/8477) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update `lz4` to fix a MemorySanitizer failure. [#8181](https://github.com/ClickHouse/ClickHouse/pull/8181) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Suppress a known MemorySanitizer false positive in exception handling. [#8182](https://github.com/ClickHouse/ClickHouse/pull/8182) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Update `gcc` and `g++` to version 9 in `build/docker/build.sh` [#7766](https://github.com/ClickHouse/ClickHouse/pull/7766) ([TLightSky](https://github.com/tlightsky)) -* Add performance test case to test that `PREWHERE` is worse than `WHERE`. [#7768](https://github.com/ClickHouse/ClickHouse/pull/7768) ([Amos Bird](https://github.com/amosbird)) -* Progress towards fixing one flacky test. [#8621](https://github.com/ClickHouse/ClickHouse/pull/8621) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Avoid MemorySanitizer report for data from `libunwind`. [#8539](https://github.com/ClickHouse/ClickHouse/pull/8539) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Updated `libc++` to the latest version. [#8324](https://github.com/ClickHouse/ClickHouse/pull/8324) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Build ICU library from sources. This fixes [#6460](https://github.com/ClickHouse/ClickHouse/issues/6460). [#8219](https://github.com/ClickHouse/ClickHouse/pull/8219) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Switched from `libressl` to `openssl`. ClickHouse should support TLS 1.3 and SNI after this change. This fixes [#8171](https://github.com/ClickHouse/ClickHouse/issues/8171). [#8218](https://github.com/ClickHouse/ClickHouse/pull/8218) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed UBSan report when using `chacha20_poly1305` from SSL (happens on connect to https://yandex.ru/). [#8214](https://github.com/ClickHouse/ClickHouse/pull/8214) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix mode of default password file for `.deb` linux distros. [#8075](https://github.com/ClickHouse/ClickHouse/pull/8075) ([proller](https://github.com/proller)) -* Improved expression for getting `clickhouse-server` PID in `clickhouse-test`. [#8063](https://github.com/ClickHouse/ClickHouse/pull/8063) ([Alexander Kazakov](https://github.com/Akazz)) -* Updated contrib/googletest to v1.10.0. [#8587](https://github.com/ClickHouse/ClickHouse/pull/8587) ([Alexander Burmak](https://github.com/Alex-Burmak)) -* Fixed ThreadSaninitizer report in `base64` library. Also updated this library to the latest version, but it doesn't matter. This fixes [#8397](https://github.com/ClickHouse/ClickHouse/issues/8397). [#8403](https://github.com/ClickHouse/ClickHouse/pull/8403) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix `00600_replace_running_query` for processors. [#8272](https://github.com/ClickHouse/ClickHouse/pull/8272) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Remove support for `tcmalloc` to make `CMakeLists.txt` simpler. [#8310](https://github.com/ClickHouse/ClickHouse/pull/8310) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Release gcc builds now use `libc++` instead of `libstdc++`. Recently `libc++` was used only with clang. This will improve consistency of build configurations and portability. [#8311](https://github.com/ClickHouse/ClickHouse/pull/8311) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Enable ICU library for build with MemorySanitizer. [#8222](https://github.com/ClickHouse/ClickHouse/pull/8222) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Suppress warnings from `CapNProto` library. [#8224](https://github.com/ClickHouse/ClickHouse/pull/8224) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Removed special cases of code for `tcmalloc`, because it's no longer supported. [#8225](https://github.com/ClickHouse/ClickHouse/pull/8225) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* In CI coverage task, kill the server gracefully to allow it to save the coverage report. This fixes incomplete coverage reports we've been seeing lately. [#8142](https://github.com/ClickHouse/ClickHouse/pull/8142) ([alesapin](https://github.com/alesapin)) -* Performance tests for all codecs against `Float64` and `UInt64` values. [#8349](https://github.com/ClickHouse/ClickHouse/pull/8349) ([Vasily Nemkov](https://github.com/Enmk)) -* `termcap` is very much deprecated and lead to various problems (f.g. missing "up" cap and echoing `^J` instead of multi line) . Favor `terminfo` or bundled `ncurses`. [#7737](https://github.com/ClickHouse/ClickHouse/pull/7737) ([Amos Bird](https://github.com/amosbird)) -* Fix `test_storage_s3` integration test. [#7734](https://github.com/ClickHouse/ClickHouse/pull/7734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Support `StorageFile(, null) ` to insert block into given format file without actually write to disk. This is required for performance tests. [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos Bird](https://github.com/amosbird)) -* Added argument `--print-time` to functional tests which prints execution time per test. [#8001](https://github.com/ClickHouse/ClickHouse/pull/8001) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Added asserts to `KeyCondition` while evaluating RPN. This will fix warning from gcc-9. [#8279](https://github.com/ClickHouse/ClickHouse/pull/8279) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Dump cmake options in CI builds. [#8273](https://github.com/ClickHouse/ClickHouse/pull/8273) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Don't generate debug info for some fat libraries. [#8271](https://github.com/ClickHouse/ClickHouse/pull/8271) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Make `log_to_console.xml` always log to stderr, regardless of is it interactive or not. [#8395](https://github.com/ClickHouse/ClickHouse/pull/8395) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Removed some unused features from `clickhouse-performance-test` tool. [#8555](https://github.com/ClickHouse/ClickHouse/pull/8555) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now we will also search for `lld-X` with corresponding `clang-X` version. [#8092](https://github.com/ClickHouse/ClickHouse/pull/8092) ([alesapin](https://github.com/alesapin)) -* Parquet build improvement. [#8421](https://github.com/ClickHouse/ClickHouse/pull/8421) ([maxulan](https://github.com/maxulan)) -* More GCC warnings [#8221](https://github.com/ClickHouse/ClickHouse/pull/8221) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* Package for Arch Linux now allows to run ClickHouse server, and not only client. [#8534](https://github.com/ClickHouse/ClickHouse/pull/8534) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix test with processors. Tiny performance fixes. [#7672](https://github.com/ClickHouse/ClickHouse/pull/7672) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Update contrib/protobuf. [#8256](https://github.com/ClickHouse/ClickHouse/pull/8256) ([Matwey V. Kornilov](https://github.com/matwey)) -* In preparation of switching to c++20 as a new year celebration. "May the C++ force be with ClickHouse." [#8447](https://github.com/ClickHouse/ClickHouse/pull/8447) ([Amos Bird](https://github.com/amosbird)) - -#### Experimental Feature -* Added experimental setting `min_bytes_to_use_mmap_io`. It allows to read big files without copying data from kernel to userspace. The setting is disabled by default. Recommended threshold is about 64 MB, because mmap/munmap is slow. [#8520](https://github.com/ClickHouse/ClickHouse/pull/8520) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Reworked quotas as a part of access control system. Added new table `system.quotas`, new functions `currentQuota`, `currentQuotaKey`, new SQL syntax `CREATE QUOTA`, `ALTER QUOTA`, `DROP QUOTA`, `SHOW QUOTA`. [#7257](https://github.com/ClickHouse/ClickHouse/pull/7257) ([Vitaly Baranov](https://github.com/vitlibar)) -* Allow skipping unknown settings with warnings instead of throwing exceptions. [#7653](https://github.com/ClickHouse/ClickHouse/pull/7653) ([Vitaly Baranov](https://github.com/vitlibar)) -* Reworked row policies as a part of access control system. Added new table `system.row_policies`, new function `currentRowPolicies()`, new SQL syntax `CREATE POLICY`, `ALTER POLICY`, `DROP POLICY`, `SHOW CREATE POLICY`, `SHOW POLICIES`. [#7808](https://github.com/ClickHouse/ClickHouse/pull/7808) ([Vitaly Baranov](https://github.com/vitlibar)) - -#### Security Fix -* Fixed the possibility of reading directories structure in tables with `File` table engine. This fixes [#8536](https://github.com/ClickHouse/ClickHouse/issues/8536). [#8537](https://github.com/ClickHouse/ClickHouse/pull/8537) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -## [Changelog for 2019](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/changelog/2019.md) diff --git a/docs/en/whats_new/changelog/index.md b/docs/en/whats_new/changelog/index.md new file mode 120000 index 00000000000..2bf6a6cbd6d --- /dev/null +++ b/docs/en/whats_new/changelog/index.md @@ -0,0 +1 @@ +../../../../CHANGELOG.md \ No newline at end of file From 3ee89344af4bcd7958f1c949cab8c8a7a041e01c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 20 Apr 2020 18:37:28 +0300 Subject: [PATCH 469/743] Statically dispatch on whether the argument is a Tuple --- .../AggregateFunctionSumMap.cpp | 100 ++++++++++++------ .../AggregateFunctionSumMap.h | 51 +++++---- 2 files changed, 100 insertions(+), 51 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 3fc24f4c064..0625d8f8af4 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -36,23 +36,29 @@ struct WithoutOverflowPolicy } }; -template -using SumMapWithOverflow = AggregateFunctionSumMap; +template +struct SumMap +{ + template + using F = AggregateFunctionSumMap, + tuple_argument>; +}; -template -using SumMapWithoutOverflow = AggregateFunctionSumMap; +template +struct SumMapFiltered +{ + template + using F = AggregateFunctionSumMapFiltered, + tuple_argument>; +}; -template -using SumMapFilteredWithOverflow = AggregateFunctionSumMapFiltered; -template -using SumMapFilteredWithoutOverflow = AggregateFunctionSumMapFiltered; - -using SumMapArgs = std::pair; - -SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) +auto parseArguments(const std::string & name, const DataTypes & arguments) { DataTypes args; + bool tuple_argument = false; if (arguments.size() == 1) { @@ -66,9 +72,13 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) const auto elems = tuple_type->getElements(); args.insert(args.end(), elems.begin(), elems.end()); + tuple_argument = true; } else + { args.insert(args.end(), arguments.begin(), arguments.end()); + tuple_argument = false; + } if (args.size() < 2) throw Exception("Aggregate function " + name + " requires at least two arguments of Array type or one argument of tuple of two arrays", @@ -92,28 +102,42 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) values_types.push_back(array_type->getNestedType()); } - return {std::move(keys_type), std::move(values_types)}; + return std::tuple{std::move(keys_type), std::move(values_types), + tuple_argument}; } -template