mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
use Atomic for system database
This commit is contained in:
parent
4aff48dc3f
commit
a6ff049eec
@ -468,6 +468,9 @@ int Server::main(const std::vector<std::string> & /*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();
|
||||
}
|
||||
|
@ -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) \
|
||||
|
@ -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)
|
||||
|
@ -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<DatabaseOrdinary>(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, "<no file>", true);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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()));
|
||||
|
@ -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_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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(
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
unsigned num_streams) override;
|
||||
|
||||
protected:
|
||||
StorageSystemColumns(const std::string & name_);
|
||||
StorageSystemColumns(const StorageID & table_id_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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"});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
unsigned num_streams) override;
|
||||
|
||||
protected:
|
||||
StorageSystemDisks(const std::string & name_);
|
||||
StorageSystemDisks(const StorageID & table_id_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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<DataTypeUInt8>()}}));
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
unsigned num_streams) override;
|
||||
|
||||
protected:
|
||||
StorageSystemOne(const std::string & name_);
|
||||
StorageSystemOne(const StorageID & table_id_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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<DataTypeString>()},
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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_));
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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<DataTypeString>()},
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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({
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
unsigned num_streams) override;
|
||||
|
||||
protected:
|
||||
StorageSystemReplicas(const std::string & name_);
|
||||
StorageSystemReplicas(const StorageID & table_id_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -126,8 +126,8 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
StorageSystemStackTrace::StorageSystemStackTrace(const String & name_)
|
||||
: IStorageSystemOneBlock<StorageSystemStackTrace>(name_)
|
||||
StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_)
|
||||
: IStorageSystemOneBlock<StorageSystemStackTrace>(table_id_)
|
||||
{
|
||||
notification_pipe.open();
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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(
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
unsigned num_streams) override;
|
||||
|
||||
protected:
|
||||
StorageSystemStoragePolicies(const std::string & name_);
|
||||
StorageSystemStoragePolicies(const StorageID & table_id_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
unsigned num_streams) override;
|
||||
|
||||
protected:
|
||||
StorageSystemTables(const std::string & name_);
|
||||
StorageSystemTables(const StorageID & table_id_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -66,79 +66,105 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template<typename StorageT>
|
||||
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<StorageSystemOne>(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<StorageSystemDatabases>(system_database, "databases");
|
||||
attach<StorageSystemTables>(system_database, "tables");
|
||||
attach<StorageSystemColumns>(system_database, "columns");
|
||||
attach<StorageSystemFunctions>(system_database, "functions");
|
||||
attach<StorageSystemEvents>(system_database, "events");
|
||||
attach<StorageSystemSettings>(system_database, "settings");
|
||||
attach<SystemMergeTreeSettings>(system_database, "merge_tree_settings");
|
||||
attach<StorageSystemBuildOptions>(system_database, "build_options");
|
||||
attach<StorageSystemFormats>(system_database, "formats");
|
||||
attach<StorageSystemTableFunctions>(system_database, "table_functions");
|
||||
attach<StorageSystemAggregateFunctionCombinators>(system_database, "aggregate_function_combinators");
|
||||
attach<StorageSystemDataTypeFamilies>(system_database, "data_type_families");
|
||||
attach<StorageSystemCollations>(system_database, "collations");
|
||||
attach<StorageSystemTableEngines>(system_database, "table_engines");
|
||||
attach<StorageSystemContributors>(system_database, "contributors");
|
||||
attach<StorageSystemUsers>(system_database, "users");
|
||||
attach<StorageSystemRoles>(system_database, "roles");
|
||||
attach<StorageSystemGrants>(system_database, "grants");
|
||||
attach<StorageSystemRoleGrants>(system_database, "role_grants");
|
||||
attach<StorageSystemCurrentRoles>(system_database, "current_roles");
|
||||
attach<StorageSystemEnabledRoles>(system_database, "enabled_roles");
|
||||
attach<StorageSystemSettingsProfiles>(system_database, "settings_profiles");
|
||||
attach<StorageSystemSettingsProfileElements>(system_database, "settings_profile_elements");
|
||||
attach<StorageSystemRowPolicies>(system_database, "row_policies");
|
||||
attach<StorageSystemQuotas>(system_database, "quotas");
|
||||
attach<StorageSystemQuotaLimits>(system_database, "quota_limits");
|
||||
attach<StorageSystemQuotaUsage>(system_database, "quota_usage");
|
||||
attach<StorageSystemQuotasUsage>(system_database, "quotas_usage");
|
||||
attach<StorageSystemPrivileges>(system_database, "privileges");
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
system_database.attachTable("licenses", StorageSystemLicenses::create("licenses"));
|
||||
attach<StorageSystemLicenses>(system_database, "licenses");
|
||||
#endif
|
||||
#ifdef OS_LINUX
|
||||
system_database.attachTable("stack_trace", StorageSystemStackTrace::create("stack_trace"));
|
||||
attach<StorageSystemStackTrace>(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<StorageSystemParts>(system_database, "parts");
|
||||
//attach<>(system_database, "detached_parts", createDetachedPartsTable());
|
||||
attach<StorageSystemPartsColumns>(system_database, "parts_columns");
|
||||
attach<StorageSystemDisks>(system_database, "disks");
|
||||
attach<StorageSystemStoragePolicies>(system_database, "storage_policies");
|
||||
attach<StorageSystemProcesses>(system_database, "processes");
|
||||
attach<StorageSystemMetrics>(system_database, "metrics");
|
||||
attach<StorageSystemMerges>(system_database, "merges");
|
||||
attach<StorageSystemMutations>(system_database, "mutations");
|
||||
attach<StorageSystemReplicas>(system_database, "replicas");
|
||||
attach<StorageSystemReplicationQueue>(system_database, "replication_queue");
|
||||
attach<StorageSystemDistributionQueue>(system_database, "distribution_queue");
|
||||
attach<StorageSystemDictionaries>(system_database, "dictionaries");
|
||||
attach<StorageSystemModels>(system_database, "models");
|
||||
attach<StorageSystemClusters>(system_database, "clusters");
|
||||
attach<StorageSystemGraphite>(system_database, "graphite_retentions");
|
||||
attach<StorageSystemMacros>(system_database, "macros");
|
||||
|
||||
if (has_zookeeper)
|
||||
system_database.attachTable("zookeeper", StorageSystemZooKeeper::create("zookeeper"));
|
||||
attach<StorageSystemZooKeeper>(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));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,7 +2,6 @@
|
||||
<profiles>
|
||||
<default>
|
||||
<default_database_engine>Atomic</default_database_engine>
|
||||
<allow_experimental_database_atomic>1</allow_experimental_database_atomic>
|
||||
<show_table_uuid_in_table_create_query_if_not_nil>0</show_table_uuid_in_table_create_query_if_not_nil>
|
||||
</default>
|
||||
</profiles>
|
||||
|
@ -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" &
|
||||
|
@ -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;
|
||||
|
@ -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"
|
||||
|
@ -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"
|
||||
|
Loading…
Reference in New Issue
Block a user