Add setting check_referential_table_dependencies to check referential dependencies on DROP TABLE.

This commit is contained in:
Vitaly Baranov 2023-02-02 00:30:49 +01:00
parent 9f480b8fa2
commit 4869d3806c
15 changed files with 178 additions and 69 deletions

View File

@ -641,6 +641,7 @@ class IColumn;
\
M(Bool, force_remove_data_recursively_on_drop, false, "Recursively remove data on DROP query. Avoids 'Directory not empty' error, but may silently remove detached data", 0) \
M(Bool, check_table_dependencies, true, "Check that DDL query (such as DROP TABLE or RENAME) will not break dependencies", 0) \
M(Bool, check_referential_table_dependencies, false, "Check that DDL query (such as DROP TABLE or RENAME) will not break referential dependencies", 0) \
M(Bool, use_local_cache_for_remote_storage, true, "Use local cache for remote storage like HDFS or S3, it's used for remote table engine only", 0) \
\
M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \

View File

@ -522,11 +522,13 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new
{
/// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard
if (query_context->getSettingsRef().check_table_dependencies)
bool check_ref_deps = query_context->getSettingsRef().check_referential_table_dependencies;
bool check_loading_deps = !check_ref_deps && query_context->getSettingsRef().check_table_dependencies;
if (check_ref_deps || check_loading_deps)
{
std::lock_guard lock(mutex);
for (auto & table : tables)
DatabaseCatalog::instance().checkTableCanBeRemovedOrRenamed({database_name, table.first});
DatabaseCatalog::instance().checkTableCanBeRemovedOrRenamed({database_name, table.first}, check_ref_deps, check_loading_deps);
}
try

View File

@ -2,6 +2,7 @@
#include <Common/logger_useful.h>
#include <Databases/DatabaseMemory.h>
#include <Databases/DatabasesCommon.h>
#include <Databases/DDLDependencyVisitor.h>
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
@ -143,8 +144,10 @@ void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & tabl
applyMetadataChangesToCreateQuery(it->second, metadata);
auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second);
DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies);
/// The create query of the table has been just changed, we need to update dependencies too.
auto ref_dependencies = getDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second);
auto loading_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second);
DatabaseCatalog::instance().updateDependencies(table_id, ref_dependencies, loading_dependencies);
}
std::vector<std::pair<ASTPtr, StoragePtr>> DatabaseMemory::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const

View File

@ -4,6 +4,7 @@
#include <Databases/DatabaseOnDisk.h>
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabasesCommon.h>
#include <Databases/DDLDependencyVisitor.h>
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Databases/TablesLoader.h>
#include <IO/ReadBufferFromFile.h>
@ -312,8 +313,10 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta
out.close();
}
auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast);
DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies);
/// The create query of the table has been just changed, we need to update dependencies too.
auto ref_dependencies = getDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast);
auto loading_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast);
DatabaseCatalog::instance().updateDependencies(table_id, ref_dependencies, loading_dependencies);
commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context);
}

View File

