Revert "Replace --no-system-tables with loading virtual tables of system database lazily"

This commit is contained in:
Azat Khuzhin 2023-12-14 09:07:47 +01:00 committed by GitHub
parent e4901323fd
commit 0e86e16a01
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 107 additions and 220 deletions

View File

@ -216,6 +216,7 @@ Arguments:
- `--logger.level` — Log level.
- `--ignore-error` — do not stop processing if a query failed.
- `-c`, `--config-file` — path to configuration file in same format as for ClickHouse server, by default the configuration empty.
- `--no-system-tables` — do not attach system tables.
- `--help` — arguments references for `clickhouse-local`.
- `-V`, `--version` — print version information and exit.

View File

@ -45,6 +45,7 @@ $ clickhouse-local --structure "table_structure" --input-format "format_of_incom
- `--logger.level` — уровень логирования.
- `--ignore-error` — не прекращать обработку если запрос выдал ошибку.
- `-c`, `--config-file` — путь к файлу конфигурации. По умолчанию `clickhouse-local` запускается с пустой конфигурацией. Конфигурационный файл имеет тот же формат, что и для сервера ClickHouse, и в нём можно использовать все конфигурационные параметры сервера. Обычно подключение конфигурации не требуется; если требуется установить отдельный параметр, то это можно сделать ключом с именем параметра.
- `--no-system-tables` — запуск без использования системных таблиц.
- `--help` — вывод справочной информации о `clickhouse-local`.
- `-V`, `--version` — вывод текущей версии и выход.

View File

@ -45,6 +45,7 @@ clickhouse-local --structure "table_structure" --input-format "format_of_incomin
- `--logger.level` — 日志级别。
- `--ignore-error` — 当查询失败时,不停止处理。
- `-c`, `--config-file` — 与ClickHouse服务器格式相同配置文件的路径默认情况下配置为空。
- `--no-system-tables` — 不附加系统表。
- `--help``clickhouse-local`使用帮助信息。
- `-V`, `--version` — 打印版本信息并退出。

View File

@ -744,7 +744,7 @@ void LocalServer::processConfig()
LOG_DEBUG(log, "Loading metadata from {}", path);
auto startup_system_tasks = loadMetadataSystem(global_context);
attachSystemTablesLocal</* lazy= */ true>(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
waitLoad(TablesLoaderForegroundPoolId, startup_system_tasks);
@ -761,9 +761,9 @@ void LocalServer::processConfig()
LOG_DEBUG(log, "Loaded metadata.");
}
else
else if (!config().has("no-system-tables"))
{
attachSystemTablesLocal</* lazy= */ true>(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
}
@ -842,6 +842,7 @@ void LocalServer::addOptions(OptionsDescription & options_description)
("logger.log", po::value<std::string>(), "Log file name")
("logger.level", po::value<std::string>(), "Log level")
("no-system-tables", "do not attach system tables (better startup time)")
("path", po::value<std::string>(), "Storage path")
("only-system-tables", "attach only system tables from specified path")
("top_level_domains_path", po::value<std::string>(), "Path to lists with custom TLDs")
@ -870,6 +871,8 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp
config().setString("table-file", options["file"].as<std::string>());
if (options.count("structure"))
config().setString("table-structure", options["structure"].as<std::string>());
if (options.count("no-system-tables"))
config().setBool("no-system-tables", true);
if (options.count("only-system-tables"))
config().setBool("only-system-tables", true);
if (options.count("database"))

View File

@ -89,14 +89,15 @@ void DatabaseAtomic::drop(ContextPtr)
fs::remove_all(getMetadataPath());
}
void DatabaseAtomic::attachTableUnlocked(ContextPtr local_context, const String & name, const StoragePtr & table, const String & relative_table_path)
void DatabaseAtomic::attachTable(ContextPtr /* context_ */, 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 = cleanupDetachedTables();
auto table_id = table->getStorageID();
assertDetachedTableNotInUse(table_id.uuid);
DatabaseOrdinary::attachTableUnlocked(local_context, name, table, relative_table_path);
DatabaseOrdinary::attachTableUnlocked(name, table);
table_name_to_path.emplace(std::make_pair(name, relative_table_path));
}
@ -324,7 +325,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
/// It throws if `table_metadata_path` already exists (it's possible if table was detached)
renameNoReplace(table_metadata_tmp_path, table_metadata_path); /// Commit point (a sort of)
DatabaseWithOwnTablesBase::attachTableUnlocked(query_context, query.getTable(), table, /*relative_table_path=*/ {}); /// Should never throw
attachTableUnlocked(query.getTable(), table); /// Should never throw
table_name_to_path.emplace(query.getTable(), table_data_path);
}
catch (...)

