Merge pull request #4056 from 4ertus2/ast

Helpers for ASTIdentifier (based on 'QueryNormalizer refactoring' changes)
This commit is contained in:
alexey-milovidov 2019-01-15 00:22:54 +03:00 committed by GitHub
commit 8a251753be
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
49 changed files with 253 additions and 226 deletions

View File

@ -317,9 +317,9 @@ static DataTypePtr create(const ASTPtr & arguments)
params_row[i] = lit->value;
}
}
else if (const ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(arguments->children[0].get()))
else if (auto opt_name = getIdentifierName(arguments->children[0]))
{
function_name = identifier->name;
function_name = *opt_name;
}
else if (typeid_cast<ASTLiteral *>(arguments->children[0].get()))
{

View File

@ -7,6 +7,7 @@
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabaseMemory.h>
#include <Databases/DatabasesCommon.h>
#include <Common/typeid_cast.h>
#include <Common/escapeForFileName.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Stopwatch.h>

View File

@ -1,5 +1,6 @@
#include <sstream>
#include <Common/typeid_cast.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>

View File

@ -7,7 +7,6 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Storages/IStorage.h>
@ -118,8 +117,7 @@ NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpressio
}
else if (table_expression.database_and_table_name)
{
const auto & identifier = static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name);
DatabaseAndTableWithAlias database_table(identifier);
DatabaseAndTableWithAlias database_table(table_expression.database_and_table_name);
const auto & table = context.getTable(database_table.database, database_table.table);
names_and_type_list = table->getSampleBlockNonMaterialized().getNamesAndTypesList();
}

View File

@ -58,7 +58,7 @@ private:
NameToNameMap & array_join_alias_to_name = data.array_join_alias_to_name;
NameToNameMap & array_join_result_to_source = data.array_join_result_to_source;
if (!node.general())
if (!getColumnIdentifierName(node))
return;
auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1

View File

@ -31,7 +31,7 @@ bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast, bool is_public)
void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node, bool is_public)
{
if (!node.general())
if (!getColumnIdentifierName(node))
return;
required_names.insert(node.name);

View File

@ -39,9 +39,7 @@ struct ColumnNamesContext
std::optional<String> name() const
{
if (expr)
if (auto * node = expr->database_and_table_name.get())
if (auto * identifier = typeid_cast<const ASTIdentifier *>(node))
return identifier->name;
return getIdentifierName(expr->database_and_table_name);
return {};
}

View File

@ -54,8 +54,6 @@ size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifie
{
size_t num_qualifiers_to_strip = 0;
auto get_identifier_name = [](const ASTPtr & ast) { return static_cast<const ASTIdentifier &>(*ast).name; };
/// It is compound identifier
if (!identifier.children.empty())
{
@ -64,16 +62,16 @@ size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifie
/// database.table.column
if (num_components >= 3
&& !names.database.empty()
&& get_identifier_name(identifier.children[0]) == names.database
&& get_identifier_name(identifier.children[1]) == names.table)
&& *getIdentifierName(identifier.children[0]) == names.database
&& *getIdentifierName(identifier.children[1]) == names.table)
{
num_qualifiers_to_strip = 2;
}
/// table.column or alias.column. If num_components > 2, it is like table.nested.column.
if (num_components >= 2
&& ((!names.table.empty() && get_identifier_name(identifier.children[0]) == names.table)
|| (!names.alias.empty() && get_identifier_name(identifier.children[0]) == names.alias)))
&& ((!names.table.empty() && *getIdentifierName(identifier.children[0]) == names.table)
|| (!names.alias.empty() && *getIdentifierName(identifier.children[0]) == names.alias)))
{
num_qualifiers_to_strip = 1;
}
@ -94,26 +92,24 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident
if (identifier.children.size() != 2)
throw Exception("Logical error: number of components in table expression not equal to two", ErrorCodes::LOGICAL_ERROR);
const ASTIdentifier * db_identifier = typeid_cast<const ASTIdentifier *>(identifier.children[0].get());
const ASTIdentifier * table_identifier = typeid_cast<const ASTIdentifier *>(identifier.children[1].get());
if (!db_identifier || !table_identifier)
throw Exception("Logical error: identifiers expected", ErrorCodes::LOGICAL_ERROR);
database = db_identifier->name;
table = table_identifier->name;
getIdentifierName(identifier.children[0], database);
getIdentifierName(identifier.children[1], table);
}
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database)
{
const auto * identifier = typeid_cast<const ASTIdentifier *>(node.get());
if (!identifier)
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
*this = DatabaseAndTableWithAlias(*identifier, current_database);
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database)
{
if (table_expression.database_and_table_name)
{
const auto * identifier = typeid_cast<const ASTIdentifier *>(table_expression.database_and_table_name.get());
if (!identifier)
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
*this = DatabaseAndTableWithAlias(*identifier, current_database);
}
*this = DatabaseAndTableWithAlias(table_expression.database_and_table_name, current_database);
else if (table_expression.table_function)
alias = table_expression.table_function->tryGetAlias();
else if (table_expression.subquery)
@ -207,14 +203,10 @@ std::optional<DatabaseAndTableWithAlias> getDatabaseAndTable(const ASTSelectQuer
return {};
ASTPtr database_and_table_name = table_expression->database_and_table_name;
if (!database_and_table_name)
if (!database_and_table_name || !isIdentifier(database_and_table_name))
return {};
const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(database_and_table_name.get());
if (!identifier)
return {};
return *identifier;
return DatabaseAndTableWithAlias(database_and_table_name);
}
ASTPtr getTableFunctionOrSubquery(const ASTSelectQuery & select, size_t table_number)

View File

@ -24,6 +24,7 @@ struct DatabaseAndTableWithAlias
String alias;
DatabaseAndTableWithAlias() = default;
DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database);