@ -36,6 +36,7 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database
, strictness_mode(strictness_mode_)
, referential_dependencies("ReferentialDeps")
, loading_dependencies("LoadingDeps")
, all_loading_dependencies("LoadingDeps")
{
metadata.default_database = global_context->getCurrentDatabase();
log = &Poco::Logger::get("TablesLoader");
@ -74,7 +75,7 @@ void TablesLoader::loadTables()
buildDependencyGraph();
/// Update existing info (it's important for ATTACH DATABASE)
DatabaseCatalog::instance().addDependencies(referential_dependencies);
DatabaseCatalog::instance().addDependencies(referential_dependencies, loading_dependencies);
/// Remove tables that do not exist
removeUnresolvableDependencies();
@ -95,18 +96,22 @@ void TablesLoader::buildDependencyGraph()
{
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())
referential_dependencies.addDependencies(table_name, new_loading_dependencies);
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.
loading_dependencies.addDependencies(table_name, new_loading_dependencies);
all_loading_dependencies.addDependencies(table_name, new_loading_dependencies);
}
referential_dependencies.log();
loading_dependencies.log();
all_loading_dependencies.log();
}
@ -130,7 +135,7 @@ void TablesLoader::removeUnresolvableDependencies()
log,
"Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently."
"Consider converting it to DDL dictionary.",
fmt::join(loading_dependencies.getDependents(table_id), ", "),
fmt::join(all_loading_dependencies.getDependents(table_id), ", "),
table_id);
}
else
@ -141,12 +146,12 @@ void TablesLoader::removeUnresolvableDependencies()
LOG_WARNING(
log,
"Tables {} depend on {}, but seems like that does not exist. Will ignore it and try to load existing tables",
fmt::join(loading_dependencies.getDependents(table_id), ", "),
fmt::join(all_loading_dependencies.getDependents(table_id), ", "),
table_id);
}
size_t num_dependencies, num_dependents;
loading_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents);
all_loading_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents);
if (num_dependencies || !num_dependents)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependencies and dependent tables as it expected to."
"It's a bug", table_id);
@ -154,13 +159,13 @@ void TablesLoader::removeUnresolvableDependencies()
return true; /// Exclude this dependency.
};
loading_dependencies.removeTablesIf(need_exclude_dependency);
all_loading_dependencies.removeTablesIf(need_exclude_dependency);
if (loading_dependencies.getNumberOfTables() != metadata.parsed_tables.size())
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.
loading_dependencies.checkNoCyclicDependencies();
all_loading_dependencies.checkNoCyclicDependencies();
}
@ -172,7 +177,7 @@ void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool)
load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1);
/// Load tables in parallel.
auto tables_to_load = loading_dependencies.getTablesSortedByDependencyForParallel();
auto tables_to_load = all_loading_dependencies.getTablesSortedByDependencyForParallel();
for (size_t level = 0; level != tables_to_load.size(); ++level)
{

View File

@ -69,6 +69,7 @@ private:
ParsedTablesMetadata metadata;
TablesDependencyGraph referential_dependencies;
TablesDependencyGraph loading_dependencies;
TablesDependencyGraph all_loading_dependencies;
Poco::Logger * log;
std::atomic<size_t> tables_processed{0};
AtomicStopwatch stopwatch;

View File

@ -224,6 +224,7 @@ void DatabaseCatalog::shutdownImpl()
}) == uuid_map.end());
databases.clear();
referential_dependencies.clear();
loading_dependencies.clear();
view_dependencies.clear();
}
@ -471,10 +472,13 @@ void DatabaseCatalog::updateDatabaseName(const String & old_name, const String &
databases.erase(it);
databases.emplace(new_name, db);
/// Update dependencies.
for (const auto & table_name : tables_in_database)
{
auto dependencies = referential_dependencies.removeDependencies(StorageID{old_name, table_name}, /* remove_isolated_tables= */ true);
referential_dependencies.addDependencies(StorageID{new_name, table_name}, dependencies);
auto removed_ref_deps = referential_dependencies.removeDependencies(StorageID{old_name, table_name}, /* remove_isolated_tables= */ true);
auto removed_loading_deps = loading_dependencies.removeDependencies(StorageID{old_name, table_name}, /* remove_isolated_tables= */ true);
referential_dependencies.addDependencies(StorageID{new_name, table_name}, removed_ref_deps);
loading_dependencies.addDependencies(StorageID{new_name, table_name}, removed_loading_deps);
}
}
@ -645,6 +649,7 @@ std::unique_ptr<DatabaseCatalog> DatabaseCatalog::database_catalog;
DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_)
: WithMutableContext(global_context_)
, referential_dependencies{"ReferentialDeps"}
, loading_dependencies{"LoadingDeps"}
, view_dependencies{"ViewDeps"}
, log(&Poco::Logger::get("DatabaseCatalog"))
{
@ -1040,53 +1045,107 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid)
});
}
void DatabaseCatalog::addDependencies(const StorageID & table_id, const std::vector<StorageID> & dependencies)
void DatabaseCatalog::addDependencies(
const StorageID & table_id,
const std::vector<StorageID> & new_referential_dependencies,
const std::vector<StorageID> & new_loading_dependencies)
{
if (new_referential_dependencies.empty() && new_loading_dependencies.empty())
return;
std::lock_guard lock{databases_mutex};
referential_dependencies.addDependencies(table_id, dependencies);
if (!new_referential_dependencies.empty())
referential_dependencies.addDependencies(table_id, new_referential_dependencies);
if (!new_loading_dependencies.empty())
loading_dependencies.addDependencies(table_id, new_loading_dependencies);
}
void DatabaseCatalog::addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies)
void DatabaseCatalog::addDependencies(
const QualifiedTableName & table_name,
const TableNamesSet & new_referential_dependencies,
const TableNamesSet & new_loading_dependencies)
{
if (new_referential_dependencies.empty() && new_loading_dependencies.empty())
return;
std::lock_guard lock{databases_mutex};
referential_dependencies.addDependencies(table_name, dependencies);
if (!new_referential_dependencies.empty())
referential_dependencies.addDependencies(table_name, new_referential_dependencies);
if (!new_loading_dependencies.empty())
loading_dependencies.addDependencies(table_name, new_loading_dependencies);
}
void DatabaseCatalog::addDependencies(const TablesDependencyGraph & extra_graph)
void DatabaseCatalog::addDependencies(
const TablesDependencyGraph & new_referential_dependencies, const TablesDependencyGraph & new_loading_dependencies)
{
std::lock_guard lock{databases_mutex};
referential_dependencies.mergeWith(extra_graph);
referential_dependencies.mergeWith(new_referential_dependencies);
loading_dependencies.mergeWith(new_loading_dependencies);
}
std::vector<StorageID> DatabaseCatalog::getDependencies(const StorageID & table_id) const
std::vector<StorageID> DatabaseCatalog::getReferentialDependencies(const StorageID & table_id) const
{
std::lock_guard lock{databases_mutex};
return referential_dependencies.getDependencies(table_id);
}
std::vector<StorageID> DatabaseCatalog::getDependents(const StorageID & table_id) const
std::vector<StorageID> DatabaseCatalog::getReferentialDependents(const StorageID & table_id) const
{
std::lock_guard lock{databases_mutex};
return referential_dependencies.getDependents(table_id);
}
std::vector<StorageID> DatabaseCatalog::removeDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database)
std::vector<StorageID> DatabaseCatalog::getLoadingDependencies(const StorageID & table_id) const
{
std::lock_guard lock{databases_mutex};
if (check_dependencies)
checkTableCanBeRemovedOrRenamedUnlocked(table_id, is_drop_database);
return referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true);
return loading_dependencies.getDependencies(table_id);
}
void DatabaseCatalog::checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database) const
std::vector<StorageID> DatabaseCatalog::getLoadingDependents(const StorageID & table_id) const
{
std::lock_guard lock{databases_mutex};
return checkTableCanBeRemovedOrRenamedUnlocked(table_id, is_drop_database);
return loading_dependencies.getDependents(table_id);
}
void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool is_drop_database) const
std::pair<std::vector<StorageID>, std::vector<StorageID>> DatabaseCatalog::removeDependencies(
const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database)
{
const auto & dependents = referential_dependencies.getDependents(removing_table);
std::lock_guard lock{databases_mutex};
checkTableCanBeRemovedOrRenamedUnlocked(table_id, check_referential_dependencies, check_loading_dependencies, is_drop_database);
return {referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true),
loading_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true)};
}
void DatabaseCatalog::updateDependencies(
const StorageID & table_id, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies)
{
std::lock_guard lock{databases_mutex};
referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true);
loading_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true);
if (!new_referential_dependencies.empty())
referential_dependencies.addDependencies(table_id, new_referential_dependencies);
if (!new_loading_dependencies.empty())
loading_dependencies.addDependencies(table_id, new_loading_dependencies);
}
void DatabaseCatalog::checkTableCanBeRemovedOrRenamed(
const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database) const
{
if (!check_referential_dependencies && !check_loading_dependencies)
return;
std::lock_guard lock{databases_mutex};
return checkTableCanBeRemovedOrRenamedUnlocked(table_id, check_referential_dependencies, check_loading_dependencies, is_drop_database);
}
void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked(
const StorageID & removing_table, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database) const
{
chassert(!check_referential_dependencies || !check_loading_dependencies); /// These flags must not be both set.
std::vector<StorageID> dependents;
if (check_referential_dependencies)
dependents = referential_dependencies.getDependents(removing_table);
else if (check_loading_dependencies)
dependents = loading_dependencies.getDependents(removing_table);
else
return;
if (!is_drop_database)
{
@ -1108,13 +1167,6 @@ void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked(const StorageID &
removing_table, fmt::join(from_other_databases, ", "));
}
void DatabaseCatalog::updateDependencies(const StorageID & table_id, const TableNamesSet & new_dependencies)
{
std::lock_guard lock{databases_mutex};
referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true);
referential_dependencies.addDependencies(table_id, new_dependencies);
}
void DatabaseCatalog::cleanupStoreDirectoryTask()
{
for (const auto & [disk_name, disk] : getContext()->getDisksMap())

View File

@ -220,17 +220,19 @@ public:
/// Referential dependencies between tables: table "A" depends on table "B"
/// if "B" is referenced in the definition of "A".
void addDependencies(const StorageID & table_id, const std::vector<StorageID> & dependencies);
void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies);
void addDependencies(const TablesDependencyGraph & extra_graph);
std::vector<StorageID> removeDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database = false);
/// Loading dependencies were used to check whether a table can be removed before we had those referential dependencies.
/// Now we support this mode (see `check_table_referential_dependencies` in Setting.h) for compatibility.
void addDependencies(const StorageID & table_id, const std::vector<StorageID> & new_referential_dependencies, const std::vector<StorageID> & new_loading_dependencies);
void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies);
void addDependencies(const TablesDependencyGraph & new_referential_dependencies, const TablesDependencyGraph & new_loading_dependencies);
std::pair<std::vector<StorageID>, std::vector<StorageID>> removeDependencies(const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database = false);
std::vector<StorageID> getReferentialDependencies(const StorageID & table_id) const;
std::vector<StorageID> getReferentialDependents(const StorageID & table_id) const;
std::vector<StorageID> getLoadingDependencies(const StorageID & table_id) const;
std::vector<StorageID> getLoadingDependents(const StorageID & table_id) const;
void updateDependencies(const StorageID & table_id, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies);
std::vector<StorageID> getDependencies(const StorageID & table_id) const;
std::vector<StorageID> getDependents(const StorageID & table_id) const;
void updateDependencies(const StorageID & table_id, const TableNamesSet & new_dependencies);
void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database = false) const;
void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database = false) const;
private:
// The global instance of database catalog. unique_ptr is to allow
@ -244,7 +246,7 @@ private:
void shutdownImpl();
void checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool is_drop_database) const TSA_REQUIRES(databases_mutex);
void checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database) const TSA_REQUIRES(databases_mutex);
struct UUIDToStorageMapPart
{
@ -287,6 +289,9 @@ private:
/// if the table "B" is referenced in the definition of the table "A".
TablesDependencyGraph referential_dependencies TSA_GUARDED_BY(databases_mutex);
/// Loading dependencies were used to check whether a table can be removed before we had referential dependencies.
TablesDependencyGraph loading_dependencies TSA_GUARDED_BY(databases_mutex);
/// View dependencies between a source table and its view.
TablesDependencyGraph view_dependencies TSA_GUARDED_BY(databases_mutex);

View File

@ -1243,9 +1243,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
/// If table has dependencies - add them to the graph
QualifiedTableName qualified_name{database_name, create.getTable()};
TableNamesSet dependencies = getLoadingDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr);
if (!dependencies.empty())
DatabaseCatalog::instance().addDependencies(qualified_name, dependencies);
auto ref_dependencies = getDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr);
auto loading_dependencies = getLoadingDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr);
DatabaseCatalog::instance().addDependencies(qualified_name, ref_dependencies, loading_dependencies);
return fillTableIfNeeded(create);
}