View File

@ -38,6 +38,7 @@ public:
void dropTable(ContextPtr context, const String & table_name, bool sync) override;
void dropTableImpl(ContextPtr context, const String & table_name, bool sync);
void attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) override;
StoragePtr detachTable(ContextPtr context, const String & name) override;
String getTableDataPath(const String & table_name) const override;
@ -65,8 +66,6 @@ public:
void setDetachedTableNotInUseForce(const UUID & uuid) override;
protected:
void attachTableUnlocked(ContextPtr local_context, const String & name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override;
void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, ContextPtr query_context) override;
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context) override;

View File

@ -168,9 +168,10 @@ bool DatabaseLazy::empty() const
return tables_cache.empty();
}
void DatabaseLazy::attachTableUnlocked(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &)
void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &)
{
LOG_DEBUG(log, "Attach table {}.", backQuote(table_name));
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,

View File

@ -64,15 +64,14 @@ public:
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override;
void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override;
StoragePtr detachTable(ContextPtr context, const String & table_name) override;
void shutdown() override;
~DatabaseLazy() override;
protected:
void attachTableUnlocked(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override;
private:
struct CacheExpirationQueueElement
{

View File

@ -33,13 +33,13 @@ DatabaseMemory::DatabaseMemory(const String & name_, ContextPtr context_)
}
void DatabaseMemory::createTable(
ContextPtr local_context,
ContextPtr /*context*/,
const String & table_name,
const StoragePtr & table,
const ASTPtr & query)
{
std::lock_guard lock{mutex};
attachTableUnlocked(local_context, table_name, table, /*relative_table_path=*/ {});
attachTableUnlocked(table_name, table);
/// Clean the query from temporary flags.
ASTPtr query_to_store = query;
@ -56,7 +56,7 @@ void DatabaseMemory::createTable(
}
void DatabaseMemory::dropTable(
ContextPtr local_context,
ContextPtr /*context*/,
const String & table_name,
bool /*sync*/)
{
@ -83,7 +83,7 @@ void DatabaseMemory::dropTable(
catch (...)
{
std::lock_guard lock{mutex};
attachTableUnlocked(local_context, table_name, table, /*relative_table_path=*/ {});
attachTableUnlocked(table_name, table);
throw;
}

View File

@ -7,7 +7,6 @@
#include <Parsers/formatAST.h>
#include <Storages/StorageDictionary.h>
#include <Storages/StorageFactory.h>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <Common/CurrentMetrics.h>
#include <Common/escapeForFileName.h>
@ -200,7 +199,7 @@ DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const
bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, ContextPtr) const
{
std::lock_guard lock(mutex);
return tables.find(table_name) != tables.end() || lazy_tables.find(table_name) != lazy_tables.end();
return tables.find(table_name) != tables.end();
}
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, ContextPtr) const
@ -212,9 +211,6 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, Con
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const
{
std::lock_guard lock(mutex);
loadLazyTables();
if (!filter_by_table_name)
return std::make_unique<DatabaseTablesSnapshotIterator>(tables, database_name);
@ -261,7 +257,13 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
return res;
}
void DatabaseWithOwnTablesBase::attachTableUnlocked(ContextPtr, const String & name, const StoragePtr & table, const String &)
void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &)
{
std::lock_guard lock(mutex);
attachTableUnlocked(table_name, table);
}
void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table)
{
auto table_id = table->getStorageID();
if (table_id.database_name != database_name)
@ -274,7 +276,7 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(ContextPtr, const String & n
DatabaseCatalog::instance().addUUIDMapping(table_id.uuid, shared_from_this(), table);
}
if (!tables.emplace(name, table).second)
if (!tables.emplace(table_name, table).second)
{
if (table_id.hasUUID())
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
@ -287,12 +289,6 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(ContextPtr, const String & n
CurrentMetrics::add(CurrentMetrics::AttachedTable, 1);
}
void DatabaseWithOwnTablesBase::registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator, const String & relative_table_path)
{
if (!lazy_tables.emplace(table_name, std::make_pair(relative_table_path, std::move(table_creator))).second)
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already registered.", table_name);
}
void DatabaseWithOwnTablesBase::shutdown()
{
/// You can not hold a lock during shutdown.
@ -393,45 +389,10 @@ void DatabaseWithOwnTablesBase::createTableRestoredFromBackup(const ASTPtr & cre
StoragePtr DatabaseWithOwnTablesBase::tryGetTableNoWait(const String & table_name) const
{
std::lock_guard lock(mutex);
auto it = tables.find(table_name);
if (it != tables.end())
return it->second;
const auto lazy_it = lazy_tables.find(table_name);
if (lazy_it != lazy_tables.end())
{
LOG_DEBUG(log, "Attaching lazy table {}", backQuoteIfNeed(table_name));
auto relative_table_path = lazy_it->second.first;
auto storage = lazy_it->second.second();
lazy_tables.erase(lazy_it);
(const_cast<DatabaseWithOwnTablesBase *>(this))->attachTableUnlocked(Context::getGlobalContextInstance(), table_name, storage, relative_table_path);
it = tables.find(table_name);
if (it != tables.end())
return it->second;
}
return {};
}
void DatabaseWithOwnTablesBase::loadLazyTables() const
{
if (lazy_tables.empty())
return;
ContextPtr global_context = Context::getGlobalContextInstance();
while (!lazy_tables.empty())
{
auto lazy_it = lazy_tables.begin();
const auto table_name = lazy_it->first;
LOG_DEBUG(log, "Attaching lazy table {}", backQuoteIfNeed(table_name));
auto relative_table_path = lazy_it->second.first;
auto storage = lazy_it->second.second();
lazy_tables.erase(lazy_it);
(const_cast<DatabaseWithOwnTablesBase *>(this))->attachTableUnlocked(global_context, table_name, storage, relative_table_path);
}
}
}

View File

@ -30,6 +30,8 @@ public:
bool empty() const override;
void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override;
StoragePtr detachTable(ContextPtr context, const String & table_name) override;
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override;
@ -43,19 +45,14 @@ public:
protected:
Tables tables TSA_GUARDED_BY(mutex);
/// Tables that are attached lazily
mutable LazyTables lazy_tables TSA_GUARDED_BY(mutex);
Poco::Logger * log;
DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context);
void attachTableUnlocked(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override;
void registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator, const String & relative_table_path) TSA_REQUIRES(mutex) override;
void attachTableUnlocked(const String & table_name, const StoragePtr & table) TSA_REQUIRES(mutex);
StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex);
StoragePtr getTableUnlocked(const String & table_name) const TSA_REQUIRES(mutex);
StoragePtr tryGetTableNoWait(const String & table_name) const;
void loadLazyTables() const TSA_REQUIRES(mutex);
};
}

