ClickHouse/src/Databases/TablesLoader.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

207 lines
8.0 KiB
C++
Raw Normal View History

#include <Databases/TablesLoader.h>
2021-08-31 08:53:48 +00:00
#include <Databases/IDatabase.h>
#include <Databases/DDLDependencyVisitor.h>
#include <Databases/DDLLoadingDependencyVisitor.h>
2021-08-31 08:53:48 +00:00
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Context.h>
2021-09-01 19:42:49 +00:00
#include <Interpreters/ExternalDictionariesLoader.h>
2021-08-31 08:53:48 +00:00
#include <Poco/Util/AbstractConfiguration.h>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
2021-08-31 08:53:48 +00:00
#include <Common/ThreadPool.h>
2021-09-01 19:42:49 +00:00
#include <numeric>
namespace DB
{
2021-08-31 08:53:48 +00:00
namespace ErrorCodes
{
2021-09-02 13:34:46 +00:00
extern const int LOGICAL_ERROR;
2021-08-31 08:53:48 +00:00
}
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch)
{
if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
{
LOG_INFO(log, "{}%", processed * 100.0 / total);
watch.restart();
}
}
2022-07-20 20:54:43 +00:00
TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases databases_, LoadingStrictnessLevel strictness_mode_)
: global_context(global_context_)
, databases(std::move(databases_))
2022-07-20 20:54:43 +00:00
, strictness_mode(strictness_mode_)
, referential_dependencies("ReferentialDeps")
, loading_dependencies("LoadingDeps")
, all_loading_dependencies("LoadingDeps")
{
2021-09-13 19:11:16 +00:00
metadata.default_database = global_context->getCurrentDatabase();
log = &Poco::Logger::get("TablesLoader");
}
void TablesLoader::loadTables()
{
2021-08-31 08:53:48 +00:00
bool need_resolve_dependencies = !global_context->getConfigRef().has("ignore_table_dependencies_on_metadata_loading");
2021-09-01 19:42:49 +00:00
/// Load all Lazy, MySQl, PostgreSQL, SQLite, etc databases first.
for (auto & database : databases)
{
2021-09-01 19:42:49 +00:00
if (need_resolve_dependencies && database.second->supportsLoadingInTopologicalOrder())
databases_to_load.push_back(database.first);
else
2022-07-20 20:54:43 +00:00
database.second->loadStoredObjects(global_context, strictness_mode, /* skip_startup_tables */ true);
}
2021-09-13 19:11:16 +00:00
if (databases_to_load.empty())
return;
2021-09-01 19:42:49 +00:00
/// Read and parse metadata from Ordinary, Atomic, Materialized*, Replicated, etc databases. Build dependency graph.
for (auto & database_name : databases_to_load)
2021-08-31 08:53:48 +00:00
{
2022-07-20 20:54:43 +00:00
databases[database_name]->beforeLoadingMetadata(global_context, strictness_mode);
bool is_startup = LoadingStrictnessLevel::FORCE_ATTACH <= strictness_mode;
databases[database_name]->loadTablesMetadata(global_context, metadata, is_startup);
2021-08-31 08:53:48 +00:00
}
2021-09-02 13:34:46 +00:00
LOG_INFO(log, "Parsed metadata of {} tables in {} databases in {} sec",
2021-09-13 19:11:16 +00:00
metadata.parsed_tables.size(), databases_to_load.size(), stopwatch.elapsedSeconds());
2021-09-01 19:42:49 +00:00
stopwatch.restart();
buildDependencyGraph();
/// Update existing info (it's important for ATTACH DATABASE)
DatabaseCatalog::instance().addDependencies(referential_dependencies, loading_dependencies);
/// Remove tables that do not exist
removeUnresolvableDependencies();
2021-09-01 19:42:49 +00:00
loadTablesInTopologicalOrder(pool);
}
2021-09-01 19:42:49 +00:00
void TablesLoader::startupTables()
{
/// Startup tables after all tables are loaded. Background tasks (merges, mutations, etc) may slow down data parts loading.
for (auto & database : databases)
2022-07-20 20:54:43 +00:00
database.second->startupTables(pool, strictness_mode);
2021-09-01 19:42:49 +00:00
}
void TablesLoader::buildDependencyGraph()
2021-09-01 19:42:49 +00:00
{
for (const auto & [table_name, table_metadata] : metadata.parsed_tables)
{
auto new_ref_dependencies = getDependenciesFromCreateQuery(global_context, table_name, table_metadata.ast);
auto new_loading_dependencies = getLoadingDependenciesFromCreateQuery(global_context, table_name, table_metadata.ast);
if (!new_ref_dependencies.empty())
referential_dependencies.addDependencies(table_name, new_ref_dependencies);
if (!new_loading_dependencies.empty())
loading_dependencies.addDependencies(table_name, new_loading_dependencies);
/// We're adding `new_loading_dependencies` to the graph here even if they're empty because
/// we need to have all tables from `metadata.parsed_tables` in the graph.
all_loading_dependencies.addDependencies(table_name, new_loading_dependencies);
}
referential_dependencies.log();
all_loading_dependencies.log();
}
void TablesLoader::removeUnresolvableDependencies()
{
auto need_exclude_dependency = [this](const StorageID & table_id)
2021-08-31 08:53:48 +00:00
{
2021-09-01 19:42:49 +00:00
/// Table exists and will be loaded
if (metadata.parsed_tables.contains(table_id.getQualifiedName()))
return false;
if (DatabaseCatalog::instance().isTableExist(table_id, global_context))
{
/// Table exists and it's already loaded
}
else if (table_id.database_name == metadata.default_database &&
global_context->getExternalDictionariesLoader().has(table_id.table_name))
{
/// Tables depend on a XML dictionary.
LOG_WARNING(
log,
"Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently."
"Consider converting it to DDL dictionary.",
fmt::join(all_loading_dependencies.getDependents(table_id), ", "),
table_id);
}
else
{
/// Some tables depend on table "table_id", but there is no such table in DatabaseCatalog and we don't have its metadata.
/// We will ignore it and try to load dependent tables without "table_id"
/// (but most likely dependent tables will fail to load).
LOG_WARNING(
log,
"Tables {} depend on {}, but seems like that does not exist. Will ignore it and try to load existing tables",
fmt::join(all_loading_dependencies.getDependents(table_id), ", "),
table_id);
}
size_t num_dependencies, num_dependents;
all_loading_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents);
if (num_dependencies || !num_dependents)
2021-09-01 19:42:49 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependencies and dependent tables as it expected to."
"It's a bug", table_id);
return true; /// Exclude this dependency.
2021-09-01 19:42:49 +00:00
};
all_loading_dependencies.removeTablesIf(need_exclude_dependency);
if (all_loading_dependencies.getNumberOfTables() != metadata.parsed_tables.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of tables to be loaded is not as expected. It's a bug");
/// Cannot load tables with cyclic dependencies.
all_loading_dependencies.checkNoCyclicDependencies();
2021-09-01 19:42:49 +00:00
}
2021-09-01 19:42:49 +00:00
void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool)
{
/// Compatibility setting which should be enabled by default on attach
/// Otherwise server will be unable to start for some old-format of IPv6/IPv4 types of columns
ContextMutablePtr load_context = Context::createCopy(global_context);
load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1);
/// Load tables in parallel.
auto tables_to_load = all_loading_dependencies.getTablesSortedByDependencyForParallel();
for (size_t level = 0; level != tables_to_load.size(); ++level)
{
startLoadingTables(pool, load_context, tables_to_load[level], level);
pool.wait();
}
}
void TablesLoader::startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector<StorageID> & tables_to_load, size_t level)
{
2021-09-13 19:11:16 +00:00
size_t total_tables = metadata.parsed_tables.size();
LOG_INFO(log, "Loading {} tables with dependency level {}", tables_to_load.size(), level);
for (const auto & table_id : tables_to_load)
{
pool.scheduleOrThrowOnError([this, load_context, total_tables, table_name = table_id.getQualifiedName()]()
{
2021-09-13 19:11:16 +00:00
const auto & path_and_query = metadata.parsed_tables[table_name];
2022-07-20 20:54:43 +00:00
databases[table_name.database]->loadTableFromMetadata(load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode);
2021-09-01 19:42:49 +00:00
logAboutProgress(log, ++tables_processed, total_tables, stopwatch);
});
}
}
2021-09-01 19:42:49 +00:00
}