View File

@ -187,8 +187,9 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
if (query.permanently)
{
/// Server may fail to restart of DETACH PERMANENTLY if table has dependent ones
DatabaseCatalog::instance().removeDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies,
is_drop_or_detach_database);
bool check_ref_deps = getContext()->getSettingsRef().check_referential_table_dependencies;
bool check_loading_deps = !check_ref_deps && getContext()->getSettingsRef().check_table_dependencies;
DatabaseCatalog::instance().removeDependencies(table_id, check_ref_deps, check_loading_deps, is_drop_or_detach_database);
/// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart
database->detachTablePermanently(context_, table_id.table_name);
}
@ -233,8 +234,9 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
table->checkTableCanBeDropped();
/// Check dependencies before shutting table down
if (context_->getSettingsRef().check_table_dependencies)
DatabaseCatalog::instance().checkTableCanBeRemovedOrRenamed(table_id, is_drop_or_detach_database);
bool check_ref_deps = getContext()->getSettingsRef().check_referential_table_dependencies;
bool check_loading_deps = !check_ref_deps && getContext()->getSettingsRef().check_table_dependencies;
DatabaseCatalog::instance().checkTableCanBeRemovedOrRenamed(table_id, check_ref_deps, check_loading_deps, is_drop_or_detach_database);
table->flushAndShutdown();
@ -242,8 +244,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
if (database->getUUID() == UUIDHelpers::Nil)
table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout);
DatabaseCatalog::instance().removeDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies,
is_drop_or_detach_database);
DatabaseCatalog::instance().removeDependencies(table_id, check_ref_deps, check_loading_deps, is_drop_or_detach_database);
database->dropTable(context_, table_id.table_name, query.sync);
/// We have to drop mmapio cache when dropping table from Ordinary database

