2021-08-26 13:19:52 +00:00
|
|
|
#include <Databases/TablesLoader.h>
|
2021-08-31 08:53:48 +00:00
|
|
|
#include <Databases/IDatabase.h>
|
2021-11-01 18:53:07 +00:00
|
|
|
#include <Databases/DDLDependencyVisitor.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>
|
2021-08-26 13:19:52 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2021-08-31 08:53:48 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int INFINITE_LOOP;
|
2021-09-02 13:34:46 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2021-08-31 08:53:48 +00:00
|
|
|
}
|
|
|
|
|
2021-08-26 13:19:52 +00:00
|
|
|
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
|
|
|
|
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
|
|
|
|
|
2021-11-01 18:53:07 +00:00
|
|
|
void mergeDependenciesGraphs(DependenciesInfos & main_dependencies_info, const DependenciesInfos & additional_info)
|
|
|
|
{
|
|
|
|
for (const auto & table_and_info : additional_info)
|
|
|
|
{
|
|
|
|
const QualifiedTableName & table = table_and_info.first;
|
|
|
|
const TableNamesSet & dependent_tables = table_and_info.second.dependent_database_objects;
|
|
|
|
const TableNamesSet & dependencies = table_and_info.second.dependencies;
|
|
|
|
|
|
|
|
DependenciesInfo & maybe_existing_info = main_dependencies_info[table];
|
|
|
|
maybe_existing_info.dependent_database_objects.insert(dependent_tables.begin(), dependent_tables.end());
|
|
|
|
if (!dependencies.empty())
|
|
|
|
{
|
|
|
|
if (maybe_existing_info.dependencies.empty())
|
|
|
|
maybe_existing_info.dependencies = dependencies;
|
|
|
|
else if (maybe_existing_info.dependencies != dependencies)
|
2021-11-23 09:41:54 +00:00
|
|
|
{
|
|
|
|
/// Can happen on DatabaseReplicated recovery
|
|
|
|
LOG_WARNING(&Poco::Logger::get("TablesLoader"), "Replacing outdated dependencies ({}) of {} with: {}",
|
|
|
|
fmt::join(maybe_existing_info.dependencies, ", "),
|
|
|
|
table,
|
|
|
|
fmt::join(dependencies, ", "));
|
|
|
|
for (const auto & old_dependency : maybe_existing_info.dependencies)
|
|
|
|
{
|
|
|
|
[[maybe_unused]] bool removed = main_dependencies_info[old_dependency].dependent_database_objects.erase(table);
|
|
|
|
assert(removed);
|
|
|
|
}
|
|
|
|
maybe_existing_info.dependencies = dependencies;
|
|
|
|
}
|
2021-11-01 18:53:07 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-08-26 13:19:52 +00:00
|
|
|
|
|
|
|
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_)
|
2021-08-26 13:19:52 +00:00
|
|
|
: global_context(global_context_)
|
|
|
|
, databases(std::move(databases_))
|
2022-07-20 20:54:43 +00:00
|
|
|
, strictness_mode(strictness_mode_)
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
2021-09-13 19:11:16 +00:00
|
|
|
metadata.default_database = global_context->getCurrentDatabase();
|
2021-08-26 13:19:52 +00:00
|
|
|
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.
|
2021-08-26 13:19:52 +00:00
|
|
|
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);
|
2021-08-26 13:19:52 +00:00
|
|
|
else
|
2022-07-20 20:54:43 +00:00
|
|
|
database.second->loadStoredObjects(global_context, strictness_mode, /* skip_startup_tables */ true);
|
2021-08-26 13:19:52 +00:00
|
|
|
}
|
|
|
|
|
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-08-26 13:19:52 +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();
|
|
|
|
|
|
|
|
logDependencyGraph();
|
|
|
|
|
2021-11-01 18:53:07 +00:00
|
|
|
/// Remove tables that do not exist
|
|
|
|
removeUnresolvableDependencies(/* remove_loaded */ false);
|
|
|
|
|
|
|
|
/// Update existing info (it's important for ATTACH DATABASE)
|
|
|
|
DatabaseCatalog::instance().addLoadingDependencies(metadata.dependencies_info);
|
|
|
|
|
2021-09-01 19:42:49 +00:00
|
|
|
/// Some tables were loaded by database with loadStoredObjects(...). Remove them from graph if necessary.
|
2021-11-01 18:53:07 +00:00
|
|
|
removeUnresolvableDependencies(/* remove_loaded */ true);
|
2021-09-01 19:42:49 +00:00
|
|
|
|
|
|
|
loadTablesInTopologicalOrder(pool);
|
|
|
|
}
|
|
|
|
|
|
|
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-11-01 18:53:07 +00:00
|
|
|
void TablesLoader::removeUnresolvableDependencies(bool remove_loaded)
|
2021-09-01 19:42:49 +00:00
|
|
|
{
|
2021-11-01 18:53:07 +00:00
|
|
|
auto need_exclude_dependency = [this, remove_loaded](const QualifiedTableName & dependency_name, const DependenciesInfo & info)
|
2021-08-31 08:53:48 +00:00
|
|
|
{
|
2021-09-01 19:42:49 +00:00
|
|
|
/// Table exists and will be loaded
|
2021-09-13 19:11:16 +00:00
|
|
|
if (metadata.parsed_tables.contains(dependency_name))
|
2021-08-26 13:19:52 +00:00
|
|
|
return false;
|
2021-09-01 19:42:49 +00:00
|
|
|
/// Table exists and it's already loaded
|
2021-08-26 13:19:52 +00:00
|
|
|
if (DatabaseCatalog::instance().isTableExist(StorageID(dependency_name.database, dependency_name.table), global_context))
|
2021-11-01 18:53:07 +00:00
|
|
|
return remove_loaded;
|
2021-12-06 13:35:34 +00:00
|
|
|
/// It's XML dictionary.
|
2021-09-13 19:11:16 +00:00
|
|
|
if (dependency_name.database == metadata.default_database &&
|
2021-09-01 19:42:49 +00:00
|
|
|
global_context->getExternalDictionariesLoader().has(dependency_name.table))
|
2021-12-06 13:35:34 +00:00
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently."
|
|
|
|
"Consider converting it to DDL dictionary.", fmt::join(info.dependent_database_objects, ", "), dependency_name);
|
|
|
|
return true;
|
|
|
|
}
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2021-09-01 19:42:49 +00:00
|
|
|
/// Some tables depends on table "dependency_name", 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 "dependency_name"
|
|
|
|
/// (but most likely dependent tables will fail to load).
|
|
|
|
LOG_WARNING(log, "Tables {} depend on {}, but seems like the it does not exist. Will ignore it and try to load existing tables",
|
2021-09-13 19:11:16 +00:00
|
|
|
fmt::join(info.dependent_database_objects, ", "), dependency_name);
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2021-11-01 18:53:07 +00:00
|
|
|
if (!info.dependencies.empty())
|
2021-09-01 19:42:49 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not exist, but we have seen its AST and found {} dependencies."
|
2021-11-01 18:53:07 +00:00
|
|
|
"It's a bug", dependency_name, info.dependencies.size());
|
2021-09-13 19:11:16 +00:00
|
|
|
if (info.dependent_database_objects.empty())
|
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", dependency_name);
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2021-09-01 19:42:49 +00:00
|
|
|
return true;
|
|
|
|
};
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
auto table_it = metadata.dependencies_info.begin();
|
|
|
|
while (table_it != metadata.dependencies_info.end())
|
2021-08-31 08:53:48 +00:00
|
|
|
{
|
2021-09-01 19:42:49 +00:00
|
|
|
auto & info = table_it->second;
|
|
|
|
if (need_exclude_dependency(table_it->first, info))
|
2021-09-13 19:11:16 +00:00
|
|
|
table_it = removeResolvedDependency(table_it, metadata.independent_database_objects);
|
2021-09-01 19:42:49 +00:00
|
|
|
else
|
|
|
|
++table_it;
|
|
|
|
}
|
|
|
|
}
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2021-09-01 19:42:49 +00:00
|
|
|
void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool)
|
|
|
|
{
|
2022-06-29 15:53:08 +00:00
|
|
|
/// 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);
|
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
/// Load independent tables in parallel.
|
|
|
|
/// Then remove loaded tables from dependency graph, find tables/dictionaries that do not have unresolved dependencies anymore,
|
|
|
|
/// move them to the list of independent tables and load.
|
|
|
|
/// Repeat until we have some tables to load.
|
|
|
|
/// If we do not, then either all objects are loaded or there is cyclic dependency.
|
|
|
|
/// Complexity: O(V + E)
|
2021-08-26 13:19:52 +00:00
|
|
|
size_t level = 0;
|
|
|
|
do
|
|
|
|
{
|
2021-09-13 19:11:16 +00:00
|
|
|
assert(metadata.parsed_tables.size() == tables_processed + metadata.independent_database_objects.size() + getNumberOfTablesWithDependencies());
|
2021-09-01 19:42:49 +00:00
|
|
|
logDependencyGraph();
|
|
|
|
|
2022-06-29 15:53:08 +00:00
|
|
|
startLoadingIndependentTables(pool, level, load_context);
|
2021-09-01 19:42:49 +00:00
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
TableNames new_independent_database_objects;
|
|
|
|
for (const auto & table_name : metadata.independent_database_objects)
|
2021-09-01 19:42:49 +00:00
|
|
|
{
|
2021-09-13 19:11:16 +00:00
|
|
|
auto info_it = metadata.dependencies_info.find(table_name);
|
|
|
|
if (info_it == metadata.dependencies_info.end())
|
2021-09-01 19:42:49 +00:00
|
|
|
{
|
|
|
|
/// No tables depend on table_name and it was not even added to dependencies_info
|
|
|
|
continue;
|
|
|
|
}
|
2021-09-13 19:11:16 +00:00
|
|
|
removeResolvedDependency(info_it, new_independent_database_objects);
|
2021-09-01 19:42:49 +00:00
|
|
|
}
|
|
|
|
|
2021-08-26 13:19:52 +00:00
|
|
|
pool.wait();
|
2021-09-01 19:42:49 +00:00
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
metadata.independent_database_objects = std::move(new_independent_database_objects);
|
2021-08-26 13:19:52 +00:00
|
|
|
++level;
|
2021-09-13 19:11:16 +00:00
|
|
|
} while (!metadata.independent_database_objects.empty());
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2021-09-01 19:42:49 +00:00
|
|
|
checkCyclicDependencies();
|
2021-08-26 13:19:52 +00:00
|
|
|
}
|
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
DependenciesInfosIter TablesLoader::removeResolvedDependency(const DependenciesInfosIter & info_it, TableNames & independent_database_objects)
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
2021-11-01 18:53:07 +00:00
|
|
|
const QualifiedTableName & table_name = info_it->first;
|
|
|
|
const DependenciesInfo & info = info_it->second;
|
|
|
|
if (!info.dependencies.empty())
|
2021-09-01 19:42:49 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} is in list of independent tables, but dependencies count is {}."
|
2021-11-01 18:53:07 +00:00
|
|
|
"It's a bug", table_name, info.dependencies.size());
|
2021-09-13 19:11:16 +00:00
|
|
|
if (info.dependent_database_objects.empty())
|
2021-11-01 18:53:07 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependent tables. It's a bug", table_name);
|
2021-09-01 19:42:49 +00:00
|
|
|
|
|
|
|
/// Decrement number of dependencies for each dependent table
|
2021-11-01 18:53:07 +00:00
|
|
|
for (const auto & dependent_table : info.dependent_database_objects)
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
2021-09-13 19:11:16 +00:00
|
|
|
auto & dependent_info = metadata.dependencies_info[dependent_table];
|
2021-11-01 18:53:07 +00:00
|
|
|
auto & dependencies_set = dependent_info.dependencies;
|
|
|
|
if (dependencies_set.empty())
|
2021-09-01 19:42:49 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to decrement 0 dependencies counter for {}. It's a bug", dependent_table);
|
2021-11-01 18:53:07 +00:00
|
|
|
if (!dependencies_set.erase(table_name))
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove {} from dependencies set of {}, it contains only {}",
|
|
|
|
table_name, dependent_table, fmt::join(dependencies_set, ", "));
|
|
|
|
if (dependencies_set.empty())
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
2021-09-13 19:11:16 +00:00
|
|
|
independent_database_objects.push_back(dependent_table);
|
|
|
|
if (dependent_info.dependent_database_objects.empty())
|
|
|
|
metadata.dependencies_info.erase(dependent_table);
|
2021-08-26 13:19:52 +00:00
|
|
|
}
|
|
|
|
}
|
2021-09-01 19:42:49 +00:00
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
return metadata.dependencies_info.erase(info_it);
|
2021-08-26 13:19:52 +00:00
|
|
|
}
|
|
|
|
|
2022-06-29 15:53:08 +00:00
|
|
|
void TablesLoader::startLoadingIndependentTables(ThreadPool & pool, size_t level, ContextMutablePtr load_context)
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
2021-09-13 19:11:16 +00:00
|
|
|
size_t total_tables = metadata.parsed_tables.size();
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
LOG_INFO(log, "Loading {} tables with {} dependency level", metadata.independent_database_objects.size(), level);
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
for (const auto & table_name : metadata.independent_database_objects)
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
2022-06-29 15:53:08 +00:00
|
|
|
pool.scheduleOrThrowOnError([this, load_context, total_tables, &table_name]()
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
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-08-26 13:19:52 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-09-01 19:42:49 +00:00
|
|
|
size_t TablesLoader::getNumberOfTablesWithDependencies() const
|
|
|
|
{
|
|
|
|
size_t number_of_tables_with_dependencies = 0;
|
2021-09-13 19:11:16 +00:00
|
|
|
for (const auto & info : metadata.dependencies_info)
|
2021-11-01 18:53:07 +00:00
|
|
|
if (!info.second.dependencies.empty())
|
2021-09-01 19:42:49 +00:00
|
|
|
++number_of_tables_with_dependencies;
|
|
|
|
return number_of_tables_with_dependencies;
|
|
|
|
}
|
|
|
|
|
2021-08-26 13:19:52 +00:00
|
|
|
void TablesLoader::checkCyclicDependencies() const
|
|
|
|
{
|
2021-09-01 19:42:49 +00:00
|
|
|
/// Loading is finished if all dependencies are resolved
|
2021-09-13 19:11:16 +00:00
|
|
|
if (metadata.dependencies_info.empty())
|
2021-08-26 13:19:52 +00:00
|
|
|
return;
|
|
|
|
|
2021-09-13 19:11:16 +00:00
|
|
|
for (const auto & info : metadata.dependencies_info)
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
2021-09-01 19:42:49 +00:00
|
|
|
LOG_WARNING(log, "Cannot resolve dependencies: Table {} have {} dependencies and {} dependent tables. List of dependent tables: {}",
|
2021-11-01 18:53:07 +00:00
|
|
|
info.first, info.second.dependencies.size(),
|
2021-09-13 19:11:16 +00:00
|
|
|
info.second.dependent_database_objects.size(), fmt::join(info.second.dependent_database_objects, ", "));
|
2021-11-01 18:53:07 +00:00
|
|
|
assert(info.second.dependencies.empty());
|
2021-08-26 13:19:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
throw Exception(ErrorCodes::INFINITE_LOOP, "Cannot attach {} tables due to cyclic dependencies. "
|
2021-09-13 19:11:16 +00:00
|
|
|
"See server log for details.", metadata.dependencies_info.size());
|
2021-08-26 13:19:52 +00:00
|
|
|
}
|
|
|
|
|
2021-09-01 19:42:49 +00:00
|
|
|
void TablesLoader::logDependencyGraph() const
|
|
|
|
{
|
2021-09-13 19:11:16 +00:00
|
|
|
LOG_TEST(log, "Have {} independent tables: {}",
|
|
|
|
metadata.independent_database_objects.size(),
|
|
|
|
fmt::join(metadata.independent_database_objects, ", "));
|
|
|
|
for (const auto & dependencies : metadata.dependencies_info)
|
2021-09-01 19:42:49 +00:00
|
|
|
{
|
2021-09-13 19:11:16 +00:00
|
|
|
LOG_TEST(log,
|
2021-09-01 19:42:49 +00:00
|
|
|
"Table {} have {} dependencies and {} dependent tables. List of dependent tables: {}",
|
|
|
|
dependencies.first,
|
2021-11-01 18:53:07 +00:00
|
|
|
dependencies.second.dependencies.size(),
|
2021-09-13 19:11:16 +00:00
|
|
|
dependencies.second.dependent_database_objects.size(),
|
|
|
|
fmt::join(dependencies.second.dependent_database_objects, ", "));
|
2021-09-01 19:42:49 +00:00
|
|
|
}
|
2021-08-26 13:19:52 +00:00
|
|
|
}
|
|
|
|
|
2021-09-01 19:42:49 +00:00
|
|
|
}
|