View File

@ -548,8 +548,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
/// TODO This syntax does not support specifying a database name.
if (table_to_join.database_and_table_name)
{
const auto & identifier = static_cast<const ASTIdentifier &>(*table_to_join.database_and_table_name);
DatabaseAndTableWithAlias database_table(identifier);
DatabaseAndTableWithAlias database_table(table_to_join.database_and_table_name);
StoragePtr table = context.tryGetTable(database_table.database, database_table.table);
if (table)

View File

@ -33,9 +33,9 @@ public:
private:
static std::vector<ASTPtr *> visit(const ASTIdentifier & node, ASTPtr &, Data & data)
{
if (node.special())
if (StoragePtr external_storage = data.context.tryGetExternalTable(node.name))
data.external_tables[node.name] = external_storage;
if (auto opt_name = getTableIdentifierName(node))
if (StoragePtr external_storage = data.context.tryGetExternalTable(*opt_name))
data.external_tables[*opt_name] = external_storage;
return {};
}
};

View File

@ -5,7 +5,6 @@
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
@ -82,12 +81,7 @@ void forEachTable(IAST * node, F && f)
StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & context)
{
const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(database_and_table.get());
if (!id)
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
DatabaseAndTableWithAlias db_and_table(*id);
DatabaseAndTableWithAlias db_and_table(database_and_table);
return context.tryGetTable(db_and_table.database, db_and_table.table);
}

View File

@ -95,13 +95,11 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
auto database_ptr = identifier->children[0];
auto table_ptr = identifier->children[1];
if (database_ptr)
database_name = typeid_cast<ASTIdentifier &>(*database_ptr).name;
if (table_ptr)
table_name = typeid_cast<ASTIdentifier &>(*table_ptr).name;
getIdentifierName(database_ptr, database_name);
getIdentifierName(table_ptr, table_name);
}
else
table_name = typeid_cast<ASTIdentifier &>(*identifier).name;
getIdentifierName(identifier, table_name);
table = context.getTable(database_name, table_name);
}

View File