View File

@ -124,10 +124,17 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
}
else
{
std::vector<StorageID> dependencies;
StorageID from_table_id{elem.from_database_name, elem.from_table_name};
StorageID to_table_id{elem.to_database_name, elem.to_table_name};
std::vector<StorageID> ref_dependencies;
std::vector<StorageID> loading_dependencies;
if (!exchange_tables)
dependencies = database_catalog.removeDependencies(StorageID(elem.from_database_name, elem.from_table_name),
getContext()->getSettingsRef().check_table_dependencies);
{
bool check_ref_deps = getContext()->getSettingsRef().check_referential_table_dependencies;
bool check_loading_deps = !check_ref_deps && getContext()->getSettingsRef().check_table_dependencies;
std::tie(ref_dependencies, loading_dependencies) = database_catalog.removeDependencies(from_table_id, check_ref_deps, check_loading_deps);
}
database->renameTable(
getContext(),
@ -137,8 +144,7 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
exchange_tables,
rename.dictionary);
if (!dependencies.empty())
DatabaseCatalog::instance().addDependencies(StorageID(elem.to_database_name, elem.to_table_name), dependencies);
DatabaseCatalog::instance().addDependencies(to_table_id, ref_dependencies, loading_dependencies);
}
}

View File

@ -348,6 +348,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons
InterpreterSystemQuery::startStopActionInDatabase(action, /* start */ false, database_name, database, context, log);
local_context->setSetting("check_table_dependencies", false);
local_context->setSetting("check_referential_table_dependencies", false);
convertOrdinaryDatabaseToAtomic(log, local_context, database, database_name, tmp_name);
LOG_INFO(log, "Will start background operations after renaming tables in database {}", database_name);

