2021-08-26 13:19:52 +00:00
|
|
|
#include <Databases/DDLDependencyVisitor.h>
|
2021-09-02 13:34:46 +00:00
|
|
|
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
2023-05-07 22:57:13 +00:00
|
|
|
#include <Databases/removeWhereConditionPlaceholder.h>
|
2022-12-10 23:43:42 +00:00
|
|
|
#include <Interpreters/Cluster.h>
|
2021-11-01 18:53:07 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2022-12-23 19:57:19 +00:00
|
|
|
#include <Interpreters/misc.h>
|
2022-12-10 22:57:14 +00:00
|
|
|
#include <Interpreters/InDepthNodeVisitor.h>
|
2022-12-10 23:36:19 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
2022-12-10 23:43:42 +00:00
|
|
|
#include <Interpreters/getClusterName.h>
|
2021-11-26 18:27:16 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
2022-12-02 14:05:46 +00:00
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2023-05-05 14:27:12 +00:00
|
|
|
#include <Parsers/ParserSelectWithUnionQuery.h>
|
|
|
|
#include <Parsers/parseQuery.h>
|
2022-12-10 23:43:42 +00:00
|
|
|
#include <Common/KnownObjectNames.h>
|
2021-09-02 13:48:41 +00:00
|
|
|
#include <Poco/String.h>
|
2021-08-26 13:19:52 +00:00
|
|
|
|
|
|
|
|
2022-12-02 14:05:46 +00:00
|
|
|
namespace DB
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
|
|
|
|
2022-12-02 14:05:46 +00:00
|
|
|
namespace
|
2021-08-26 13:19:52 +00:00
|
|
|
{
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Data for DDLDependencyVisitor.
|
|
|
|
/// Used to visits ASTCreateQuery and extracts the names of all tables explicitly referenced in the create query.
|
|
|
|
class DDLDependencyVisitorData
|
2021-09-24 10:22:22 +00:00
|
|
|
{
|
2023-05-05 14:27:12 +00:00
|
|
|
friend void tryVisitNestedSelect(const String & query, DDLDependencyVisitorData & data);
|
2022-12-10 22:57:14 +00:00
|
|
|
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_)
|
2022-12-02 14:05:46 +00:00
|
|
|
{
|
|
|
|
}
|
2021-09-24 10:22:22 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Acquire the result of visiting the create query.
|
|
|
|
TableNamesSet getDependencies() &&
|
2022-12-02 14:05:46 +00:00
|
|
|
{
|
2022-12-10 22:57:14 +00:00
|
|
|
dependencies.erase(table_name);
|
|
|
|
return std::move(dependencies);
|
2022-12-02 14:05:46 +00:00
|
|
|
}
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
bool needChildVisit(const ASTPtr & child) const { return !skip_asts.contains(child.get()); }
|
2021-09-02 13:48:41 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
void visit(const ASTPtr & ast)
|
|
|
|
{
|
|
|
|
if (auto * create = ast->as<ASTCreateQuery>())
|
|
|
|
{
|
|
|
|
visitCreateQuery(*create);
|
|
|
|
}
|
|
|
|
else if (auto * dictionary = ast->as<ASTDictionary>())
|
|
|
|
{
|
|
|
|
visitDictionaryDef(*dictionary);
|
|
|
|
}
|
|
|
|
else if (auto * expr = ast->as<ASTTableExpression>())
|
|
|
|
{
|
|
|
|
visitTableExpression(*expr);
|
|
|
|
}
|
|
|
|
else if (const auto * function = ast->as<ASTFunction>())
|
|
|
|
{
|
|
|
|
if (function->kind == ASTFunction::Kind::TABLE_ENGINE)
|
|
|
|
visitTableEngine(*function);
|
|
|
|
else
|
|
|
|
visitFunction(*function);
|
|
|
|
}
|
|
|
|
}
|
2022-10-05 18:58:27 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
private:
|
|
|
|
ASTPtr create_query;
|
|
|
|
std::unordered_set<const IAST *> skip_asts;
|
|
|
|
QualifiedTableName table_name;
|
|
|
|
String current_database;
|
|
|
|
ContextPtr context;
|
|
|
|
TableNamesSet dependencies;
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// CREATE TABLE or CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query.
|
|
|
|
void visitCreateQuery(const ASTCreateQuery & create)
|
2022-12-02 14:05:46 +00:00
|
|
|
{
|
2022-12-10 22:57:14 +00:00
|
|
|
QualifiedTableName to_table{create.to_table_id.database_name, create.to_table_id.table_name};
|
|
|
|
if (!to_table.table.empty())
|
|
|
|
{
|
|
|
|
/// TO target_table (for materialized views)
|
|
|
|
if (to_table.database.empty())
|
|
|
|
to_table.database = current_database;
|
|
|
|
dependencies.emplace(to_table);
|
|
|
|
}
|
|
|
|
|
|
|
|
QualifiedTableName as_table{create.as_database, create.as_table};
|
|
|
|
if (!as_table.table.empty())
|
|
|
|
{
|
|
|
|
/// AS table_name
|
|
|
|
if (as_table.database.empty())
|
|
|
|
as_table.database = current_database;
|
|
|
|
dependencies.emplace(as_table);
|
|
|
|
}
|
2022-12-02 14:05:46 +00:00
|
|
|
}
|
2021-09-02 13:34:46 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// The definition of a dictionary: SOURCE(CLICKHOUSE(...)) LAYOUT(...) LIFETIME(...)
|
|
|
|
void visitDictionaryDef(const ASTDictionary & dictionary)
|
|
|
|
{
|
|
|
|
if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements)
|
|
|
|
return;
|
2021-09-13 19:11:16 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
auto config = getDictionaryConfigurationFromAST(create_query->as<ASTCreateQuery &>(), context);
|
|
|
|
auto info = getInfoIfClickHouseDictionarySource(config, context);
|
2021-09-02 13:34:46 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// We consider only dependencies on local tables.
|
|
|
|
if (!info || !info->is_local)
|
|
|
|
return;
|
2021-09-02 13:34:46 +00:00
|
|
|
|
2023-05-05 14:27:12 +00:00
|
|
|
if (!info->table_name.table.empty())
|
|
|
|
{
|
|
|
|
if (info->table_name.database.empty())
|
|
|
|
info->table_name.database = current_database;
|
|
|
|
dependencies.emplace(std::move(info->table_name));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// We don't have a table name, we have a select query instead
|
|
|
|
tryVisitNestedSelect(info->query, *this);
|
|
|
|
}
|
2022-12-10 22:57:14 +00:00
|
|
|
}
|
2021-10-06 12:39:14 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// ASTTableExpression represents a reference to a table in SELECT query.
|
|
|
|
/// DDLDependencyVisitor should handle ASTTableExpression because some CREATE queries can contain SELECT queries after AS
|
|
|
|
/// (for example, CREATE VIEW).
|
|
|
|
void visitTableExpression(const ASTTableExpression & expr)
|
2022-12-02 14:05:46 +00:00
|
|
|
{
|
2022-12-10 22:57:14 +00:00
|
|
|
if (!expr.database_and_table_name)
|
2022-12-02 14:05:46 +00:00
|
|
|
return;
|
2021-10-06 12:39:14 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
const ASTIdentifier * identifier = dynamic_cast<const ASTIdentifier *>(expr.database_and_table_name.get());
|
|
|
|
if (!identifier)
|
2022-12-02 14:05:46 +00:00
|
|
|
return;
|
2021-09-02 13:34:46 +00:00
|
|
|
|
2022-12-02 14:05:46 +00:00
|
|
|
auto table_identifier = identifier->createTable();
|
|
|
|
if (!table_identifier)
|
|
|
|
return;
|
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
QualifiedTableName qualified_name{table_identifier->getDatabaseName(), table_identifier->shortName()};
|
|
|
|
if (qualified_name.table.empty())
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (qualified_name.database.empty())
|
|
|
|
{
|
|
|
|
/// It can be table/dictionary from default database or XML dictionary, but we cannot distinguish it here.
|
|
|
|
qualified_name.database = current_database;
|
|
|
|
}
|
|
|
|
|
|
|
|
dependencies.emplace(qualified_name);
|
2022-12-02 14:05:46 +00:00
|
|
|
}
|
2022-12-10 22:57:14 +00:00
|
|
|
|
|
|
|
/// Finds dependencies of a table engine.
|
|
|
|
void visitTableEngine(const ASTFunction & table_engine)
|
2022-12-02 14:05:46 +00:00
|
|
|
{
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Dictionary(db_name.dictionary_name)
|
|
|
|
if (table_engine.name == "Dictionary")
|
|
|
|
addQualifiedNameFromArgument(table_engine, 0);
|
|
|
|
|
|
|
|
/// Buffer('db_name', 'dest_table_name')
|
|
|
|
if (table_engine.name == "Buffer")
|
|
|
|
addDatabaseAndTableNameFromArguments(table_engine, 0, 1);
|
2022-12-10 23:43:42 +00:00
|
|
|
|
|
|
|
/// Distributed(cluster_name, db_name, table_name, ...)
|
|
|
|
if (table_engine.name == "Distributed")
|
|
|
|
visitDistributedTableEngine(table_engine);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Distributed(cluster_name, database_name, table_name, ...)
|
|
|
|
void visitDistributedTableEngine(const ASTFunction & table_engine)
|
|
|
|
{
|
|
|
|
/// We consider only dependencies on local tables.
|
|
|
|
bool has_local_replicas = false;
|
|
|
|
|
|
|
|
if (auto cluster_name = tryGetClusterNameFromArgument(table_engine, 0))
|
|
|
|
{
|
|
|
|
auto cluster = context->tryGetCluster(*cluster_name);
|
|
|
|
if (cluster && cluster->getLocalShardCount())
|
|
|
|
has_local_replicas = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (has_local_replicas)
|
|
|
|
addDatabaseAndTableNameFromArguments(table_engine, 1, 2);
|
2022-12-02 14:05:46 +00:00
|
|
|
}
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Finds dependencies of a function.
|
|
|
|
void visitFunction(const ASTFunction & function)
|
2022-12-02 14:05:46 +00:00
|
|
|
{
|
2022-12-23 19:57:19 +00:00
|
|
|
if (functionIsJoinGet(function.name) || functionIsDictGet(function.name))
|
2022-12-10 22:57:14 +00:00
|
|
|
{
|
|
|
|
/// dictGet('dict_name', attr_names, id_expr)
|
|
|
|
/// dictHas('dict_name', id_expr)
|
|
|
|
/// joinGet(join_storage_table_name, `value_column`, join_keys)
|
|
|
|
addQualifiedNameFromArgument(function, 0);
|
|
|
|
}
|
2023-02-07 01:25:58 +00:00
|
|
|
else if (functionIsInOrGlobalInOperator(function.name))
|
2022-12-10 22:57:14 +00:00
|
|
|
{
|
2023-02-07 01:25:58 +00:00
|
|
|
/// x IN table_name.
|
|
|
|
/// We set evaluate=false here because we don't want to evaluate a subquery in "x IN subquery".
|
|
|
|
addQualifiedNameFromArgument(function, 1, /* evaluate= */ false);
|
2022-12-10 22:57:14 +00:00
|
|
|
}
|
|
|
|
else if (function.name == "dictionary")
|
|
|
|
{
|
|
|
|
/// dictionary(dict_name)
|
|
|
|
addQualifiedNameFromArgument(function, 0);
|
|
|
|
}
|
2022-12-10 23:43:42 +00:00
|
|
|
else if (function.name == "remote" || function.name == "remoteSecure")
|
|
|
|
{
|
|
|
|
visitRemoteFunction(function, /* is_cluster_function= */ false);
|
|
|
|
}
|
|
|
|
else if (function.name == "cluster" || function.name == "clusterAllReplicas")
|
|
|
|
{
|
|
|
|
visitRemoteFunction(function, /* is_cluster_function= */ true);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/// remote('addresses_expr', db_name.table_name, ...)
|
|
|
|
/// remote('addresses_expr', 'db_name', 'table_name', ...)
|
|
|
|
/// remote('addresses_expr', table_function(), ...)
|
|
|
|
/// cluster('cluster_name', db_name.table_name, ...)
|
|
|
|
/// cluster('cluster_name', 'db_name', 'table_name', ...)
|
|
|
|
/// cluster('cluster_name', table_function(), ...)
|
|
|
|
void visitRemoteFunction(const ASTFunction & function, bool is_cluster_function)
|
|
|
|
{
|
|
|
|
/// We consider dependencies on local tables only.
|
|
|
|
bool has_local_replicas = false;
|
|
|
|
|
|
|
|
if (is_cluster_function)
|
|
|
|
{
|
|
|
|
if (auto cluster_name = tryGetClusterNameFromArgument(function, 0))
|
|
|
|
{
|
|
|
|
if (auto cluster = context->tryGetCluster(*cluster_name))
|
|
|
|
{
|
|
|
|
if (cluster->getLocalShardCount())
|
|
|
|
has_local_replicas = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// remote() and remoteSecure() are not fully supported. To properly support them we would need to check the first
|
|
|
|
/// argument to decide whether the host & port pattern specified in the first argument contains the local host or not
|
|
|
|
/// which is not trivial. For now we just always assume that the host & port pattern doesn't contain the local host.
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!function.arguments)
|
|
|
|
return;
|
|
|
|
|
|
|
|
ASTs & args = function.arguments->children;
|
|
|
|
if (args.size() < 2)
|
|
|
|
return;
|
|
|
|
|
|
|
|
const ASTFunction * table_function = nullptr;
|
|
|
|
if (const auto * second_arg_as_function = args[1]->as<ASTFunction>();
|
|
|
|
second_arg_as_function && KnownTableFunctionNames::instance().exists(second_arg_as_function->name))
|
|
|
|
{
|
|
|
|
table_function = second_arg_as_function;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (has_local_replicas && !table_function)
|
|
|
|
{
|
2023-02-07 01:25:58 +00:00
|
|
|
/// We set `apply_current_database=false` here because if this argument is an identifier without dot,
|
|
|
|
/// then it's not the name of a table within the current database, it's the name of a database, and
|
|
|
|
/// the name of a table will be in the following argument.
|
|
|
|
auto maybe_qualified_name = tryGetQualifiedNameFromArgument(function, 1, /* evaluate= */ true, /* apply_current_database= */ false);
|
2022-12-10 23:43:42 +00:00
|
|
|
if (!maybe_qualified_name)
|
|
|
|
return;
|
|
|
|
auto & qualified_name = *maybe_qualified_name;
|
|
|
|
if (qualified_name.database.empty())
|
|
|
|
{
|
|
|
|
auto table = tryGetStringFromArgument(function, 2);
|
|
|
|
if (!table)
|
|
|
|
return;
|
|
|
|
qualified_name.database = std::move(qualified_name.table);
|
|
|
|
qualified_name.table = std::move(table).value();
|
|
|
|
}
|
|
|
|
dependencies.insert(qualified_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!has_local_replicas && table_function)
|
|
|
|
{
|
|
|
|
/// `table function` will be executed remotely, so we won't check it or its arguments for dependencies.
|
|
|
|
skip_asts.emplace(table_function);
|
|
|
|
}
|
2022-12-02 14:05:46 +00:00
|
|
|
}
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Gets an argument as a string, evaluates constants if necessary.
|
2023-02-07 01:25:58 +00:00
|
|
|
std::optional<String> tryGetStringFromArgument(const ASTFunction & function, size_t arg_idx, bool evaluate = true) const
|
2022-12-10 22:57:14 +00:00
|
|
|
{
|
|
|
|
if (!function.arguments)
|
|
|
|
return {};
|
|
|
|
|
|
|
|
const ASTs & args = function.arguments->children;
|
|
|
|
if (arg_idx >= args.size())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
const auto & arg = args[arg_idx];
|
|
|
|
|
2023-02-07 01:25:58 +00:00
|
|
|
if (evaluate)
|
2022-12-10 22:57:14 +00:00
|
|
|
{
|
2023-02-07 01:25:58 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
/// We're just searching for dependencies here, it's not safe to execute subqueries now.
|
|
|
|
auto evaluated = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
|
|
|
const auto * literal = evaluated->as<ASTLiteral>();
|
|
|
|
if (!literal || (literal->value.getType() != Field::Types::String))
|
|
|
|
return {};
|
|
|
|
return literal->value.safeGet<String>();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
2022-12-10 22:57:14 +00:00
|
|
|
}
|
2023-02-07 01:25:58 +00:00
|
|
|
else
|
2022-12-10 22:57:14 +00:00
|
|
|
{
|
2023-02-07 01:25:58 +00:00
|
|
|
if (const auto * id = arg->as<ASTIdentifier>())
|
|
|
|
return id->name();
|
|
|
|
if (const auto * literal = arg->as<ASTLiteral>())
|
|
|
|
{
|
|
|
|
if (literal->value.getType() == Field::Types::String)
|
|
|
|
return literal->value.safeGet<String>();
|
|
|
|
}
|
2022-12-10 23:36:19 +00:00
|
|
|
return {};
|
2022-12-10 22:57:14 +00:00
|
|
|
}
|
|
|
|
}
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Gets an argument as a qualified table name.
|
|
|
|
/// Accepts forms db_name.table_name (as an identifier) and 'db_name.table_name' (as a string).
|
|
|
|
/// The function doesn't replace an empty database name with the current_database (the caller must do that).
|
2023-02-07 01:25:58 +00:00
|
|
|
std::optional<QualifiedTableName> tryGetQualifiedNameFromArgument(
|
|
|
|
const ASTFunction & function, size_t arg_idx, bool evaluate = true, bool apply_current_database = true) const
|
2022-12-10 22:57:14 +00:00
|
|
|
{
|
|
|
|
if (!function.arguments)
|
|
|
|
return {};
|
|
|
|
|
|
|
|
const ASTs & args = function.arguments->children;
|
|
|
|
if (arg_idx >= args.size())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
const auto & arg = args[arg_idx];
|
|
|
|
QualifiedTableName qualified_name;
|
|
|
|
|
|
|
|
if (const auto * identifier = dynamic_cast<const ASTIdentifier *>(arg.get()))
|
|
|
|
{
|
|
|
|
/// ASTIdentifier or ASTTableIdentifier
|
|
|
|
auto table_identifier = identifier->createTable();
|
|
|
|
if (!table_identifier)
|
|
|
|
return {};
|
|
|
|
|
|
|
|
qualified_name.database = table_identifier->getDatabaseName();
|
|
|
|
qualified_name.table = table_identifier->shortName();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2023-02-07 01:25:58 +00:00
|
|
|
auto qualified_name_as_string = tryGetStringFromArgument(function, arg_idx, evaluate);
|
2022-12-10 22:57:14 +00:00
|
|
|
if (!qualified_name_as_string)
|
|
|
|
return {};
|
|
|
|
|
|
|
|
auto maybe_qualified_name = QualifiedTableName::tryParseFromString(*qualified_name_as_string);
|
|
|
|
if (!maybe_qualified_name)
|
|
|
|
return {};
|
|
|
|
|
|
|
|
qualified_name = std::move(maybe_qualified_name).value();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (qualified_name.database.empty() && apply_current_database)
|
|
|
|
qualified_name.database = current_database;
|
|
|
|
|
|
|
|
return qualified_name;
|
|
|
|
}
|
2021-09-13 19:11:16 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Adds a qualified table name from an argument to the collection of dependencies.
|
|
|
|
/// Accepts forms db_name.table_name (as an identifier) and 'db_name.table_name' (as a string).
|
2023-02-07 01:25:58 +00:00
|
|
|
void addQualifiedNameFromArgument(const ASTFunction & function, size_t arg_idx, bool evaluate = true)
|
2022-12-10 22:57:14 +00:00
|
|
|
{
|
2023-02-07 01:25:58 +00:00
|
|
|
if (auto qualified_name = tryGetQualifiedNameFromArgument(function, arg_idx, evaluate))
|
2022-12-10 22:57:14 +00:00
|
|
|
dependencies.emplace(std::move(qualified_name).value());
|
|
|
|
}
|
2022-12-02 14:05:46 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Returns a database name and a table name extracted from two separate arguments.
|
|
|
|
std::optional<QualifiedTableName> tryGetDatabaseAndTableNameFromArguments(
|
|
|
|
const ASTFunction & function, size_t database_arg_idx, size_t table_arg_idx, bool apply_current_database = true) const
|
|
|
|
{
|
|
|
|
auto database = tryGetStringFromArgument(function, database_arg_idx);
|
|
|
|
if (!database)
|
|
|
|
return {};
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
auto table = tryGetStringFromArgument(function, table_arg_idx);
|
2023-02-02 14:45:16 +00:00
|
|
|
if (!table || table->empty())
|
2022-12-10 22:57:14 +00:00
|
|
|
return {};
|
2022-12-02 14:05:46 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
QualifiedTableName qualified_name;
|
|
|
|
qualified_name.database = std::move(database).value();
|
|
|
|
qualified_name.table = std::move(table).value();
|
2022-12-02 14:05:46 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
if (qualified_name.database.empty() && apply_current_database)
|
|
|
|
qualified_name.database = current_database;
|
2022-12-02 14:05:46 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
return qualified_name;
|
2022-12-02 14:05:46 +00:00
|
|
|
}
|
2022-12-10 22:57:14 +00:00
|
|
|
|
|
|
|
/// Adds a database name and a table name from two separate arguments to the collection of dependencies.
|
|
|
|
void addDatabaseAndTableNameFromArguments(const ASTFunction & function, size_t database_arg_idx, size_t table_arg_idx)
|
2022-12-02 14:05:46 +00:00
|
|
|
{
|
2022-12-10 22:57:14 +00:00
|
|
|
if (auto qualified_name = tryGetDatabaseAndTableNameFromArguments(function, database_arg_idx, table_arg_idx))
|
|
|
|
dependencies.emplace(std::move(qualified_name).value());
|
2022-12-02 14:05:46 +00:00
|
|
|
}
|
2022-12-10 23:43:42 +00:00
|
|
|
|
|
|
|
std::optional<String> tryGetClusterNameFromArgument(const ASTFunction & function, size_t arg_idx) const
|
|
|
|
{
|
|
|
|
if (!function.arguments)
|
|
|
|
return {};
|
|
|
|
|
|
|
|
ASTs & args = function.arguments->children;
|
|
|
|
if (arg_idx >= args.size())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
auto cluster_name = ::DB::tryGetClusterName(*args[arg_idx]);
|
|
|
|
if (cluster_name)
|
|
|
|
return cluster_name;
|
|
|
|
|
|
|
|
return tryGetStringFromArgument(function, arg_idx);
|
|
|
|
}
|
2022-12-10 22:57:14 +00:00
|
|
|
};
|
2021-08-26 13:19:52 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
/// Visits ASTCreateQuery and extracts the names of all tables explicitly referenced in the create query.
|
|
|
|
class DDLDependencyVisitor
|
2021-12-06 13:35:34 +00:00
|
|
|
{
|
2022-12-10 22:57:14 +00:00
|
|
|
public:
|
|
|
|
using Data = DDLDependencyVisitorData;
|
|
|
|
using Visitor = ConstInDepthNodeVisitor<DDLDependencyVisitor, /* top_to_bottom= */ true, /* need_child_accept_data= */ true>;
|
2022-10-05 18:58:27 +00:00
|
|
|
|
2022-12-10 22:57:14 +00:00
|
|
|
static bool needChildVisit(const ASTPtr &, const ASTPtr & child, const Data & data) { return data.needChildVisit(child); }
|
|
|
|
static void visit(const ASTPtr & ast, Data & data) { data.visit(ast); }
|
|
|
|
};
|
2023-05-05 14:27:12 +00:00
|
|
|
|
|
|
|
void tryVisitNestedSelect(const String & query, DDLDependencyVisitorData & data)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
ParserSelectWithUnionQuery parser;
|
|
|
|
String description = fmt::format("Query for ClickHouse dictionary {}", data.table_name);
|
2023-05-05 18:50:13 +00:00
|
|
|
String fixed_query = removeWhereConditionPlaceholder(query);
|
|
|
|
ASTPtr select = parseQuery(parser, fixed_query, description,
|
2023-05-05 14:27:12 +00:00
|
|
|
data.context->getSettingsRef().max_query_size, data.context->getSettingsRef().max_parser_depth);
|
|
|
|
|
|
|
|
DDLDependencyVisitor::Visitor visitor{data};
|
|
|
|
visitor.visit(select);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException("DDLDependencyVisitor");
|
|
|
|
}
|
|
|
|
}
|
2022-10-05 18:58:27 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-12-04 11:33:06 +00:00
|
|
|
TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & context, const QualifiedTableName & table_name, const ASTPtr & ast)
|
2022-10-05 18:58:27 +00:00
|
|
|
{
|
2022-12-10 22:57:14 +00:00
|
|
|
DDLDependencyVisitor::Data data{context, table_name, ast};
|
2022-12-02 14:05:46 +00:00
|
|
|
DDLDependencyVisitor::Visitor visitor{data};
|
|
|
|
visitor.visit(ast);
|
2022-12-10 22:57:14 +00:00
|
|
|
return std::move(data).getDependencies();
|
2022-10-05 18:58:27 +00:00
|
|
|
}
|
|
|
|
|
2021-08-26 13:19:52 +00:00
|
|
|
}
|