@ -83,8 +83,8 @@ static void appendTableNameAndAlias(std::vector<String> & hidden, const ASTPtr &
if (!alias.empty())
hidden.push_back(alias);
if (auto * identifier = typeid_cast<const ASTIdentifier *>(table_expression->children[0].get()))
hidden.push_back(identifier->name);
if (auto opt_name = getIdentifierName(table_expression->children[0]))
hidden.push_back(*opt_name);
else if (alias.empty())
throw Exception("Expected Identifier or subquery with alias", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -68,9 +68,12 @@ void QueryNormalizer::visit(ASTFunction & node, const ASTPtr &, Data & data)
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
if (functionIsInOrGlobalInOperator(func_name))
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(func_arguments->children.at(1).get()))
if (!aliases.count(right->name))
right->setSpecial();
{
auto & ast = func_arguments->children.at(1);
if (auto opt_name = getIdentifierName(ast))
if (!aliases.count(*opt_name))
setIdentifierSpecial(ast);
}
/// Special cases for count function.
String func_name_lowercase = Poco::toLower(func_name);
@ -97,7 +100,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
auto & current_asts = data.current_asts;
String & current_alias = data.current_alias;
if (!node.general())
if (!getColumnIdentifierName(node))
return;
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
@ -114,9 +117,8 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
if (!my_alias.empty() && my_alias != alias_node->getAliasOrColumnName())
{
/// Avoid infinite recursion here
auto replace_to_identifier = typeid_cast<ASTIdentifier *>(alias_node.get());
bool is_cycle = replace_to_identifier && replace_to_identifier->general()
&& replace_to_identifier->name == node.name;
auto opt_name = getColumnIdentifierName(alias_node);
bool is_cycle = opt_name && *opt_name == node.name;
if (!is_cycle)
{
@ -195,10 +197,8 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &
{
if (node.table_expression)
{
auto & database_and_table_name = static_cast<ASTTableExpression &>(*node.table_expression).database_and_table_name;
if (database_and_table_name)
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(database_and_table_name.get()))
right->setSpecial();
auto & expr = static_cast<ASTTableExpression &>(*node.table_expression);
setIdentifierSpecial(expr.database_and_table_name);
}
}

View File

@ -443,7 +443,7 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
const String nested_table_name = ast->getColumnName();
const String nested_table_alias = ast->getAliasOrColumnName();
if (nested_table_alias == nested_table_name && !typeid_cast<const ASTIdentifier *>(ast.get()))
if (nested_table_alias == nested_table_name && !isIdentifier(ast))
throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name,
ErrorCodes::ALIAS_REQUIRED);
@ -471,7 +471,7 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
String result_name = expr->getAliasOrColumnName();
/// This is an array.
if (!typeid_cast<ASTIdentifier *>(expr.get()) || source_columns_set.count(source_name))
if (!isIdentifier(expr) || source_columns_set.count(source_name))
{
result.array_join_result_to_source[result_name] = source_name;
}
@ -528,10 +528,10 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS
std::function<TableBelonging(const ASTPtr &)> get_table_belonging;
get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging
{
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
if (identifier)
if (getColumnIdentifierName(ast))
{
if (identifier->general())
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
{
auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names);
auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names);
@ -567,9 +567,10 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS
std::function<void(ASTPtr &, const DatabaseAndTableWithAlias &, bool)> translate_qualified_names;
translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table)
{
if (auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get()))
if (getColumnIdentifierName(ast))
{
if (identifier->general())
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
{
auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names);
stripIdentifier(ast, num_components);

View File

@ -55,7 +55,7 @@ std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(const ASTIdentifier
const NameSet & source_columns = data.source_columns;
const std::vector<DatabaseAndTableWithAlias> & tables = data.tables;
if (identifier.general())
if (getColumnIdentifierName(identifier))
{
/// Select first table name with max number of qualifiers which can be stripped.
size_t max_num_qualifiers_to_strip = 0;

View File

@ -499,7 +499,7 @@ void executeQuery(
}
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->format).name
? *getIdentifierName(ast_query_with_output->format)
: context.getDefaultFormat();
BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader());

View File

@ -22,6 +22,7 @@
#include <Common/escapeForFileName.h>
#include <Common/Stopwatch.h>
#include <Common/typeid_cast.h>
namespace DB

View File

@ -1,4 +1,5 @@
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
@ -38,4 +39,69 @@ void ASTIdentifier::appendColumnNameImpl(WriteBuffer & ostr) const
writeString(name, ostr);
}
bool isIdentifier(const IAST * const ast)
{
if (ast)
return typeid_cast<const ASTIdentifier *>(ast);
return false;
}
std::optional<String> getIdentifierName(const IAST * const ast)
{
if (ast)
if (auto node = typeid_cast<const ASTIdentifier *>(ast))
return node->name;
return {};
}
bool getIdentifierName(const ASTPtr & ast, String & name)
{
if (ast)
if (auto node = typeid_cast<const ASTIdentifier *>(ast.get()))
{
name = node->name;
return true;
}
return false;
}
std::optional<String> getColumnIdentifierName(const ASTIdentifier & node)
{
if (!node.special())
return node.name;
return {};
}
std::optional<String> getColumnIdentifierName(const ASTPtr & ast)
{
if (ast)
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
if (!id->special())
return id->name;
return {};
}
std::optional<String> getTableIdentifierName(const ASTIdentifier & node)
{
if (node.special())
return node.name;
return {};
}
std::optional<String> getTableIdentifierName(const ASTPtr & ast)
{
if (ast)
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
if (id->special())
return id->name;
return {};
}
void setIdentifierSpecial(ASTPtr & ast)
{
if (ast)
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(ast.get()))
id->setSpecial();
}
}

View File

