mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Merge pull request #65405 from Avogar/check-cyclic-dependencies-on-create
Check cyclic dependencies on CREATE/REPLACE/RENAME/EXCHANGE queries
This commit is contained in:
commit
c97ef5dc32
@ -438,7 +438,7 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_
|
||||
String create_table_query_str = serializeAST(*create_table_query);
|
||||
|
||||
bool is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table});
|
||||
auto table_dependencies = getDependenciesFromCreateQuery(context, table_name, create_table_query);
|
||||
auto table_dependencies = getDependenciesFromCreateQuery(context, table_name, create_table_query, context->getCurrentDatabase());
|
||||
bool table_has_data = backup->hasFiles(data_path_in_backup);
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
|
@ -30,8 +30,8 @@ namespace
|
||||
{
|
||||
friend void tryVisitNestedSelect(const String & query, DDLDependencyVisitorData & data);
|
||||
public:
|
||||
DDLDependencyVisitorData(const ContextPtr & context_, const QualifiedTableName & table_name_, const ASTPtr & ast_)
|
||||
: create_query(ast_), table_name(table_name_), current_database(context_->getCurrentDatabase()), context(context_)
|
||||
DDLDependencyVisitorData(const ContextPtr & global_context_, const QualifiedTableName & table_name_, const ASTPtr & ast_, const String & current_database_)
|
||||
: create_query(ast_), table_name(table_name_), default_database(global_context_->getCurrentDatabase()), current_database(current_database_), global_context(global_context_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -71,8 +71,9 @@ namespace
|
||||
ASTPtr create_query;
|
||||
std::unordered_set<const IAST *> skip_asts;
|
||||
QualifiedTableName table_name;
|
||||
String default_database;
|
||||
String current_database;
|
||||
ContextPtr context;
|
||||
ContextPtr global_context;
|
||||
TableNamesSet dependencies;
|
||||
|
||||
/// CREATE TABLE or CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query.
|
||||
@ -95,6 +96,11 @@ namespace
|
||||
as_table.database = current_database;
|
||||
dependencies.emplace(as_table);
|
||||
}
|
||||
|
||||
/// Visit nested select query only for views, for other cases it's not
|
||||
/// an actual dependency as it will be executed only once to fill the table.
|
||||
if (create.select && !create.isView())
|
||||
skip_asts.insert(create.select);
|
||||
}
|
||||
|
||||
/// The definition of a dictionary: SOURCE(CLICKHOUSE(...)) LAYOUT(...) LIFETIME(...)
|
||||
@ -103,8 +109,8 @@ namespace
|
||||
if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements)
|
||||
return;
|
||||
|
||||
auto config = getDictionaryConfigurationFromAST(create_query->as<ASTCreateQuery &>(), context);
|
||||
auto info = getInfoIfClickHouseDictionarySource(config, context);
|
||||
auto config = getDictionaryConfigurationFromAST(create_query->as<ASTCreateQuery &>(), global_context);
|
||||
auto info = getInfoIfClickHouseDictionarySource(config, global_context);
|
||||
|
||||
/// We consider only dependencies on local tables.
|
||||
if (!info || !info->is_local)
|
||||
@ -112,14 +118,21 @@ namespace
|
||||
|
||||
if (!info->table_name.table.empty())
|
||||
{
|
||||
/// If database is not specified in dictionary source, use database of the dictionary itself, not the current/default database.
|
||||
if (info->table_name.database.empty())
|
||||
info->table_name.database = current_database;
|
||||
info->table_name.database = table_name.database;
|
||||
dependencies.emplace(std::move(info->table_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We don't have a table name, we have a select query instead
|
||||
/// We don't have a table name, we have a select query instead.
|
||||
/// All tables from select query in dictionary definition won't
|
||||
/// use current database, as this query is executed with global context.
|
||||
/// Use default database from global context while visiting select query.
|
||||
String current_database_ = current_database;
|
||||
current_database = default_database;
|
||||
tryVisitNestedSelect(info->query, *this);
|
||||
current_database = current_database_;
|
||||
}
|
||||
}
|
||||
|
||||
@ -176,7 +189,7 @@ namespace
|
||||
|
||||
if (auto cluster_name = tryGetClusterNameFromArgument(table_engine, 0))
|
||||
{
|
||||
auto cluster = context->tryGetCluster(*cluster_name);
|
||||
auto cluster = global_context->tryGetCluster(*cluster_name);
|
||||
if (cluster && cluster->getLocalShardCount())
|
||||
has_local_replicas = true;
|
||||
}
|
||||
@ -231,7 +244,7 @@ namespace
|
||||
{
|
||||
if (auto cluster_name = tryGetClusterNameFromArgument(function, 0))
|
||||
{
|
||||
if (auto cluster = context->tryGetCluster(*cluster_name))
|
||||
if (auto cluster = global_context->tryGetCluster(*cluster_name))
|
||||
{
|
||||
if (cluster->getLocalShardCount())
|
||||
has_local_replicas = true;
|
||||
@ -303,7 +316,10 @@ namespace
|
||||
try
|
||||
{
|
||||
/// We're just searching for dependencies here, it's not safe to execute subqueries now.
|
||||
auto evaluated = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
/// Use copy of the global_context and set current database, because expressions can contain currentDatabase() function.
|
||||
ContextMutablePtr global_context_copy = Context::createCopy(global_context);
|
||||
global_context_copy->setCurrentDatabase(current_database);
|
||||
auto evaluated = evaluateConstantExpressionOrIdentifierAsLiteral(arg, global_context_copy);
|
||||
const auto * literal = evaluated->as<ASTLiteral>();
|
||||
if (!literal || (literal->value.getType() != Field::Types::String))
|
||||
return {};
|
||||
@ -444,7 +460,7 @@ namespace
|
||||
ParserSelectWithUnionQuery parser;
|
||||
String description = fmt::format("Query for ClickHouse dictionary {}", data.table_name);
|
||||
String fixed_query = removeWhereConditionPlaceholder(query);
|
||||
const Settings & settings = data.context->getSettingsRef();
|
||||
const Settings & settings = data.global_context->getSettingsRef();
|
||||
ASTPtr select = parseQuery(parser, fixed_query, description,
|
||||
settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks);
|
||||
|
||||
@ -459,12 +475,19 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & context, const QualifiedTableName & table_name, const ASTPtr & ast)
|
||||
TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & current_database)
|
||||
{
|
||||
DDLDependencyVisitor::Data data{context, table_name, ast};
|
||||
DDLDependencyVisitor::Data data{global_global_context, table_name, ast, current_database};
|
||||
DDLDependencyVisitor::Visitor visitor{data};
|
||||
visitor.visit(ast);
|
||||
return std::move(data).getDependencies();
|
||||
}
|
||||
|
||||
TableNamesSet getDependenciesFromDictionaryNestedSelectQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & select_query, const String & current_database)
|
||||
{
|
||||
DDLDependencyVisitor::Data data{global_context, table_name, ast, current_database};
|
||||
tryVisitNestedSelect(select_query, data);
|
||||
return std::move(data).getDependencies();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -13,6 +13,9 @@ using TableNamesSet = std::unordered_set<QualifiedTableName>;
|
||||
/// Returns a list of all tables explicitly referenced in the create query of a specified table.
|
||||
/// For example, a column default expression can use dictGet() and thus reference a dictionary.
|
||||
/// Does not validate AST, works a best-effort way.
|
||||
TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & context, const QualifiedTableName & table_name, const ASTPtr & ast);
|
||||
TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & current_database);
|
||||
|
||||
/// Returns a list of all tables explicitly referenced in the select query specified as a dictionary source.
|
||||
TableNamesSet getDependenciesFromDictionaryNestedSelectQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & select_query, const String & current_database);
|
||||
|
||||
}
|
||||
|
@ -110,19 +110,30 @@ void DDLLoadingDependencyVisitor::visit(const ASTFunctionWithKeyValueArguments &
|
||||
auto config = getDictionaryConfigurationFromAST(data.create_query->as<ASTCreateQuery &>(), data.global_context);
|
||||
auto info = getInfoIfClickHouseDictionarySource(config, data.global_context);
|
||||
|
||||
if (!info || !info->is_local || info->table_name.table.empty())
|
||||
if (!info || !info->is_local)
|
||||
return;
|
||||
|
||||
if (info->table_name.database.empty())
|
||||
info->table_name.database = data.default_database;
|
||||
data.dependencies.emplace(std::move(info->table_name));
|
||||
if (!info->table_name.table.empty())
|
||||
{
|
||||
/// If database is not specified in dictionary source, use database of the dictionary itself, not the current/default database.
|
||||
if (info->table_name.database.empty())
|
||||
info->table_name.database = data.table_name.database;
|
||||
data.dependencies.emplace(std::move(info->table_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We don't have a table name, we have a select query instead that will be executed during dictionary loading.
|
||||
/// We need to find all tables used in this select query and add them to dependencies.
|
||||
auto select_query_dependencies = getDependenciesFromDictionaryNestedSelectQuery(data.global_context, data.table_name, data.create_query, info->query, data.default_database);
|
||||
data.dependencies.merge(select_query_dependencies);
|
||||
}
|
||||
}
|
||||
|
||||
void DDLLoadingDependencyVisitor::visit(const ASTStorage & storage, Data & data)
|
||||
{
|
||||
if (storage.ttl_table)
|
||||
{
|
||||
auto ttl_dependensies = getDependenciesFromCreateQuery(data.global_context, data.table_name, storage.ttl_table->ptr());
|
||||
auto ttl_dependensies = getDependenciesFromCreateQuery(data.global_context, data.table_name, storage.ttl_table->ptr(), data.default_database);
|
||||
data.dependencies.merge(ttl_dependensies);
|
||||
}
|
||||
|
||||
|
@ -154,7 +154,7 @@ void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & tabl
|
||||
applyMetadataChangesToCreateQuery(it->second, metadata);
|
||||
|
||||
/// 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 ref_dependencies = getDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second, local_context->getCurrentDatabase());
|
||||
auto loading_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second);
|
||||
DatabaseCatalog::instance().updateDependencies(table_id, ref_dependencies, loading_dependencies);
|
||||
}
|
||||
|
@ -539,7 +539,7 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta
|
||||
}
|
||||
|
||||
/// 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 ref_dependencies = getDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast, local_context->getCurrentDatabase());
|
||||
auto loading_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast);
|
||||
DatabaseCatalog::instance().updateDependencies(table_id, ref_dependencies, loading_dependencies);
|
||||
|
||||
|
@ -1165,7 +1165,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
/// And QualifiedTableName::parseFromString doesn't handle this.
|
||||
auto qualified_name = QualifiedTableName{.database = getDatabaseName(), .table = table_name};
|
||||
auto query_ast = parseQueryFromMetadataInZooKeeper(table_name, create_table_query);
|
||||
tables_dependencies.addDependencies(qualified_name, getDependenciesFromCreateQuery(getContext(), qualified_name, query_ast));
|
||||
tables_dependencies.addDependencies(qualified_name, getDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ast, getContext()->getCurrentDatabase()));
|
||||
}
|
||||
|
||||
tables_dependencies.checkNoCyclicDependencies();
|
||||
|
@ -137,7 +137,7 @@ 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_ref_dependencies = getDependenciesFromCreateQuery(global_context, table_name, table_metadata.ast, global_context->getCurrentDatabase());
|
||||
auto new_loading_dependencies = getLoadingDependenciesFromCreateQuery(global_context, table_name, table_metadata.ast);
|
||||
|
||||
if (!new_ref_dependencies.empty())
|
||||
|
@ -63,6 +63,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int HAVE_DEPENDENT_OBJECTS;
|
||||
extern const int UNFINISHED;
|
||||
extern const int INFINITE_LOOP;
|
||||
}
|
||||
|
||||
class DatabaseNameHints : public IHints<>
|
||||
@ -1473,6 +1474,114 @@ void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked(
|
||||
removing_table, fmt::join(from_other_databases, ", "));
|
||||
}
|
||||
|
||||
void DatabaseCatalog::checkTableCanBeAddedWithNoCyclicDependencies(
|
||||
const QualifiedTableName & table_name,
|
||||
const TableNamesSet & new_referential_dependencies,
|
||||
const TableNamesSet & new_loading_dependencies)
|
||||
{
|
||||
std::lock_guard lock{databases_mutex};
|
||||
|
||||
StorageID table_id = StorageID{table_name};
|
||||
|
||||
auto check = [&](TablesDependencyGraph & dependencies, const TableNamesSet & new_dependencies)
|
||||
{
|
||||
auto old_dependencies = dependencies.removeDependencies(table_id);
|
||||
dependencies.addDependencies(table_name, new_dependencies);
|
||||
auto restore_dependencies = [&]()
|
||||
{
|
||||
dependencies.removeDependencies(table_id);
|
||||
if (!old_dependencies.empty())
|
||||
dependencies.addDependencies(table_id, old_dependencies);
|
||||
};
|
||||
|
||||
if (dependencies.hasCyclicDependencies())
|
||||
{
|
||||
auto cyclic_dependencies_description = dependencies.describeCyclicDependencies();
|
||||
restore_dependencies();
|
||||
throw Exception(
|
||||
ErrorCodes::INFINITE_LOOP,
|
||||
"Cannot add dependencies for '{}', because it will lead to cyclic dependencies: {}",
|
||||
table_name.getFullName(),
|
||||
cyclic_dependencies_description);
|
||||
}
|
||||
|
||||
restore_dependencies();
|
||||
};
|
||||
|
||||
check(referential_dependencies, new_referential_dependencies);
|
||||
check(loading_dependencies, new_loading_dependencies);
|
||||
}
|
||||
|
||||
void DatabaseCatalog::checkTableCanBeRenamedWithNoCyclicDependencies(const StorageID & from_table_id, const StorageID & to_table_id)
|
||||
{
|
||||
std::lock_guard lock{databases_mutex};
|
||||
|
||||
auto check = [&](TablesDependencyGraph & dependencies)
|
||||
{
|
||||
auto old_dependencies = dependencies.removeDependencies(from_table_id);
|
||||
dependencies.addDependencies(to_table_id, old_dependencies);
|
||||
auto restore_dependencies = [&]()
|
||||
{
|
||||
dependencies.removeDependencies(to_table_id);
|
||||
dependencies.addDependencies(from_table_id, old_dependencies);
|
||||
};
|
||||
|
||||
if (dependencies.hasCyclicDependencies())
|
||||
{
|
||||
auto cyclic_dependencies_description = dependencies.describeCyclicDependencies();
|
||||
restore_dependencies();
|
||||
throw Exception(
|
||||
ErrorCodes::INFINITE_LOOP,
|
||||
"Cannot rename '{}' to '{}', because it will lead to cyclic dependencies: {}",
|
||||
from_table_id.getFullTableName(),
|
||||
to_table_id.getFullTableName(),
|
||||
cyclic_dependencies_description);
|
||||
}
|
||||
|
||||
restore_dependencies();
|
||||
};
|
||||
|
||||
check(referential_dependencies);
|
||||
check(loading_dependencies);
|
||||
}
|
||||
|
||||
void DatabaseCatalog::checkTablesCanBeExchangedWithNoCyclicDependencies(const StorageID & table_id_1, const StorageID & table_id_2)
|
||||
{
|
||||
std::lock_guard lock{databases_mutex};
|
||||
|
||||
auto check = [&](TablesDependencyGraph & dependencies)
|
||||
{
|
||||
auto old_dependencies_1 = dependencies.removeDependencies(table_id_1);
|
||||
auto old_dependencies_2 = dependencies.removeDependencies(table_id_2);
|
||||
dependencies.addDependencies(table_id_1, old_dependencies_2);
|
||||
dependencies.addDependencies(table_id_2, old_dependencies_1);
|
||||
auto restore_dependencies = [&]()
|
||||
{
|
||||
dependencies.removeDependencies(table_id_1);
|
||||
dependencies.removeDependencies(table_id_2);
|
||||
dependencies.addDependencies(table_id_1, old_dependencies_1);
|
||||
dependencies.addDependencies(table_id_2, old_dependencies_2);
|
||||
};
|
||||
|
||||
if (dependencies.hasCyclicDependencies())
|
||||
{
|
||||
auto cyclic_dependencies_description = dependencies.describeCyclicDependencies();
|
||||
restore_dependencies();
|
||||
throw Exception(
|
||||
ErrorCodes::INFINITE_LOOP,
|
||||
"Cannot exchange '{}' and '{}', because it will lead to cyclic dependencies: {}",
|
||||
table_id_1.getFullTableName(),
|
||||
table_id_2.getFullTableName(),
|
||||
cyclic_dependencies_description);
|
||||
}
|
||||
|
||||
restore_dependencies();
|
||||
};
|
||||
|
||||
check(referential_dependencies);
|
||||
check(loading_dependencies);
|
||||
}
|
||||
|
||||
void DatabaseCatalog::cleanupStoreDirectoryTask()
|
||||
{
|
||||
for (const auto & [disk_name, disk] : getContext()->getDisksMap())
|
||||
|
@ -245,6 +245,9 @@ public:
|
||||
|
||||
void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database = false) const;
|
||||
|
||||
void checkTableCanBeAddedWithNoCyclicDependencies(const QualifiedTableName & table_name, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies);
|
||||
void checkTableCanBeRenamedWithNoCyclicDependencies(const StorageID & from_table_id, const StorageID & to_table_id);
|
||||
void checkTablesCanBeExchangedWithNoCyclicDependencies(const StorageID & table_id_1, const StorageID & table_id_2);
|
||||
|
||||
struct TableMarkedAsDropped
|
||||
{
|
||||
|
@ -1111,6 +1111,27 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
void addTableDependencies(const ASTCreateQuery & create, const ASTPtr & query_ptr, const ContextPtr & context)
|
||||
{
|
||||
QualifiedTableName qualified_name{create.getDatabase(), create.getTable()};
|
||||
auto ref_dependencies = getDependenciesFromCreateQuery(context->getGlobalContext(), qualified_name, query_ptr, context->getCurrentDatabase());
|
||||
auto loading_dependencies = getLoadingDependenciesFromCreateQuery(context->getGlobalContext(), qualified_name, query_ptr);
|
||||
DatabaseCatalog::instance().addDependencies(qualified_name, ref_dependencies, loading_dependencies);
|
||||
}
|
||||
|
||||
void checkTableCanBeAddedWithNoCyclicDependencies(const ASTCreateQuery & create, const ASTPtr & query_ptr, const ContextPtr & context)
|
||||
{
|
||||
QualifiedTableName qualified_name{create.getDatabase(), create.getTable()};
|
||||
auto ref_dependencies = getDependenciesFromCreateQuery(context->getGlobalContext(), qualified_name, query_ptr, context->getCurrentDatabase());
|
||||
auto loading_dependencies = getLoadingDependenciesFromCreateQuery(context->getGlobalContext(), qualified_name, query_ptr);
|
||||
DatabaseCatalog::instance().checkTableCanBeAddedWithNoCyclicDependencies(qualified_name, ref_dependencies, loading_dependencies);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
{
|
||||
/// Temporary tables are created out of databases.
|
||||
@ -1356,11 +1377,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
return {};
|
||||
|
||||
/// If table has dependencies - add them to the graph
|
||||
QualifiedTableName qualified_name{database_name, create.getTable()};
|
||||
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);
|
||||
|
||||
addTableDependencies(create, query_ptr, getContext());
|
||||
return fillTableIfNeeded(create);
|
||||
}
|
||||
|
||||
@ -1512,6 +1529,9 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find UUID mapping for {}, it's a bug", create.uuid);
|
||||
}
|
||||
|
||||
/// Before actually creating the table, check if it will lead to cyclic dependencies.
|
||||
checkTableCanBeAddedWithNoCyclicDependencies(create, query_ptr, getContext());
|
||||
|
||||
StoragePtr res;
|
||||
/// NOTE: CREATE query may be rewritten by Storage creator or table function
|
||||
if (create.as_table_function)
|
||||
@ -1623,6 +1643,9 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
|
||||
ContextMutablePtr create_context = Context::createCopy(current_context);
|
||||
create_context->setQueryContext(std::const_pointer_cast<Context>(current_context));
|
||||
|
||||
/// Before actually creating/replacing the table, check if it will lead to cyclic dependencies.
|
||||
checkTableCanBeAddedWithNoCyclicDependencies(create, query_ptr, create_context);
|
||||
|
||||
auto make_drop_context = [&]() -> ContextMutablePtr
|
||||
{
|
||||
ContextMutablePtr drop_context = Context::createCopy(current_context);
|
||||
@ -1669,6 +1692,9 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
|
||||
assert(done);
|
||||
created = true;
|
||||
|
||||
/// If table has dependencies - add them to the graph
|
||||
addTableDependencies(create, query_ptr, getContext());
|
||||
|
||||
/// Try fill temporary table
|
||||
BlockIO fill_io = fillTableIfNeeded(create);
|
||||
executeTrivialBlockIO(fill_io, getContext());
|
||||
|
@ -127,14 +127,23 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
|
||||
{
|
||||
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;
|
||||
std::vector<StorageID> from_ref_dependencies;
|
||||
std::vector<StorageID> from_loading_dependencies;
|
||||
std::vector<StorageID> to_ref_dependencies;
|
||||
std::vector<StorageID> to_loading_dependencies;
|
||||
|
||||
if (!exchange_tables)
|
||||
if (exchange_tables)
|
||||
{
|
||||
DatabaseCatalog::instance().checkTablesCanBeExchangedWithNoCyclicDependencies(from_table_id, to_table_id);
|
||||
std::tie(from_ref_dependencies, from_loading_dependencies) = database_catalog.removeDependencies(from_table_id, false, false);
|
||||
std::tie(to_ref_dependencies, to_loading_dependencies) = database_catalog.removeDependencies(to_table_id, false, false);
|
||||
}
|
||||
else
|
||||
{
|
||||
DatabaseCatalog::instance().checkTableCanBeRenamedWithNoCyclicDependencies(from_table_id, to_table_id);
|
||||
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);
|
||||
std::tie(from_ref_dependencies, from_loading_dependencies) = database_catalog.removeDependencies(from_table_id, check_ref_deps, check_loading_deps);
|
||||
}
|
||||
|
||||
try
|
||||
@ -147,12 +156,17 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
|
||||
exchange_tables,
|
||||
rename.dictionary);
|
||||
|
||||
DatabaseCatalog::instance().addDependencies(to_table_id, ref_dependencies, loading_dependencies);
|
||||
DatabaseCatalog::instance().addDependencies(to_table_id, from_ref_dependencies, from_loading_dependencies);
|
||||
if (!to_ref_dependencies.empty() || !to_loading_dependencies.empty())
|
||||
DatabaseCatalog::instance().addDependencies(from_table_id, to_ref_dependencies, to_loading_dependencies);
|
||||
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Restore dependencies if RENAME fails
|
||||
DatabaseCatalog::instance().addDependencies(from_table_id, ref_dependencies, loading_dependencies);
|
||||
DatabaseCatalog::instance().addDependencies(from_table_id, from_ref_dependencies, from_loading_dependencies);
|
||||
if (!to_ref_dependencies.empty() || !to_loading_dependencies.empty())
|
||||
DatabaseCatalog::instance().addDependencies(to_table_id, to_ref_dependencies, to_loading_dependencies);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
@ -134,3 +134,5 @@ def test_create_or_replace(database, instance_to_create_dictionary, instances_to
|
||||
expected_result = TSV([[0, 1], [5, 26], [7, 50], [11, 0]])
|
||||
assert instance.query(select_query) == expected_result
|
||||
assert instance.query(select_query, user="dictget_user") == expected_result
|
||||
|
||||
instance_to_create_dictionary.query(f"DROP DICTIONARY IF EXISTS {database}.dict")
|
||||
|
@ -9,10 +9,6 @@ CREATE TABLE distr (x UInt8) ENGINE = Distributed(test_shard_localhost, currentD
|
||||
CREATE TABLE distr0 (x UInt8) ENGINE = Distributed(test_shard_localhost, '', distr0); -- { serverError INFINITE_LOOP }
|
||||
|
||||
CREATE TABLE distr1 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr2);
|
||||
CREATE TABLE distr2 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr1);
|
||||
|
||||
SELECT * FROM distr1; -- { serverError TOO_LARGE_DISTRIBUTED_DEPTH }
|
||||
SELECT * FROM distr2; -- { serverError TOO_LARGE_DISTRIBUTED_DEPTH }
|
||||
CREATE TABLE distr2 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr1); -- { serverError INFINITE_LOOP }
|
||||
|
||||
DROP TABLE distr1;
|
||||
DROP TABLE distr2;
|
||||
|
@ -88,6 +88,7 @@ SELECT sum(n) from rich_syntax;
|
||||
SYSTEM DROP DNS CACHE;
|
||||
|
||||
DROP TABLE file;
|
||||
DROP DICTIONARY dict;
|
||||
DROP TABLE url;
|
||||
DROP TABLE view;
|
||||
DROP TABLE buffer;
|
||||
@ -96,4 +97,3 @@ DROP TABLE merge_tf;
|
||||
DROP TABLE distributed;
|
||||
DROP TABLE distributed_tf;
|
||||
DROP TABLE rich_syntax;
|
||||
DROP DICTIONARY dict;
|
||||
|
@ -17,7 +17,7 @@ SELECT name, status FROM system.dictionaries WHERE database='test_01191';
|
||||
SELECT name, engine FROM system.tables WHERE database='test_01191' ORDER BY name;
|
||||
|
||||
RENAME DICTIONARY test_01191.table TO test_01191.table1; -- {serverError UNKNOWN_TABLE}
|
||||
EXCHANGE DICTIONARIES test_01191._ AND test_01191.dict; -- {serverError INCORRECT_QUERY}
|
||||
EXCHANGE DICTIONARIES test_01191._ AND test_01191.dict; -- {serverError INFINITE_LOOP}
|
||||
EXCHANGE TABLES test_01191.t AND test_01191.dict;
|
||||
SELECT name, status FROM system.dictionaries WHERE database='test_01191';
|
||||
SELECT name, engine FROM system.tables WHERE database='test_01191' ORDER BY name;
|
||||
|
@ -16,5 +16,5 @@ LIFETIME(MIN 10 MAX 10);
|
||||
|
||||
SELECT dictGet(currentDatabase() || '.dict', 's', number) FROM numbers(2);
|
||||
|
||||
DROP TABLE src;
|
||||
DROP DICTIONARY dict;
|
||||
DROP TABLE src;
|
||||
|
@ -27,5 +27,5 @@ SELECT dictGet('ddl_dictionary_test', 'value', number) FROM system.numbers LIMIT
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('ddl_dictionary_test', number) FROM system.numbers LIMIT 3;
|
||||
|
||||
DROP TABLE ddl_dictonary_test_source;
|
||||
DROP DICTIONARY ddl_dictionary_test;
|
||||
DROP TABLE ddl_dictonary_test_source;
|
||||
|
@ -25,8 +25,8 @@ SELECT * FROM 01760_db.example_simple_key_dictionary;
|
||||
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
|
||||
DROP TABLE 01760_db.example_simple_key_source;
|
||||
DROP DICTIONARY 01760_db.example_simple_key_dictionary;
|
||||
DROP TABLE 01760_db.example_simple_key_source;
|
||||
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
|
||||
@ -53,7 +53,7 @@ SELECT * FROM 01760_db.example_complex_key_dictionary;
|
||||
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
|
||||
DROP TABLE 01760_db.example_complex_key_source;
|
||||
DROP DICTIONARY 01760_db.example_complex_key_dictionary;
|
||||
DROP TABLE 01760_db.example_complex_key_source;
|
||||
|
||||
DROP DATABASE 01760_db;
|
||||
|
@ -17,19 +17,6 @@ ENGINE = Distributed('test_shard_localhost', '', 'tt7', rand());
|
||||
|
||||
DROP TABLE IF EXISTS tt7;
|
||||
|
||||
CREATE TABLE tt7 as tt6 ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand());
|
||||
|
||||
INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError TOO_LARGE_DISTRIBUTED_DEPTH }
|
||||
|
||||
SELECT * FROM tt6; -- { serverError TOO_LARGE_DISTRIBUTED_DEPTH }
|
||||
|
||||
SET max_distributed_depth = 0;
|
||||
|
||||
-- stack overflow
|
||||
INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError TOO_DEEP_RECURSION}
|
||||
|
||||
-- stack overflow
|
||||
SELECT * FROM tt6; -- { serverError TOO_DEEP_RECURSION }
|
||||
CREATE TABLE tt7 as tt6 ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand()); -- {serverError INFINITE_LOOP}
|
||||
|
||||
DROP TABLE tt6;
|
||||
DROP TABLE tt7;
|
||||
|
@ -23,5 +23,5 @@ LAYOUT(DIRECT());
|
||||
|
||||
SELECT * FROM dictionary('table_function_dictionary_test_dictionary');
|
||||
|
||||
DROP TABLE table_function_dictionary_source_table;
|
||||
DROP DICTIONARY table_function_dictionary_test_dictionary;
|
||||
DROP TABLE table_function_dictionary_source_table;
|
||||
|
@ -25,6 +25,8 @@ LAYOUT(FLAT());
|
||||
SELECT 'Flat dictionary';
|
||||
SELECT dictGet('flat_dictionary', 'decimal_value', toUInt64(1));
|
||||
|
||||
DROP DICTIONARY flat_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS hashed_dictionary;
|
||||
CREATE DICTIONARY hashed_dictionary
|
||||
(
|
||||
|
@ -310,6 +310,6 @@ SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabas
|
||||
SELECT tuple(x, y) as key, dictGet('polygon_dictionary_01862', 'name', key) FROM points_01862 FORMAT Null;
|
||||
SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabase() AND name = 'polygon_dictionary_01862';
|
||||
|
||||
DROP DICTIONARY polygon_dictionary_01862;
|
||||
DROP TABLE polygons_01862;
|
||||
DROP TABLE points_01862;
|
||||
DROP DICTIONARY polygon_dictionary_01862;
|
||||
|
@ -111,6 +111,8 @@ LAYOUT(IP_TRIE());
|
||||
SELECT 'IPTrie dictionary';
|
||||
SELECT dictGet('ip_trie_dictionary', 'value', tuple(IPv4StringToNum('127.0.0.0'))); --{serverError UNSUPPORTED_METHOD}
|
||||
|
||||
DROP DICTIONARY ip_trie_dictionary;
|
||||
|
||||
DROP TABLE dictionary_nullable_source_table;
|
||||
DROP TABLE dictionary_nullable_default_source_table;
|
||||
|
||||
|
@ -45,5 +45,5 @@ FROM numbers(3);
|
||||
|
||||
DROP TABLE dictionary_source_en;
|
||||
DROP TABLE dictionary_source_ru;
|
||||
DROP TABLE dictionary_source_view;
|
||||
DROP DICTIONARY flat_dictionary;
|
||||
DROP TABLE dictionary_source_view;
|
||||
|
@ -53,8 +53,8 @@ SELECT CountryID, StartDate, Tax FROM range_dictionary ORDER BY CountryID, Start
|
||||
SELECT 'onlySpecificColumn';
|
||||
SELECT Tax FROM range_dictionary ORDER BY CountryID, StartDate, EndDate;
|
||||
|
||||
DROP TABLE date_table;
|
||||
DROP DICTIONARY range_dictionary;
|
||||
DROP TABLE date_table;
|
||||
|
||||
CREATE TABLE date_table
|
||||
(
|
||||
@ -107,5 +107,5 @@ SELECT CountryID, StartDate, Tax FROM range_dictionary_nullable ORDER BY Country
|
||||
SELECT 'onlySpecificColumn';
|
||||
SELECT Tax FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate;
|
||||
|
||||
DROP TABLE date_table;
|
||||
DROP DICTIONARY range_dictionary_nullable;
|
||||
DROP TABLE date_table;
|
||||
|
@ -49,5 +49,5 @@ SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_vie
|
||||
SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view' AND database == currentDatabase();
|
||||
|
||||
DROP TABLE 02155_test_dictionary_view;
|
||||
DROP TABLE 02155_test_table;
|
||||
DROP DICTIONARY 02155_test_dictionary;
|
||||
DROP TABLE 02155_test_table;
|
||||
|
@ -170,8 +170,8 @@ LIFETIME(0);
|
||||
SELECT 'Polygon dictionary';
|
||||
SELECT * FROM 02183_polygon_dictionary;
|
||||
|
||||
DROP TABLE 02183_polygon_dictionary_source_table;
|
||||
DROP DICTIONARY 02183_polygon_dictionary;
|
||||
DROP TABLE 02183_polygon_dictionary_source_table;
|
||||
|
||||
DROP TABLE IF EXISTS 02183_range_dictionary_source_table;
|
||||
CREATE TABLE 02183_range_dictionary_source_table
|
||||
|
@ -60,4 +60,5 @@ SELECT dictHas('02185_range_dictionary', 0, 0);
|
||||
SELECT dictHas('02185_range_dictionary', 0, 5001);
|
||||
SELECT dictHas('02185_range_dictionary', 0, 10001);
|
||||
|
||||
DROP DICTIONARY 02185_range_dictionary;
|
||||
DROP TABLE 02185_range_dictionary_source_table;
|
||||
|
@ -10,9 +10,5 @@ DROP TABLE test;
|
||||
DROP TABLE IF EXISTS test1;
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test1 (key UInt32) Engine = Buffer(currentDatabase(), test2, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
|
||||
CREATE TABLE test2 (key UInt32) Engine = Buffer(currentDatabase(), test1, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
|
||||
SELECT * FROM test1; -- { serverError TOO_DEEP_RECURSION }
|
||||
SELECT * FROM test2; -- { serverError TOO_DEEP_RECURSION }
|
||||
SELECT * FROM system.tables WHERE table IN ('test1', 'test2') AND database = currentDatabase(); -- { serverError TOO_DEEP_RECURSION }
|
||||
CREATE TABLE test2 (key UInt32) Engine = Buffer(currentDatabase(), test1, 16, 10, 100, 10000, 1000000, 10000000, 100000000); -- { serverError INFINITE_LOOP }
|
||||
DROP TABLE test1;
|
||||
DROP TABLE test2;
|
||||
|
@ -27,5 +27,5 @@ LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10));
|
||||
SELECT dictGetOrDefault('cache_dictionary_complex_key_simple_attributes_short_circuit', 'value_first', (number, concat(toString(number))), toString(materialize('default'))) AS value_first FROM system.numbers LIMIT 20 FORMAT Null;
|
||||
SELECT dictGetOrDefault('cache_dictionary_complex_key_simple_attributes_short_circuit', 'value_first', (number, concat(toString(number))), toString(materialize('default'))) AS value_first FROM system.numbers LIMIT 20 FORMAT Null;
|
||||
|
||||
DROP TABLE IF EXISTS complex_key_simple_attributes_source_short_circuit_table;
|
||||
DROP DICTIONARY IF EXISTS cache_dictionary_complex_key_simple_attributes_short_circuit;
|
||||
DROP TABLE IF EXISTS complex_key_simple_attributes_source_short_circuit_table;
|
||||
|
@ -0,0 +1,77 @@
|
||||
-- Tags: atomic-database
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (id UInt64, value String) ENGINE=MergeTree ORDER BY id;
|
||||
INSERT INTO test SELECT number, 'str_' || toString(number) FROM numbers(10);
|
||||
DROP DICTIONARY IF EXISTS test_dict;
|
||||
CREATE DICTIONARY test_dict
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE test))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
DROP TABLE IF EXISTS view_source;
|
||||
CREATE TABLE view_source (id UInt64) ENGINE=MergeTree ORDER BY id;
|
||||
INSERT INTO view_source SELECT * FROM numbers(5);
|
||||
DROP VIEW IF EXISTS view;
|
||||
CREATE VIEW view AS SELECT id, dictGet('test_dict', 'value', id) as value FROM view_source;
|
||||
|
||||
CREATE OR REPLACE DICTIONARY test_dict
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE view))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 0 MAX 1000); -- {serverError INFINITE_LOOP}
|
||||
|
||||
REPLACE DICTIONARY test_dict
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE view))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 0 MAX 1000); -- {serverError INFINITE_LOOP}
|
||||
|
||||
|
||||
DROP DICTIONARY IF EXISTS test_dict_2;
|
||||
CREATE DICTIONARY test_dict_2
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE view))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
|
||||
EXCHANGE DICTIONARIES test_dict AND test_dict_2; -- {serverError INFINITE_LOOP}
|
||||
|
||||
DROP DICTIONARY test_dict_2;
|
||||
|
||||
CREATE OR REPLACE DICTIONARY test_dict_2
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE view))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
|
||||
EXCHANGE DICTIONARIES test_dict AND test_dict_2; -- {serverError INFINITE_LOOP}
|
||||
|
||||
DROP DICTIONARY test_dict;
|
||||
RENAME DICTIONARY test_dict_2 to test_dict; -- {serverError INFINITE_LOOP}
|
||||
|
||||
DROP DICTIONARY test_dict_2;
|
||||
DROP VIEW view;
|
||||
DROP TABLE test;
|
||||
DROP TABLE view_source;
|
||||
|
Loading…
Reference in New Issue
Block a user