View File

@ -62,20 +62,4 @@ void IDatabase::createTableRestoredFromBackup(const ASTPtr & create_table_query,
backQuoteIfNeed(create_table_query->as<const ASTCreateQuery &>().getTable()));
}
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
///
/// Note: ATTACH TABLE statement actually uses createTable method.
void IDatabase::attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) /// NOLINT
{
std::lock_guard lock(mutex);
attachTableUnlocked(context, name, table, relative_table_path);
}
void IDatabase::registerLazyTable(ContextPtr, const String & table_name, LazyTableCreator table_creator, const String & relative_table_path) /// NOLINT
{
std::lock_guard lock(mutex);
registerLazyTableUnlocked(table_name, std::move(table_creator), relative_table_path);
}
}

View File

@ -125,6 +125,7 @@ public:
using DatabaseTablesIteratorPtr = std::unique_ptr<IDatabaseTablesIterator>;
/** Database engine.
* It is responsible for:
* - initialization of set of known tables and dictionaries;
@ -137,10 +138,6 @@ using DatabaseTablesIteratorPtr = std::unique_ptr<IDatabaseTablesIterator>;
class IDatabase : public std::enable_shared_from_this<IDatabase>
{
public:
using LazyTableCreator = std::function<StoragePtr()>;
/// Map{table_name, Pair{relative_table_path, LazyTableCreator}}
using LazyTables = std::map<String, std::pair<String, LazyTableCreator>>;
IDatabase() = delete;
explicit IDatabase(String database_name_);
@ -272,17 +269,11 @@ public:
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
///
/// @param relative_table_path - only for Atomic engine
///
/// Note:
/// - ATTACH TABLE statement actually uses createTable method.
/// - Instead of overriding this method you should override attachTableUnlocked()
/// (This method is only for DatabasesOverlay to override)
virtual void attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path = {}); /// NOLINT
/// Register tables lazily (attach will be done only when the table will be used) instead of attaching it.
/// This is needed to improve startup time of clickhouse-local.
virtual void registerLazyTable(ContextPtr context, const String & table_name, LazyTableCreator table_creator, const String & relative_table_path = {});
/// Note: ATTACH TABLE statement actually uses createTable method.
virtual void attachTable(ContextPtr /* context */, const String & /*name*/, const StoragePtr & /*table*/, [[maybe_unused]] const String & relative_table_path = {}) /// NOLINT
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no ATTACH TABLE query for Database{}", getEngineName());
}
/// Forget about the table without deleting it, and return it. The database may not support this method.
virtual StoragePtr detachTable(ContextPtr /* context */, const String & /*name*/)
@ -439,16 +430,6 @@ protected:
return nullptr;
}
virtual void attachTableUnlocked(ContextPtr /*context*/, const String & /*name*/, const StoragePtr & /*table*/, const String & /*relative_table_path*/ = {}) TSA_REQUIRES(mutex) /// NOLINT
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no ATTACH TABLE query for Database{}", getEngineName());
}
virtual void registerLazyTableUnlocked(const String & /* table_name */, LazyTableCreator /* table_creator */, const String & /* relative_table_path */) TSA_REQUIRES(mutex) /// NOLINT
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There lazy table initialization support for Database{}", getEngineName());
}
mutable std::mutex mutex;
String database_name TSA_GUARDED_BY(mutex);
String comment TSA_GUARDED_BY(mutex);