@ -1,13 +1,14 @@
#pragma once
#include <optional>
#include <Parsers/ASTWithAlias.h>
namespace DB
{
/** Identifier (column or alias)
*/
/// Identifier (column, table or alias)
class ASTIdentifier : public ASTWithAlias
{
enum Kind /// TODO This is semantic, not syntax. Remove it.
@ -33,10 +34,6 @@ public:
set.insert(name);
}
void setSpecial() { kind = Special; }
bool general() const { return kind == General; }
bool special() const { return kind == Special; }
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name_)
{
return std::make_shared<ASTIdentifier>(name_, ASTIdentifier::Special);
@ -48,6 +45,35 @@ protected:
private:
Kind kind;
void setSpecial() { kind = Special; }
bool special() const { return kind == Special; }
friend void setIdentifierSpecial(ASTPtr &);
friend std::optional<String> getColumnIdentifierName(const ASTIdentifier & node);
friend std::optional<String> getColumnIdentifierName(const ASTPtr & ast);
friend std::optional<String> getTableIdentifierName(const ASTIdentifier & node);
friend std::optional<String> getTableIdentifierName(const ASTPtr & ast);
};
/// ASTIdentifier Helpers: hide casts and semantic.
bool isIdentifier(const IAST * const ast);
inline bool isIdentifier(const ASTPtr & ast) { return isIdentifier(ast.get()); }
std::optional<String> getIdentifierName(const IAST * const ast);
inline std::optional<String> getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
bool getIdentifierName(const ASTPtr & ast, String & name);
/// @returns name for column identifiers
std::optional<String> getColumnIdentifierName(const ASTIdentifier & node);
std::optional<String> getColumnIdentifierName(const ASTPtr & ast);
/// @returns name for 'not a column' identifiers
std::optional<String> getTableIdentifierName(const ASTIdentifier & node);
std::optional<String> getTableIdentifierName(const ASTPtr & ast);
void setIdentifierSpecial(ASTPtr & ast);
}

View File

@ -5,6 +5,7 @@
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromMemory.h>
#include <Common/typeid_cast.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/IAST.h>
@ -173,7 +174,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
{
if (!name.empty())
name += '.';
name += static_cast<const ASTIdentifier &>(*child.get()).name;
name += *getIdentifierName(child);
}
node = std::make_shared<ASTIdentifier>(name);
@ -222,7 +223,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
* If you do not report that the first option is an error, then the argument will be interpreted as 2014 - 01 - 01 - some number,
* and the query silently returns an unexpected result.
*/
if (typeid_cast<const ASTIdentifier &>(*identifier).name == "toDate"
if (*getIdentifierName(identifier) == "toDate"
&& contents_end - contents_begin == strlen("2014-01-01")
&& contents_begin[0] >= '2' && contents_begin[0] <= '3'
&& contents_begin[1] >= '0' && contents_begin[1] <= '9'
@ -264,7 +265,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
auto function_node = std::make_shared<ASTFunction>();
function_node->name = typeid_cast<ASTIdentifier &>(*identifier).name;
getIdentifierName(identifier, function_node->name);
/// func(DISTINCT ...) is equivalent to funcDistinct(...)
if (has_distinct_modifier)
@ -1157,7 +1158,7 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
* and in the query "SELECT x FRO FROM t", the word FRO was considered an alias.
*/
const String & name = static_cast<const ASTIdentifier &>(*node.get()).name;
const String name = *getIdentifierName(node);
for (const char ** keyword = restricted_keywords; *keyword != nullptr; ++keyword)
if (0 == strcasecmp(name.data(), *keyword))
@ -1249,18 +1250,16 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
*/
bool allow_alias_without_as_keyword_now = allow_alias_without_as_keyword;
if (allow_alias_without_as_keyword)
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(node.get()))
if (0 == strcasecmp(id->name.data(), "FROM"))
if (auto opt_id = getIdentifierName(node))
if (0 == strcasecmp(opt_id->data(), "FROM"))
allow_alias_without_as_keyword_now = false;
ASTPtr alias_node;
if (ParserAlias(allow_alias_without_as_keyword_now).parse(pos, alias_node, expected))
{
String alias_name = typeid_cast<const ASTIdentifier &>(*alias_node).name;
if (ASTWithAlias * ast_with_alias = dynamic_cast<ASTWithAlias *>(node.get()))
{
ast_with_alias->alias = alias_name;
getIdentifierName(alias_node, ast_with_alias->alias);
ast_with_alias->prefer_alias_to_column_name = prefer_alias_to_column_name;
}
else

View File

@ -1,10 +1,9 @@
#include <Common/typeid_cast.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>

View File

@ -1,3 +1,4 @@
#include <Common/typeid_cast.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
@ -312,7 +313,7 @@ bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!p_expression.parse(pos, assignment->expression, expected))
return false;
assignment->column_name = typeid_cast<const ASTIdentifier &>(*column).name;
getIdentifierName(column, assignment->column_name);
if (assignment->expression)
assignment->children.push_back(assignment->expression);

View File

@ -4,8 +4,6 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTCheckQuery.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -31,15 +29,15 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
auto query = std::make_shared<ASTCheckQuery>();
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
node = query;
}
else
{
table = database;
auto query = std::make_shared<ASTCheckQuery>();
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
getIdentifierName(table, query->table);
node = query;
}