View File

@ -491,8 +491,8 @@ protected:
if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3])
{
auto dependencies = DatabaseCatalog::instance().getDependencies(StorageID{database_name, table_name});
auto dependents = DatabaseCatalog::instance().getDependents(StorageID{database_name, table_name});
auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name});
auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name});
Array dependencies_databases;
Array dependencies_tables;

View File

@ -0,0 +1,29 @@
DROP TABLE IF EXISTS mv;
DROP TABLE IF EXISTS src;
DROP TABLE IF EXISTS dst;
CREATE TABLE src (x UInt8) ENGINE = Memory;
CREATE TABLE dst (x UInt8) ENGINE = Memory;
CREATE MATERIALIZED VIEW mv TO dst AS SELECT x FROM src;
SET check_referential_table_dependencies = 1;
-- Can't drop because of referential dependencies
DROP TABLE src; -- { serverError HAVE_DEPENDENT_OBJECTS }
DROP TABLE dst; -- { serverError HAVE_DEPENDENT_OBJECTS }
-- Ok to drop in the correct order
DROP TABLE mv;
DROP TABLE src;
DROP TABLE dst;
-- Check again with check_referential_table_dependencies = 0
CREATE TABLE src (x UInt8) ENGINE = Memory;
CREATE TABLE dst (x UInt8) ENGINE = Memory;
CREATE MATERIALIZED VIEW mv TO dst AS SELECT x FROM src;
SET check_referential_table_dependencies = 0;
DROP TABLE src;
DROP TABLE dst;
DROP TABLE mv;