View File

@ -101,10 +101,10 @@ void DatabaseMaterializedMySQL::dropTable(ContextPtr context_, const String & na
DatabaseAtomic::dropTable(context_, name, sync);
}
void DatabaseMaterializedMySQL::attachTableUnlocked(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path)
void DatabaseMaterializedMySQL::attachTable(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path)
{
checkIsInternalQuery(context_, "ATTACH TABLE");
DatabaseAtomic::attachTableUnlocked(context_, name, table, relative_table_path);
DatabaseAtomic::attachTable(context_, name, table, relative_table_path);
}
StoragePtr DatabaseMaterializedMySQL::detachTable(ContextPtr context_, const String & name)

View File

@ -48,8 +48,6 @@ protected:
LoadTaskPtr startup_mysql_database_task;
void attachTableUnlocked(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override;
public:
String getEngineName() const override { return "MaterializedMySQL"; }
@ -60,6 +58,8 @@ public:
void dropTable(ContextPtr context_, const String & name, bool sync) override;
void attachTable(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) override;
StoragePtr detachTable(ContextPtr context_, const String & name) override;
void renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override;

View File

@ -361,8 +361,10 @@ void DatabaseMySQL::cleanOutdatedTables()
}
}
void DatabaseMySQL::attachTableUnlocked(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &)
void DatabaseMySQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &)
{
std::lock_guard lock{mutex};
if (!local_tables_cache.contains(table_name))
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot attach table {}.{} because it does not exist.",
backQuoteIfNeed(database_name), backQuoteIfNeed(table_name));

View File

@ -84,9 +84,9 @@ public:
void dropTable(ContextPtr context, const String & table_name, bool sync) override;
protected:
void attachTableUnlocked(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) TSA_REQUIRES(mutex) override;
void attachTable(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) override;
protected:
ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override;
private:

View File