View File

@ -35,7 +35,7 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
auto func = std::make_shared<ASTFunction>();
func->name = typeid_cast<ASTIdentifier &>(*name).name;
getIdentifierName(name, func->name);
func->arguments = columns;
func->children.push_back(columns);
node = func;
@ -70,7 +70,7 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node,
if (non_parametric.parse(pos, ident, expected))
{
auto func = std::make_shared<ASTFunction>();
func->name = typeid_cast<ASTIdentifier &>(*ident).name;
getIdentifierName(ident, func->name);
node = func;
return true;
}
@ -257,10 +257,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->if_not_exists = if_not_exists;
query->cluster = cluster_str;
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
return true;
}
@ -405,23 +403,18 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->is_populate = is_populate;
query->temporary = is_temporary;
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
query->cluster = cluster_str;
if (to_database)
query->to_database = typeid_cast<ASTIdentifier &>(*to_database).name;
if (to_table)
query->to_table = typeid_cast<ASTIdentifier &>(*to_table).name;
getIdentifierName(to_database, query->to_database);
getIdentifierName(to_table, query->to_table);
query->set(query->columns, columns);
query->set(query->storage, storage);
if (as_database)
query->as_database = typeid_cast<ASTIdentifier &>(*as_database).name;
if (as_table)
query->as_table = typeid_cast<ASTIdentifier &>(*as_table).name;
getIdentifierName(as_database, query->as_database);
getIdentifierName(as_table, query->as_table);
query->set(query->select, select);
return true;

View File

@ -8,7 +8,6 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/CommonParsers.h>
#include <Common/typeid_cast.h>
#include <Poco/String.h>
@ -74,7 +73,7 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expect
&& type_parser.parse(pos, type, expected))
{
auto name_type_pair = std::make_shared<ASTNameTypePair>();
name_type_pair->name = typeid_cast<const ASTIdentifier &>(*name).name;
getIdentifierName(name, name_type_pair->name);
name_type_pair->type = type;
name_type_pair->children.push_back(type);
node = name_type_pair;
@ -181,7 +180,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
node = column_declaration;
column_declaration->name = typeid_cast<ASTIdentifier &>(*name).name;
getIdentifierName(name, column_declaration->name);
if (type)
{

View File

@ -4,8 +4,6 @@
#include <Parsers/CommonParsers.h>
#include <Parsers/ParserDropQuery.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -117,10 +115,10 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
query->kind = ASTDropQuery::Kind::Drop;
query->if_exists = if_exists;
query->temporary = temporary;
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
query->cluster = cluster_str;
return true;

View File

@ -9,8 +9,6 @@
#include <Parsers/ParserInsertQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -136,14 +134,11 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
else
{
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
}
if (format)
query->format = typeid_cast<ASTIdentifier &>(*format).name;
getIdentifierName(format, query->format);
query->columns = columns;
query->select = select;

View File

@ -5,8 +5,6 @@
#include <Parsers/ASTOptimizeQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -60,10 +58,8 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
auto query = std::make_shared<ASTOptimizeQuery>();
node = query;
if (database)
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
query->cluster = cluster_str;
query->partition = partition;

View File

@ -76,7 +76,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!format_p.parse(pos, query_with_output.format, expected))
return false;
typeid_cast<ASTIdentifier &>(*(query_with_output.format)).setSpecial();
setIdentifierSpecial(query_with_output.format);
query_with_output.children.push_back(query_with_output.format);
}

View File

@ -4,8 +4,6 @@
#include <Parsers/CommonParsers.h>
#include <Parsers/ParserRenameQuery.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -31,8 +29,9 @@ static bool parseDatabaseAndTable(
return false;
}
db_and_table.database = database ? typeid_cast<const ASTIdentifier &>(*database).name : "";
db_and_table.table = typeid_cast<const ASTIdentifier &>(*table).name;
db_and_table.database.clear();
getIdentifierName(database, db_and_table.database);
getIdentifierName(table, db_and_table.table);
return true;
}

