diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ddc5ec080fb..fdefd217e61 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -468,6 +468,9 @@ int Server::main(const std::vector & /*args*/) } { + Poco::File(path + "data/").createDirectories(); + Poco::File(path + "metadata/").createDirectories(); + /// Directory with metadata of tables, which was marked as dropped by Atomic database Poco::File(path + "metadata_dropped/").createDirectories(); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 60a3792ea87..54775c79b14 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -347,7 +347,6 @@ class IColumn; M(UInt64, 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(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ - M(Bool, allow_experimental_database_atomic, true, "Allow to create database with Engine=Atomic.", 0) \ M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ M(Bool, 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(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ @@ -395,7 +394,8 @@ class IColumn; M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ \ M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ - M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) + M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ + M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) #define FORMAT_FACTORY_SETTINGS(M) \ M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3f8ff0d41f2..e0e19ae317d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -142,10 +142,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) if (create.storage->engine->name == "Atomic") { - if (!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); - if (create.attach && create.uuid == UUIDHelpers::Nil) throw Exception("UUID must be specified for ATTACH", ErrorCodes::INCORRECT_QUERY); else if (create.uuid == UUIDHelpers::Nil) diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index d7c2dfa5287..e273d6a7be1 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -84,9 +84,6 @@ static void loadDatabase( } -#define SYSTEM_DATABASE "system" - - void loadMetadata(Context & context, const String & default_database_name) { Poco::Logger * log = &Poco::Logger::get("loadMetadata"); @@ -114,7 +111,7 @@ void loadMetadata(Context & context, const String & default_database_name) if (endsWith(it.name(), ".sql")) { String db_name = it.name().substr(0, it.name().size() - 4); - if (db_name != SYSTEM_DATABASE) + if (db_name != DatabaseCatalog::SYSTEM_DATABASE) databases.emplace(unescapeForFileName(db_name), path + "/" + db_name); } @@ -140,7 +137,7 @@ void loadMetadata(Context & context, const String & default_database_name) if (it.name().at(0) == '.') continue; - if (it.name() == SYSTEM_DATABASE) + if (it.name() == DatabaseCatalog::SYSTEM_DATABASE) continue; databases.emplace(unescapeForFileName(it.name()), it.path().toString()); @@ -172,21 +169,19 @@ void loadMetadata(Context & context, const String & default_database_name) void loadMetadataSystem(Context & context) { - String path = context.getPath() + "metadata/" SYSTEM_DATABASE; + String path = context.getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE; if (Poco::File(path).exists()) { /// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted. - loadDatabase(context, SYSTEM_DATABASE, path, true); + loadDatabase(context, DatabaseCatalog::SYSTEM_DATABASE, path, true); } else { /// Initialize system database manually - String global_path = context.getPath(); - Poco::File(global_path + "data/" SYSTEM_DATABASE).createDirectories(); - Poco::File(global_path + "metadata/" SYSTEM_DATABASE).createDirectories(); - - auto system_database = std::make_shared(SYSTEM_DATABASE, global_path + "metadata/" SYSTEM_DATABASE "/", context); - DatabaseCatalog::instance().attachDatabase(SYSTEM_DATABASE, system_database); + String database_create_query = "CREATE DATABASE "; + database_create_query += DatabaseCatalog::SYSTEM_DATABASE; + database_create_query += " ENGINE=Atomic"; + executeCreateQuery(database_create_query, context, DatabaseCatalog::SYSTEM_DATABASE, "", true); } } diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 7c2ef85f158..c2abb5302fa 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -21,7 +21,7 @@ protected: virtual void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const = 0; public: - IStorageSystemOneBlock(const String & name_) : IStorage({"system", name_}) + IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata metadata_; metadata_.setColumns(ColumnsDescription(Self::getNamesAndTypes())); diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp index 059ef708a81..8dabac4fb49 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp @@ -16,8 +16,8 @@ NamesAndTypesList StorageSystemAsynchronousMetrics::getNamesAndTypes() } -StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_) - : IStorageSystemOneBlock(name_), async_metrics(async_metrics_) +StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_) + : IStorageSystemOneBlock(table_id_), async_metrics(async_metrics_) { } diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.h b/src/Storages/System/StorageSystemAsynchronousMetrics.h index f31450fe086..8f79c09b098 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -25,7 +25,7 @@ private: const AsynchronousMetrics & async_metrics; protected: - StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_); + StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_); void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 57d8d13e79b..3c6bb493830 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -23,8 +23,8 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; } -StorageSystemColumns::StorageSystemColumns(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemColumns::StorageSystemColumns(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index 7336b406183..89b33214352 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -27,7 +27,7 @@ public: unsigned num_streams) override; protected: - StorageSystemColumns(const std::string & name_); + StorageSystemColumns(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index c325df1251d..b0953ac75e1 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -27,8 +27,8 @@ public: std::string getName() const override { return "SystemDetachedParts"; } protected: - explicit StorageSystemDetachedParts() - : IStorage({"system", "detached_parts"}) + explicit StorageSystemDetachedParts(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription{{ @@ -90,7 +90,7 @@ protected: StoragePtr createDetachedPartsTable() { - return StorageSystemDetachedParts::create(); + return StorageSystemDetachedParts::create(StorageID{"system", "detached_parts"}); } } diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 554b8cfd1eb..d4ace6378d9 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -11,8 +11,8 @@ namespace ErrorCodes } -StorageSystemDisks::StorageSystemDisks(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index 714a0a5428c..b5968556351 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -30,7 +30,7 @@ public: unsigned num_streams) override; protected: - StorageSystemDisks(const std::string & name_); + StorageSystemDisks(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 3329cbb035e..b29051a796d 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -11,8 +11,8 @@ namespace DB { -StorageSystemOne::StorageSystemOne(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemOne::StorageSystemOne(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription({{"dummy", std::make_shared()}})); diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 3b3e531dc86..044fb39a1ee 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -31,7 +31,7 @@ public: unsigned num_streams) override; protected: - StorageSystemOne(const std::string & name_); + StorageSystemOne(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 7c0da07ed88..a3fb2cf10e6 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -15,8 +15,8 @@ namespace DB { -StorageSystemParts::StorageSystemParts(const std::string & name_) - : StorageSystemPartsBase(name_, +StorageSystemParts::StorageSystemParts(const StorageID & table_id_) + : StorageSystemPartsBase(table_id_, { {"partition", std::make_shared()}, {"name", std::make_shared()}, diff --git a/src/Storages/System/StorageSystemParts.h b/src/Storages/System/StorageSystemParts.h index 06f4f414c88..2de4cd3f3a4 100644 --- a/src/Storages/System/StorageSystemParts.h +++ b/src/Storages/System/StorageSystemParts.h @@ -19,7 +19,7 @@ public: std::string getName() const override { return "SystemParts"; } protected: - explicit StorageSystemParts(const std::string & name_); + explicit StorageSystemParts(const StorageID & table_id_); void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) override; }; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 168b305605d..4b98089f95e 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -261,8 +261,8 @@ Pipes StorageSystemPartsBase::read( } -StorageSystemPartsBase::StorageSystemPartsBase(std::string name_, NamesAndTypesList && columns_) - : IStorage(StorageID{"system", name_}) +StorageSystemPartsBase::StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_) + : IStorage(table_id_) { ColumnsDescription tmp_columns(std::move(columns_)); diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 56c9a8fb0d0..ff338930e0c 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -72,7 +72,7 @@ private: protected: const FormatSettings format_settings; - StorageSystemPartsBase(std::string name_, NamesAndTypesList && columns_); + StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_); virtual void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) = 0; }; diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 4631bb9c4c5..0851e389e4f 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -15,8 +15,8 @@ namespace DB { -StorageSystemPartsColumns::StorageSystemPartsColumns(const std::string & name_) - : StorageSystemPartsBase(name_, +StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_) + : StorageSystemPartsBase(table_id_, { {"partition", std::make_shared()}, {"name", std::make_shared()}, diff --git a/src/Storages/System/StorageSystemPartsColumns.h b/src/Storages/System/StorageSystemPartsColumns.h index 596fd7dae45..6347a418875 100644 --- a/src/Storages/System/StorageSystemPartsColumns.h +++ b/src/Storages/System/StorageSystemPartsColumns.h @@ -21,7 +21,7 @@ public: std::string getName() const override { return "SystemPartsColumns"; } protected: - StorageSystemPartsColumns(const std::string & name_); + StorageSystemPartsColumns(const StorageID & table_id_); void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) override; }; diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 27a9cd0c4bb..211859b27ff 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -16,8 +16,8 @@ namespace DB { -StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription({ diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index b068ebc8b0a..198a7afa0fe 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -28,7 +28,7 @@ public: unsigned num_streams) override; protected: - StorageSystemReplicas(const std::string & name_); + StorageSystemReplicas(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index bd9915237d4..9edcb1ede47 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -126,8 +126,8 @@ namespace } -StorageSystemStackTrace::StorageSystemStackTrace(const String & name_) - : IStorageSystemOneBlock(name_) +StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_) + : IStorageSystemOneBlock(table_id_) { notification_pipe.open(); diff --git a/src/Storages/System/StorageSystemStackTrace.h b/src/Storages/System/StorageSystemStackTrace.h index 3f20b1e973c..a389f02eb09 100644 --- a/src/Storages/System/StorageSystemStackTrace.h +++ b/src/Storages/System/StorageSystemStackTrace.h @@ -23,7 +23,7 @@ public: String getName() const override { return "SystemStackTrace"; } static NamesAndTypesList getNamesAndTypes(); - StorageSystemStackTrace(const String & name_); + StorageSystemStackTrace(const StorageID & table_id_); protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index 76a665d871c..788ee05ec95 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -17,8 +17,8 @@ namespace ErrorCodes } -StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemStoragePolicies::StorageSystemStoragePolicies(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns( diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index a1427da8559..a22563c9543 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -30,7 +30,7 @@ public: unsigned num_streams) override; protected: - StorageSystemStoragePolicies(const std::string & name_); + StorageSystemStoragePolicies(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index b8a65183be1..2974e3f493d 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -30,8 +30,8 @@ namespace ErrorCodes } -StorageSystemTables::StorageSystemTables(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemTables::StorageSystemTables(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 54551205684..354d87adf1c 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -28,7 +28,7 @@ public: unsigned num_streams) override; protected: - StorageSystemTables(const std::string & name_); + StorageSystemTables(const StorageID & table_id_); }; } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 2b52f0fe5cc..89c8de7f430 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -66,79 +66,105 @@ namespace DB { +namespace +{ + +template +void attach(IDatabase & system_database, const String & table_name) +{ + if (system_database.getUUID() == UUIDHelpers::Nil) + { + auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); + system_database.attachTable(table_name, StorageT::create(table_id)); + } + else + { + auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); + String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); + system_database.attachTable(table_name, StorageT::create(table_id), path); + } +} + +} + + ///TODO allow store system tables in DatabaseAtomic void attachSystemTablesLocal(IDatabase & system_database) { - system_database.attachTable("one", StorageSystemOne::create("one")); - system_database.attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false)); - system_database.attachTable("numbers_mt", StorageSystemNumbers::create(StorageID("system", "numbers_mt"), true)); - system_database.attachTable("zeros", StorageSystemZeros::create(StorageID("system", "zeros"), false)); - system_database.attachTable("zeros_mt", StorageSystemZeros::create(StorageID("system", "zeros_mt"), true)); - system_database.attachTable("databases", StorageSystemDatabases::create("databases")); - system_database.attachTable("tables", StorageSystemTables::create("tables")); - system_database.attachTable("columns", StorageSystemColumns::create("columns")); - system_database.attachTable("functions", StorageSystemFunctions::create("functions")); - system_database.attachTable("events", StorageSystemEvents::create("events")); - system_database.attachTable("settings", StorageSystemSettings::create("settings")); - system_database.attachTable("merge_tree_settings", SystemMergeTreeSettings::create("merge_tree_settings")); - system_database.attachTable("build_options", StorageSystemBuildOptions::create("build_options")); - system_database.attachTable("formats", StorageSystemFormats::create("formats")); - system_database.attachTable("table_functions", StorageSystemTableFunctions::create("table_functions")); - system_database.attachTable("aggregate_function_combinators", StorageSystemAggregateFunctionCombinators::create("aggregate_function_combinators")); - system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families")); - system_database.attachTable("collations", StorageSystemCollations::create("collations")); - system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines")); - system_database.attachTable("contributors", StorageSystemContributors::create("contributors")); - system_database.attachTable("users", StorageSystemUsers::create("users")); - system_database.attachTable("roles", StorageSystemRoles::create("roles")); - system_database.attachTable("grants", StorageSystemGrants::create("grants")); - system_database.attachTable("role_grants", StorageSystemRoleGrants::create("role_grants")); - system_database.attachTable("current_roles", StorageSystemCurrentRoles::create("current_roles")); - system_database.attachTable("enabled_roles", StorageSystemEnabledRoles::create("enabled_roles")); - system_database.attachTable("settings_profiles", StorageSystemSettingsProfiles::create("settings_profiles")); - system_database.attachTable("settings_profile_elements", StorageSystemSettingsProfileElements::create("settings_profile_elements")); - system_database.attachTable("row_policies", StorageSystemRowPolicies::create("row_policies")); - system_database.attachTable("quotas", StorageSystemQuotas::create("quotas")); - system_database.attachTable("quota_limits", StorageSystemQuotaLimits::create("quota_limits")); - system_database.attachTable("quota_usage", StorageSystemQuotaUsage::create("quota_usage")); - system_database.attachTable("quotas_usage", StorageSystemQuotasUsage::create("all_quotas_usage")); - system_database.attachTable("privileges", StorageSystemPrivileges::create("privileges")); + attach(system_database, "one"); + + //system_database.attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false)); + //system_database.attachTable("numbers_mt", StorageSystemNumbers::create(StorageID("system", "numbers_mt"), true)); + //system_database.attachTable("zeros", StorageSystemZeros::create(StorageID("system", "zeros"), false)); + //system_database.attachTable("zeros_mt", StorageSystemZeros::create(StorageID("system", "zeros_mt"), true)); + + attach(system_database, "databases"); + attach(system_database, "tables"); + attach(system_database, "columns"); + attach(system_database, "functions"); + attach(system_database, "events"); + attach(system_database, "settings"); + attach(system_database, "merge_tree_settings"); + attach(system_database, "build_options"); + attach(system_database, "formats"); + attach(system_database, "table_functions"); + attach(system_database, "aggregate_function_combinators"); + attach(system_database, "data_type_families"); + attach(system_database, "collations"); + attach(system_database, "table_engines"); + attach(system_database, "contributors"); + attach(system_database, "users"); + attach(system_database, "roles"); + attach(system_database, "grants"); + attach(system_database, "role_grants"); + attach(system_database, "current_roles"); + attach(system_database, "enabled_roles"); + attach(system_database, "settings_profiles"); + attach(system_database, "settings_profile_elements"); + attach(system_database, "row_policies"); + attach(system_database, "quotas"); + attach(system_database, "quota_limits"); + attach(system_database, "quota_usage"); + attach(system_database, "quotas_usage"); + attach(system_database, "privileges"); + #if !defined(ARCADIA_BUILD) - system_database.attachTable("licenses", StorageSystemLicenses::create("licenses")); + attach(system_database, "licenses"); #endif #ifdef OS_LINUX - system_database.attachTable("stack_trace", StorageSystemStackTrace::create("stack_trace")); + attach(system_database, "stack_trace"); #endif } void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) { attachSystemTablesLocal(system_database); - system_database.attachTable("parts", StorageSystemParts::create("parts")); - system_database.attachTable("detached_parts", createDetachedPartsTable()); - system_database.attachTable("parts_columns", StorageSystemPartsColumns::create("parts_columns")); - system_database.attachTable("disks", StorageSystemDisks::create("disks")); - system_database.attachTable("storage_policies", StorageSystemStoragePolicies::create("storage_policies")); - system_database.attachTable("processes", StorageSystemProcesses::create("processes")); - system_database.attachTable("metrics", StorageSystemMetrics::create("metrics")); - system_database.attachTable("merges", StorageSystemMerges::create("merges")); - system_database.attachTable("mutations", StorageSystemMutations::create("mutations")); - system_database.attachTable("replicas", StorageSystemReplicas::create("replicas")); - system_database.attachTable("replication_queue", StorageSystemReplicationQueue::create("replication_queue")); - system_database.attachTable("distribution_queue", StorageSystemDistributionQueue::create("distribution_queue")); - system_database.attachTable("dictionaries", StorageSystemDictionaries::create("dictionaries")); - system_database.attachTable("models", StorageSystemModels::create("models")); - system_database.attachTable("clusters", StorageSystemClusters::create("clusters")); - system_database.attachTable("graphite_retentions", StorageSystemGraphite::create("graphite_retentions")); - system_database.attachTable("macros", StorageSystemMacros::create("macros")); + + attach(system_database, "parts"); + //attach<>(system_database, "detached_parts", createDetachedPartsTable()); + attach(system_database, "parts_columns"); + attach(system_database, "disks"); + attach(system_database, "storage_policies"); + attach(system_database, "processes"); + attach(system_database, "metrics"); + attach(system_database, "merges"); + attach(system_database, "mutations"); + attach(system_database, "replicas"); + attach(system_database, "replication_queue"); + attach(system_database, "distribution_queue"); + attach(system_database, "dictionaries"); + attach(system_database, "models"); + attach(system_database, "clusters"); + attach(system_database, "graphite_retentions"); + attach(system_database, "macros"); if (has_zookeeper) - system_database.attachTable("zookeeper", StorageSystemZooKeeper::create("zookeeper")); + attach(system_database, "zookeeper"); } -void attachSystemTablesAsync(IDatabase & system_database, AsynchronousMetrics & async_metrics) +void attachSystemTablesAsync(IDatabase & /*system_database*/, AsynchronousMetrics & /*async_metrics*/) { - system_database.attachTable("asynchronous_metrics", StorageSystemAsynchronousMetrics::create("asynchronous_metrics", async_metrics)); + //system_database.attachTable("asynchronous_metrics", StorageSystemAsynchronousMetrics::create("asynchronous_metrics", async_metrics)); } } diff --git a/tests/config/database_atomic_usersd.xml b/tests/config/database_atomic_usersd.xml index 9e749b231bd..201d476da24 100644 --- a/tests/config/database_atomic_usersd.xml +++ b/tests/config/database_atomic_usersd.xml @@ -2,7 +2,6 @@ Atomic - 1 0 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 18d74a1817c..9f4bfb2c436 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 --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01107 ENGINE=Atomic" +$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(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 --allow_experimental_database_atomic=1 -q "ATTACH DATABASE test_01107" +$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & diff --git a/tests/queries/0_stateless/01109_exchange_tables.sql b/tests/queries/0_stateless/01109_exchange_tables.sql index 7125bfea851..7fbb36e8ce9 100644 --- a/tests/queries/0_stateless/01109_exchange_tables.sql +++ b/tests/queries/0_stateless/01109_exchange_tables.sql @@ -1,5 +1,4 @@ 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.sh b/tests/queries/0_stateless/01114_database_atomic.sh index c7862bf5314..d23be44e784 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -8,8 +8,8 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_3" -$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 -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01114_2" $CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01114_3" $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_1" diff --git a/tests/queries/0_stateless/01192_rename_database.sh b/tests/queries/0_stateless/01192_rename_database.sh index 3d4223d09d5..c97ec14335c 100755 --- a/tests/queries/0_stateless/01192_rename_database.sh +++ b/tests/queries/0_stateless/01192_rename_database.sh @@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01192_renamed" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01192_atomic" $CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" 2>&1| grep -F "does not support" > /dev/null && echo "ok" -$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 --default_database_engine=Atomic -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" +$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" # 2. check metadata $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE DATABASE test_01192" @@ -40,7 +40,7 @@ $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW test_01192.mv TO test_01192.rmt $CLICKHOUSE_CLIENT -q "INSERT INTO test_01192.mt SELECT number FROM numbers(10)" && echo "inserted" -$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 --default_database_engine=Atomic -q "CREATE DATABASE test_01192_atomic" +$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01192_atomic" $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01192_renamed" # it's blocking $CLICKHOUSE_CLIENT -q "RENAME TABLE test_01192.mt TO test_01192_atomic.mt, test_01192.rmt TO test_01192_atomic.rmt, test_01192.mv TO test_01192_atomic.mv" && echo "renamed"