@ -216,8 +216,10 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr
}
void DatabasePostgreSQL::attachTableUnlocked(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &)
void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &)
{
std::lock_guard lock{mutex};
if (!checkPostgresTable(table_name))
throw Exception(ErrorCodes::UNKNOWN_TABLE,
"Cannot attach PostgreSQL table {} because it does not exist in PostgreSQL (database: {})",

View File

@ -54,14 +54,13 @@ public:
void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override;
void dropTable(ContextPtr, const String & table_name, bool sync) override;
void attachTable(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) override;
StoragePtr detachTable(ContextPtr context, const String & table_name) override;
void drop(ContextPtr /*context*/) override;
void shutdown() override;
protected:
void attachTableUnlocked(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) TSA_REQUIRES(mutex) override;
ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override;
private:

View File

@ -108,75 +108,72 @@
namespace DB
{
template <bool lazy>
void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database)
{
attachLazyOrNot<lazy, StorageSystemOne>(context, system_database, "one");
attachLazyOrNot<lazy, StorageSystemNumbers>(context, system_database, "numbers", false);
attachLazyOrNot<lazy, StorageSystemNumbers>(context, system_database, "numbers_mt", true);
attachLazyOrNot<lazy, StorageSystemZeros>(context, system_database, "zeros", false);
attachLazyOrNot<lazy, StorageSystemZeros>(context, system_database, "zeros_mt", true);
attachLazyOrNot<lazy, StorageSystemDatabases>(context, system_database, "databases");
attachLazyOrNot<lazy, StorageSystemTables>(context, system_database, "tables");
attachLazyOrNot<lazy, StorageSystemColumns>(context, system_database, "columns");
attachLazyOrNot<lazy, StorageSystemFunctions>(context, system_database, "functions");
attachLazyOrNot<lazy, StorageSystemEvents>(context, system_database, "events");
attachLazyOrNot<lazy, StorageSystemSettings>(context, system_database, "settings");
attachLazyOrNot<lazy, StorageSystemServerSettings>(context, system_database, "server_settings");
attachLazyOrNot<lazy, StorageSystemSettingsChanges>(context, system_database, "settings_changes");
attachLazyOrNot<lazy, SystemMergeTreeSettings<false>>(context, system_database, "merge_tree_settings");
attachLazyOrNot<lazy, SystemMergeTreeSettings<true>>(context, system_database, "replicated_merge_tree_settings");
attachLazyOrNot<lazy, StorageSystemBuildOptions>(context, system_database, "build_options");
attachLazyOrNot<lazy, StorageSystemFormats>(context, system_database, "formats");
attachLazyOrNot<lazy, StorageSystemTableFunctions>(context, system_database, "table_functions");
attachLazyOrNot<lazy, StorageSystemAggregateFunctionCombinators>(context, system_database, "aggregate_function_combinators");
attachLazyOrNot<lazy, StorageSystemDataTypeFamilies>(context, system_database, "data_type_families");
attachLazyOrNot<lazy, StorageSystemCollations>(context, system_database, "collations");
attachLazyOrNot<lazy, StorageSystemTableEngines>(context, system_database, "table_engines");
attachLazyOrNot<lazy, StorageSystemContributors>(context, system_database, "contributors");
attachLazyOrNot<lazy, StorageSystemUsers>(context, system_database, "users");
attachLazyOrNot<lazy, StorageSystemRoles>(context, system_database, "roles");
attachLazyOrNot<lazy, StorageSystemGrants>(context, system_database, "grants");
attachLazyOrNot<lazy, StorageSystemRoleGrants>(context, system_database, "role_grants");
attachLazyOrNot<lazy, StorageSystemCurrentRoles>(context, system_database, "current_roles");
attachLazyOrNot<lazy, StorageSystemEnabledRoles>(context, system_database, "enabled_roles");
attachLazyOrNot<lazy, StorageSystemSettingsProfiles>(context, system_database, "settings_profiles");
attachLazyOrNot<lazy, StorageSystemSettingsProfileElements>(context, system_database, "settings_profile_elements");
attachLazyOrNot<lazy, StorageSystemRowPolicies>(context, system_database, "row_policies");
attachLazyOrNot<lazy, StorageSystemQuotas>(context, system_database, "quotas");
attachLazyOrNot<lazy, StorageSystemQuotaLimits>(context, system_database, "quota_limits");
attachLazyOrNot<lazy, StorageSystemQuotaUsage>(context, system_database, "quota_usage");
attachLazyOrNot<lazy, StorageSystemQuotasUsage>(context, system_database, "quotas_usage");
attachLazyOrNot<lazy, StorageSystemUserDirectories>(context, system_database, "user_directories");
attachLazyOrNot<lazy, StorageSystemPrivileges>(context, system_database, "privileges");
attachLazyOrNot<lazy, StorageSystemErrors>(context, system_database, "errors");
attachLazyOrNot<lazy, StorageSystemWarnings>(context, system_database, "warnings");
attachLazyOrNot<lazy, StorageSystemDataSkippingIndices>(context, system_database, "data_skipping_indices");
attachLazyOrNot<lazy, StorageSystemLicenses>(context, system_database, "licenses");
attachLazyOrNot<lazy, StorageSystemTimeZones>(context, system_database, "time_zones");
attachLazyOrNot<lazy, StorageSystemBackups>(context, system_database, "backups");
attachLazyOrNot<lazy, StorageSystemSchemaInferenceCache>(context, system_database, "schema_inference_cache");
attachLazyOrNot<lazy, StorageSystemDroppedTables>(context, system_database, "dropped_tables");
attachLazyOrNot<lazy, StorageSystemScheduler>(context, system_database, "scheduler");
attach<StorageSystemOne>(context, system_database, "one");
attach<StorageSystemNumbers>(context, system_database, "numbers", false);
attach<StorageSystemNumbers>(context, system_database, "numbers_mt", true);
attach<StorageSystemZeros>(context, system_database, "zeros", false);
attach<StorageSystemZeros>(context, system_database, "zeros_mt", true);
attach<StorageSystemDatabases>(context, system_database, "databases");
attach<StorageSystemTables>(context, system_database, "tables");
attach<StorageSystemColumns>(context, system_database, "columns");
attach<StorageSystemFunctions>(context, system_database, "functions");
attach<StorageSystemEvents>(context, system_database, "events");
attach<StorageSystemSettings>(context, system_database, "settings");
attach<StorageSystemServerSettings>(context, system_database, "server_settings");
attach<StorageSystemSettingsChanges>(context, system_database, "settings_changes");
attach<SystemMergeTreeSettings<false>>(context, system_database, "merge_tree_settings");
attach<SystemMergeTreeSettings<true>>(context, system_database, "replicated_merge_tree_settings");
attach<StorageSystemBuildOptions>(context, system_database, "build_options");
attach<StorageSystemFormats>(context, system_database, "formats");
attach<StorageSystemTableFunctions>(context, system_database, "table_functions");
attach<StorageSystemAggregateFunctionCombinators>(context, system_database, "aggregate_function_combinators");
attach<StorageSystemDataTypeFamilies>(context, system_database, "data_type_families");
attach<StorageSystemCollations>(context, system_database, "collations");
attach<StorageSystemTableEngines>(context, system_database, "table_engines");
attach<StorageSystemContributors>(context, system_database, "contributors");
attach<StorageSystemUsers>(context, system_database, "users");
attach<StorageSystemRoles>(context, system_database, "roles");
attach<StorageSystemGrants>(context, system_database, "grants");
attach<StorageSystemRoleGrants>(context, system_database, "role_grants");
attach<StorageSystemCurrentRoles>(context, system_database, "current_roles");
attach<StorageSystemEnabledRoles>(context, system_database, "enabled_roles");
attach<StorageSystemSettingsProfiles>(context, system_database, "settings_profiles");
attach<StorageSystemSettingsProfileElements>(context, system_database, "settings_profile_elements");
attach<StorageSystemRowPolicies>(context, system_database, "row_policies");
attach<StorageSystemQuotas>(context, system_database, "quotas");
attach<StorageSystemQuotaLimits>(context, system_database, "quota_limits");
attach<StorageSystemQuotaUsage>(context, system_database, "quota_usage");
attach<StorageSystemQuotasUsage>(context, system_database, "quotas_usage");
attach<StorageSystemUserDirectories>(context, system_database, "user_directories");
attach<StorageSystemPrivileges>(context, system_database, "privileges");
attach<StorageSystemErrors>(context, system_database, "errors");
attach<StorageSystemWarnings>(context, system_database, "warnings");
attach<StorageSystemDataSkippingIndices>(context, system_database, "data_skipping_indices");
attach<StorageSystemLicenses>(context, system_database, "licenses");
attach<StorageSystemTimeZones>(context, system_database, "time_zones");
attach<StorageSystemBackups>(context, system_database, "backups");
attach<StorageSystemSchemaInferenceCache>(context, system_database, "schema_inference_cache");
attach<StorageSystemDroppedTables>(context, system_database, "dropped_tables");
attach<StorageSystemScheduler>(context, system_database, "scheduler");
#if defined(__ELF__) && !defined(OS_FREEBSD)
attachLazyOrNot<lazy, StorageSystemSymbols>(context, system_database, "symbols");
attach<StorageSystemSymbols>(context, system_database, "symbols");
#endif
#if USE_RDKAFKA
attachLazyOrNot<lazy, StorageSystemKafkaConsumers>(context, system_database, "kafka_consumers");
attach<StorageSystemKafkaConsumers>(context, system_database, "kafka_consumers");
#endif
#ifdef OS_LINUX
attachLazyOrNot<lazy, StorageSystemStackTrace>(context, system_database, "stack_trace");
attach<StorageSystemStackTrace>(context, system_database, "stack_trace");
#endif
#if USE_ROCKSDB
attachLazyOrNot<lazy, StorageSystemRocksDB>(context, system_database, "rocksdb");
attach<StorageSystemRocksDB>(context, system_database, "rocksdb");
#endif
}
template void attachSystemTablesLocal<false>(ContextPtr context, IDatabase & system_database);
template void attachSystemTablesLocal<true>(ContextPtr context, IDatabase & system_database);
void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper)
{
attachSystemTablesLocal</* lazy= */ false>(context, system_database);
attachSystemTablesLocal(context, system_database);
attach<StorageSystemParts>(context, system_database, "parts");
attach<StorageSystemProjectionParts>(context, system_database, "projection_parts");

View File

@ -9,12 +9,8 @@ namespace DB
class AsynchronousMetrics;
class IDatabase;
template <bool lazy>
void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database);
void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper);
void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database);
void attachSystemTablesAsync(ContextPtr context, IDatabase & system_database, AsynchronousMetrics & async_metrics);
extern template void attachSystemTablesLocal<false>(ContextPtr context, IDatabase & system_database);
extern template void attachSystemTablesLocal<true>(ContextPtr context, IDatabase & system_database);
}