View File

@ -31,7 +31,7 @@ static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos,
if (!value_p.parse(pos, value, expected))
return false;
change.name = typeid_cast<const ASTIdentifier &>(*name).name;
getIdentifierName(name, change.name);
change.value = typeid_cast<const ASTLiteral &>(*value).value;
return true;

View File

@ -65,8 +65,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
}
if (database)
query->from = typeid_cast<ASTIdentifier &>(*database).name;
getIdentifierName(database, query->from);
if (like)
query->like = safeGet<const String &>(typeid_cast<ASTLiteral &>(*like).value);

View File

@ -75,10 +75,8 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
}
}
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
node = query;

View File

@ -15,16 +15,15 @@ bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_use("USE");
ParserIdentifier name_p;
ASTPtr database;
if (!s_use.ignore(pos, expected))
return false;
ASTPtr database;
if (!name_p.parse(pos, database, expected))
return false;
auto query = std::make_shared<ASTUseQuery>();
query->database = typeid_cast<ASTIdentifier &>(*database).name;
getIdentifierName(database, query->database);
node = query;
return true;

View File

@ -2,7 +2,6 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Common/typeid_cast.h>
namespace DB
@ -30,13 +29,13 @@ bool parseDatabaseAndTableName(IParser::Pos & pos, Expected & expected, String &
return false;
}
database_str = typeid_cast<ASTIdentifier &>(*database).name;
table_str = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, database_str);
getIdentifierName(table, table_str);
}
else
{
database_str = "";
table_str = typeid_cast<ASTIdentifier &>(*database).name;
getIdentifierName(database, table_str);
}
return true;

View File

@ -20,7 +20,7 @@ bool parseIdentifierOrStringLiteral(IParser::Pos & pos, Expected & expected, Str
result = typeid_cast<const ASTLiteral &>(*res).value.safeGet<String>();
}
else
result = typeid_cast<const ASTIdentifier &>(*res).name;
result = *getIdentifierName(res);
return true;
}

View File

@ -50,7 +50,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
}
if (command_ast->column)
command.after_column = typeid_cast<const ASTIdentifier &>(*command_ast->column).name;
command.after_column = *getIdentifierName(command_ast->column);
command.if_not_exists = command_ast->if_not_exists;
@ -63,7 +63,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
AlterCommand command;
command.type = AlterCommand::DROP_COLUMN;
command.column_name = typeid_cast<const ASTIdentifier &>(*(command_ast->column)).name;
command.column_name = *getIdentifierName(command_ast->column);
command.if_exists = command_ast->if_exists;
return command;
}
@ -99,8 +99,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
{
AlterCommand command;
command.type = COMMENT_COLUMN;
const auto & ast_identifier = typeid_cast<ASTIdentifier &>(*command_ast->column);
command.column_name = ast_identifier.name;
command.column_name = *getIdentifierName(command_ast->column);
const auto & ast_comment = typeid_cast<ASTLiteral &>(*command_ast->comment);
command.comment = ast_comment.value.get<String>();
command.if_exists = command_ast->if_exists;

View File

@ -121,7 +121,7 @@ void MergeTreeWhereOptimizer::optimizeConjunction(ASTSelectQuery & select, ASTFu
SCOPE_EXIT(++idx);
if (cannotBeMoved(condition))
if (cannotBeMoved(conditions[idx]))
continue;
IdentifierNameSet identifiers{};
@ -193,7 +193,7 @@ void MergeTreeWhereOptimizer::optimizeArbitrary(ASTSelectQuery & select) const
auto & condition = select.where_expression;
/// do not optimize restricted expressions
if (cannotBeMoved(select.where_expression.get()))
if (cannotBeMoved(select.where_expression))
return;
IdentifierNameSet identifiers{};
@ -250,10 +250,10 @@ bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const
auto right_arg = function->arguments->children.back().get();
/// try to ensure left_arg points to ASTIdentifier
if (!typeid_cast<const ASTIdentifier *>(left_arg) && typeid_cast<const ASTIdentifier *>(right_arg))
if (!isIdentifier(left_arg) && isIdentifier(right_arg))
std::swap(left_arg, right_arg);
if (typeid_cast<const ASTIdentifier *>(left_arg))
if (isIdentifier(left_arg))
{
/// condition may be "good" if only right_arg is a constant and its value is outside the threshold
if (const auto literal = typeid_cast<const ASTLiteral *>(right_arg))
@ -286,8 +286,8 @@ bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const
void MergeTreeWhereOptimizer::collectIdentifiersNoSubqueries(const IAST * const ast, IdentifierNameSet & set)
{
if (const auto identifier = typeid_cast<const ASTIdentifier *>(ast))
return (void) set.insert(identifier->name);
if (auto opt_name = getIdentifierName(ast))
return (void) set.insert(*opt_name);
if (typeid_cast<const ASTSubquery *>(ast))
return;
@ -364,9 +364,9 @@ bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const IdentifierNameSet & i
}
bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr) const
{
if (const auto function_ptr = typeid_cast<const ASTFunction *>(ptr))
if (const auto function_ptr = typeid_cast<const ASTFunction *>(ptr.get()))
{
/// disallow arrayJoin expressions to be moved to PREWHERE for now
if (array_join_function_name == function_ptr->name)
@ -381,17 +381,16 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
if ("indexHint" == function_ptr->name)
return true;
}
else if (const auto identifier_ptr = typeid_cast<const ASTIdentifier *>(ptr))
else if (auto opt_name = getColumnIdentifierName(ptr))
{
/// disallow moving result of ARRAY JOIN to PREWHERE
if (identifier_ptr->general())
if (array_joined_names.count(identifier_ptr->name) ||
array_joined_names.count(Nested::extractTableName(identifier_ptr->name)))
return true;
if (array_joined_names.count(*opt_name) ||
array_joined_names.count(Nested::extractTableName(*opt_name)))
return true;
}
for (const auto & child : ptr->children)
if (cannotBeMoved(child.get()))
if (cannotBeMoved(child))
return true;
return false;

View File

@ -68,7 +68,7 @@ private:
*
* Also, disallow moving expressions with GLOBAL [NOT] IN.
*/
bool cannotBeMoved(const IAST * ptr) const;
bool cannotBeMoved(const ASTPtr & ptr) const;
void determineArrayJoinedNames(ASTSelectQuery & select);

View File

@ -42,13 +42,13 @@ static Names extractColumnNames(const ASTPtr & node)
Names res;
res.reserve(elements.size());
for (const auto & elem : elements)
res.push_back(typeid_cast<const ASTIdentifier &>(*elem).name);
res.push_back(*getIdentifierName(elem));
return res;
}
else
{
return { typeid_cast<const ASTIdentifier &>(*node).name };
return { *getIdentifierName(node) };
}
}
@ -481,9 +481,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
{
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get()))
merging_params.sign_column = ast->name;
else
if (!getIdentifierName(engine_args.back(), merging_params.sign_column))
throw Exception(
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
@ -495,9 +493,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// If the last element is not index_granularity or replica_name (a literal), then this is the name of the version column.
if (!engine_args.empty() && !typeid_cast<const ASTLiteral *>(engine_args.back().get()))
{
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get()))
merging_params.version_column = ast->name;
else
if (!getIdentifierName(engine_args.back(), merging_params.version_column))
throw Exception(
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
@ -535,18 +531,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
}
else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
{
if (auto ast = typeid_cast<ASTIdentifier *>(engine_args.back().get()))
merging_params.version_column = ast->name;
else
if (!getIdentifierName(engine_args.back(), merging_params.version_column))
throw Exception(
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
engine_args.pop_back();
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get()))
merging_params.sign_column = ast->name;
else
if (!getIdentifierName(engine_args.back(), merging_params.sign_column))
throw Exception(
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
@ -592,9 +584,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// Now only three parameters remain - date (or partitioning expression), primary_key, index_granularity.
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args[0].get()))
date_column_name = ast->name;
else
if (!getIdentifierName(engine_args[0], date_column_name))
throw Exception(
"Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);

View File

@ -2,7 +2,6 @@
#include <Storages/IStorage.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
namespace DB
@ -66,7 +65,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
PartitionCommand res;
res.type = CLEAR_COLUMN;
res.partition = command_ast->partition;
const Field & column_name = typeid_cast<const ASTIdentifier &>(*(command_ast->column)).name;
const Field & column_name = *getIdentifierName(command_ast->column);
res.column_name = column_name;
return res;
}

View File