View File

@ -1,42 +1,15 @@
#pragma once
#include <Databases/IDatabase.h>
#include <Databases/DatabasesCommon.h>
#include <Interpreters/DatabaseCatalog.h>
namespace DB
{
template<typename StorageT, typename... StorageArgs>
void attachLazy(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args)
{
if (system_database.getUUID() == UUIDHelpers::Nil)
{
/// Attach to Ordinary database.
auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name);
system_database.registerLazyTable(context, table_name, [table_id, ... captured_args = std::forward<StorageArgs>(args)] mutable
{
return std::make_shared<StorageT>(table_id, std::forward<StorageArgs>(captured_args)...);
});
}
else
{
/// Attach to Atomic database.
/// NOTE: UUIDs are not persistent, but it's ok since no data are stored on disk for these storages
/// and path is actually not used
auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4());
DatabaseCatalog::instance().addUUIDMapping(table_id.uuid);
String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid);
system_database.registerLazyTable(context, table_name, [table_id, ... captured_args = std::forward<StorageArgs>(args)] mutable
{
return std::make_shared<StorageT>(table_id, std::forward<StorageArgs>(captured_args)...);
}, path);
}
}
template<typename StorageT, typename... StorageArgs>
void attach(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args)
{
assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE);
if (system_database.getUUID() == UUIDHelpers::Nil)
{
/// Attach to Ordinary database.
@ -55,15 +28,4 @@ void attach(ContextPtr context, IDatabase & system_database, const String & tabl
}
}
template<bool lazy, typename StorageT, typename... StorageArgs>
void attachLazyOrNot(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args)
{
assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE);
if constexpr (lazy)
attachLazy<StorageT>(context, system_database, table_name, std::forward<StorageArgs>(args)...);
else
attach<StorageT>(context, system_database, table_name, std::forward<StorageArgs>(args)...);
}
}