@ -304,16 +304,16 @@ void registerStorageFile(StorageFactory & factory)
{
/// Will use FD if engine_args[1] is int literal or identifier with std* name
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(engine_args[1].get()))
if (auto opt_name = getIdentifierName(engine_args[1]))
{
if (identifier->name == "stdin")
if (*opt_name == "stdin")
source_fd = STDIN_FILENO;
else if (identifier->name == "stdout")
else if (*opt_name == "stdout")
source_fd = STDOUT_FILENO;
else if (identifier->name == "stderr")
else if (*opt_name == "stderr")
source_fd = STDERR_FILENO;
else
throw Exception("Unknown identifier '" + identifier->name + "' in second arg of File storage constructor",
throw Exception("Unknown identifier '" + *opt_name + "' in second arg of File storage constructor",
ErrorCodes::UNKNOWN_IDENTIFIER);
}
else if (const ASTLiteral * literal = typeid_cast<const ASTLiteral *>(engine_args[1].get()))

View File

@ -6,7 +6,6 @@
#include <Storages/StorageHDFS.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <IO/ReadBufferFromHDFS.h>
#include <Formats/FormatFactory.h>
@ -16,7 +15,6 @@
#include <DataStreams/OwningBlockInputStream.h>
#include <Poco/Path.h>
#include <Common/parseRemoteDescription.h>
#include <Common/typeid_cast.h>
namespace DB

View File

@ -3,7 +3,6 @@
#include <Interpreters/Join.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <Core/ColumnNumbers.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataTypes/NestedUtils.h>
@ -88,11 +87,11 @@ void registerStorageJoin(StorageFactory & factory)
"Storage Join requires at least 3 parameters: Join(ANY|ALL, LEFT|INNER, keys...).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTIdentifier * strictness_id = typeid_cast<const ASTIdentifier *>(engine_args[0].get());
if (!strictness_id)
auto opt_strictness_id = getIdentifierName(engine_args[0]);
if (!opt_strictness_id)
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
const String strictness_str = Poco::toLower(strictness_id->name);
const String strictness_str = Poco::toLower(*opt_strictness_id);
ASTTableJoin::Strictness strictness;
if (strictness_str == "any")
strictness = ASTTableJoin::Strictness::Any;
@ -101,11 +100,11 @@ void registerStorageJoin(StorageFactory & factory)
else
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
const ASTIdentifier * kind_id = typeid_cast<const ASTIdentifier *>(engine_args[1].get());
if (!kind_id)
auto opt_kind_id = getIdentifierName(engine_args[1]);
if (!opt_kind_id)
throw Exception("Second parameter of storage Join must be LEFT or INNER (without quotes).", ErrorCodes::BAD_ARGUMENTS);
const String kind_str = Poco::toLower(kind_id->name);
const String kind_str = Poco::toLower(*opt_kind_id);
ASTTableJoin::Kind kind;
if (kind_str == "left")
kind = ASTTableJoin::Kind::Left;
@ -122,11 +121,11 @@ void registerStorageJoin(StorageFactory & factory)
key_names.reserve(engine_args.size() - 2);
for (size_t i = 2, size = engine_args.size(); i < size; ++i)
{
const ASTIdentifier * key = typeid_cast<const ASTIdentifier *>(engine_args[i].get());
if (!key)
auto opt_key = getIdentifierName(engine_args[i]);
if (!opt_key)
throw Exception("Parameter №" + toString(i + 1) + " of storage Join don't look like column name.", ErrorCodes::BAD_ARGUMENTS);
key_names.push_back(key->name);
key_names.push_back(*opt_key);
}
auto & settings = args.context.getSettingsRef();

View File

@ -96,11 +96,9 @@ static bool isValidFunction(const ASTPtr & expression, const NameSet & columns)
if (!isValidFunction(expression->children[i], columns))
return false;
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(&*expression))
{
if (identifier->general())
return columns.count(identifier->name);
}
if (auto opt_name = getColumnIdentifierName(expression))
return columns.count(*opt_name);
return true;
}

View File

@ -76,7 +76,7 @@ static bool isCompatible(const IAST & node)
return true;
}
if (typeid_cast<const ASTIdentifier *>(&node))
if (isIdentifier(&node))
return true;
return false;

View File

@ -65,9 +65,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
}
else
{
if (auto ast_cluster = typeid_cast<const ASTIdentifier *>(args[arg_num].get()))
cluster_name = ast_cluster->name;
else
if (!getIdentifierName(args[arg_num], cluster_name))
cluster_description = getStringLiteral(*args[arg_num], "Hosts pattern");
}
++arg_num;
@ -132,9 +130,8 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
/// ExpressionAnalyzer will be created in InterpreterSelectQuery that will meet these `Identifier` when processing the request.
/// We need to mark them as the name of the database or table, because the default value is column.
for (auto & arg : args)
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
id->setSpecial();
for (auto ast : args)
setIdentifierSpecial(ast);
ClusterPtr cluster;
if (!cluster_name.empty())