Fixed unpleasant code #6401

This commit is contained in:
Alexey Milovidov 2019-08-08 23:02:30 +03:00
parent e40854d7fb
commit a6e4b34ab4
32 changed files with 94 additions and 73 deletions

View File

@ -366,7 +366,7 @@ static DataTypePtr create(const ASTPtr & arguments)
params_row[i] = literal->value; params_row[i] = literal->value;
} }
} }
else if (auto opt_name = getIdentifierName(arguments->children[0])) else if (auto opt_name = tryGetIdentifierName(arguments->children[0]))
{ {
function_name = *opt_name; function_name = *opt_name;
} }

View File

@ -88,7 +88,7 @@ static std::pair<DataTypePtr, DataTypeCustomDescPtr> create(const ASTPtr & argum
params_row[i] = lit->value; params_row[i] = lit->value;
} }
} }
else if (auto opt_name = getIdentifierName(arguments->children[0])) else if (auto opt_name = tryGetIdentifierName(arguments->children[0]))
{ {
function_name = *opt_name; function_name = *opt_name;
} }

View File

@ -463,7 +463,7 @@ void ActionsVisitor::visit(const ASTPtr & ast)
for (size_t j = 0; j < lambda_arg_asts.size(); ++j) for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
{ {
auto opt_arg_name = getIdentifierName(lambda_arg_asts[j]); auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]);
if (!opt_arg_name) if (!opt_arg_name)
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);

View File

@ -39,7 +39,7 @@ struct ColumnNamesContext
std::optional<String> name() const std::optional<String> name() const
{ {
if (expr) if (expr)
return getIdentifierName(expr->database_and_table_name); return tryGetIdentifierName(expr->database_and_table_name);
return {}; return {};
} }

View File

@ -75,7 +75,7 @@ public:
if (is_table) if (is_table)
{ {
/// If this is already an external table, you do not need to add anything. Just remember its presence. /// If this is already an external table, you do not need to add anything. Just remember its presence.
if (external_tables.end() != external_tables.find(*getIdentifierName(subquery_or_table_name))) if (external_tables.end() != external_tables.find(getIdentifierName(subquery_or_table_name)))
return; return;
} }

View File

@ -74,7 +74,7 @@ void QueryNormalizer::visit(ASTFunction & node, const ASTPtr &, Data & data)
if (functionIsInOrGlobalInOperator(func_name)) if (functionIsInOrGlobalInOperator(func_name))
{ {
auto & ast = func_arguments->children.at(1); auto & ast = func_arguments->children.at(1);
if (auto opt_name = getIdentifierName(ast)) if (auto opt_name = tryGetIdentifierName(ast))
if (!aliases.count(*opt_name)) if (!aliases.count(*opt_name))
setIdentifierSpecial(ast); setIdentifierSpecial(ast);
} }

View File

@ -253,7 +253,7 @@ void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, D
{ {
const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>(); const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>();
for (const auto & key : keys.children) for (const auto & key : keys.children)
if (auto opt_column = getIdentifierName(key)) if (auto opt_column = tryGetIdentifierName(key))
data.join_using_columns.insert(*opt_column); data.join_using_columns.insert(*opt_column);
else if (key->as<ASTLiteral>()) else if (key->as<ASTLiteral>())
data.join_using_columns.insert(key->getColumnName()); data.join_using_columns.insert(key->getColumnName());

View File

@ -565,7 +565,7 @@ void executeQuery(
} }
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr) String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
? *getIdentifierName(ast_query_with_output->format) ? getIdentifierName(ast_query_with_output->format)
: context.getDefaultFormat(); : context.getDefaultFormat();
if (ast_query_with_output && ast_query_with_output->settings_ast) if (ast_query_with_output && ast_query_with_output->settings_ast)
@ -610,7 +610,7 @@ void executeQuery(
} }
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr) String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
? *getIdentifierName(ast_query_with_output->format) ? getIdentifierName(ast_query_with_output->format)
: context.getDefaultFormat(); : context.getDefaultFormat();
if (ast_query_with_output && ast_query_with_output->settings_ast) if (ast_query_with_output && ast_query_with_output->settings_ast)

View File

@ -1,5 +1,6 @@
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/queryToString.h>
#include <IO/WriteBufferFromOStream.h> #include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Interpreters/IdentifierSemantic.h> #include <Interpreters/IdentifierSemantic.h>
@ -8,6 +9,12 @@
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int UNEXPECTED_AST_STRUCTURE;
}
ASTPtr ASTIdentifier::clone() const ASTPtr ASTIdentifier::clone() const
{ {
auto ret = std::make_shared<ASTIdentifier>(*this); auto ret = std::make_shared<ASTIdentifier>(*this);
@ -92,22 +99,32 @@ ASTPtr createTableIdentifier(const String & database_name, const String & table_
return database_and_table; return database_and_table;
} }
std::optional<String> getIdentifierName(const IAST * const ast) String getIdentifierName(const IAST * ast)
{ {
if (ast) String res;
if (const auto * node = ast->as<ASTIdentifier>()) if (tryGetIdentifierNameInto(ast, res))
return node->name; return res;
throw Exception(ast ? queryToString(ast) + " is not an identifier" : "AST node is nullptr", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
}
std::optional<String> tryGetIdentifierName(const IAST * ast)
{
String res;
if (tryGetIdentifierNameInto(ast, res))
return res;
return {}; return {};
} }
bool getIdentifierName(const ASTPtr & ast, String & name) bool tryGetIdentifierNameInto(const ASTPtr & ast, String & name)
{ {
if (ast) if (ast)
{
if (const auto * node = ast->as<ASTIdentifier>()) if (const auto * node = ast->as<ASTIdentifier>())
{ {
name = node->name; name = node->name;
return true; return true;
} }
}
return false; return false;
} }

View File

@ -70,9 +70,13 @@ private:
ASTPtr createTableIdentifier(const String & database_name, const String & table_name); ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
void setIdentifierSpecial(ASTPtr & ast); void setIdentifierSpecial(ASTPtr & ast);
std::optional<String> getIdentifierName(const IAST * const ast); String getIdentifierName(const IAST * ast);
inline std::optional<String> getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); } std::optional<String> tryGetIdentifierName(const IAST * ast);
bool getIdentifierName(const ASTPtr & ast, String & name);
inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
inline std::optional<String> tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); }
bool tryGetIdentifierNameInto(const ASTPtr & ast, String & name);
} }

View File

@ -177,7 +177,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
{ {
if (!name.empty()) if (!name.empty())
name += '.'; name += '.';
parts.emplace_back(*getIdentifierName(child)); parts.emplace_back(getIdentifierName(child));
name += parts.back(); name += parts.back();
} }
@ -225,7 +225,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, * 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. * and the query silently returns an unexpected result.
*/ */
if (*getIdentifierName(identifier) == "toDate" if (getIdentifierName(identifier) == "toDate"
&& contents_end - contents_begin == strlen("2014-01-01") && contents_end - contents_begin == strlen("2014-01-01")
&& contents_begin[0] >= '2' && contents_begin[0] <= '3' && contents_begin[0] >= '2' && contents_begin[0] <= '3'
&& contents_begin[1] >= '0' && contents_begin[1] <= '9' && contents_begin[1] >= '0' && contents_begin[1] <= '9'
@ -267,7 +267,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
} }
auto function_node = std::make_shared<ASTFunction>(); auto function_node = std::make_shared<ASTFunction>();
getIdentifierName(identifier, function_node->name); tryGetIdentifierNameInto(identifier, function_node->name);
/// func(DISTINCT ...) is equivalent to funcDistinct(...) /// func(DISTINCT ...) is equivalent to funcDistinct(...)
if (has_distinct_modifier) if (has_distinct_modifier)
@ -1158,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. * and in the query "SELECT x FRO FROM t", the word FRO was considered an alias.
*/ */
const String name = *getIdentifierName(node); const String name = getIdentifierName(node);
for (const char ** keyword = restricted_keywords; *keyword != nullptr; ++keyword) for (const char ** keyword = restricted_keywords; *keyword != nullptr; ++keyword)
if (0 == strcasecmp(name.data(), *keyword)) if (0 == strcasecmp(name.data(), *keyword))
@ -1326,7 +1326,7 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
*/ */
bool allow_alias_without_as_keyword_now = allow_alias_without_as_keyword; bool allow_alias_without_as_keyword_now = allow_alias_without_as_keyword;
if (allow_alias_without_as_keyword) if (allow_alias_without_as_keyword)
if (auto opt_id = getIdentifierName(node)) if (auto opt_id = tryGetIdentifierName(node))
if (0 == strcasecmp(opt_id->data(), "FROM")) if (0 == strcasecmp(opt_id->data(), "FROM"))
allow_alias_without_as_keyword_now = false; allow_alias_without_as_keyword_now = false;
@ -1336,7 +1336,7 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
/// FIXME: try to prettify this cast using `as<>()` /// FIXME: try to prettify this cast using `as<>()`
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(node.get())) if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(node.get()))
{ {
getIdentifierName(alias_node, ast_with_alias->alias); tryGetIdentifierNameInto(alias_node, ast_with_alias->alias);
} }
else else
{ {

View File

@ -354,7 +354,7 @@ bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!p_expression.parse(pos, assignment->expression, expected)) if (!p_expression.parse(pos, assignment->expression, expected))
return false; return false;
getIdentifierName(column, assignment->column_name); tryGetIdentifierNameInto(column, assignment->column_name);
if (assignment->expression) if (assignment->expression)
assignment->children.push_back(assignment->expression); assignment->children.push_back(assignment->expression);

View File

@ -32,13 +32,13 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!table_parser.parse(pos, table, expected)) if (!table_parser.parse(pos, table, expected))
return false; return false;
getIdentifierName(database, query->database); tryGetIdentifierNameInto(database, query->database);
getIdentifierName(table, query->table); tryGetIdentifierNameInto(table, query->table);
} }
else else
{ {
table = database; table = database;
getIdentifierName(table, query->table); tryGetIdentifierNameInto(table, query->table);
} }
if (s_partition.ignore(pos, expected)) if (s_partition.ignore(pos, expected))

View File

@ -39,7 +39,7 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false; return false;
auto func = std::make_shared<ASTFunction>(); auto func = std::make_shared<ASTFunction>();
getIdentifierName(name, func->name); tryGetIdentifierNameInto(name, func->name);
func->arguments = columns; func->arguments = columns;
func->children.push_back(columns); func->children.push_back(columns);
node = func; node = func;
@ -74,7 +74,7 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node,
if (non_parametric.parse(pos, ident, expected)) if (non_parametric.parse(pos, ident, expected))
{ {
auto func = std::make_shared<ASTFunction>(); auto func = std::make_shared<ASTFunction>();
getIdentifierName(ident, func->name); tryGetIdentifierNameInto(ident, func->name);
node = func; node = func;
return true; return true;
} }
@ -384,8 +384,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->if_not_exists = if_not_exists; query->if_not_exists = if_not_exists;
query->cluster = cluster_str; query->cluster = cluster_str;
getIdentifierName(database, query->database); tryGetIdentifierNameInto(database, query->database);
getIdentifierName(table, query->table); tryGetIdentifierNameInto(table, query->table);
return true; return true;
} }
@ -542,18 +542,18 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->temporary = is_temporary; query->temporary = is_temporary;
query->replace_view = replace_view; query->replace_view = replace_view;
getIdentifierName(database, query->database); tryGetIdentifierNameInto(database, query->database);
getIdentifierName(table, query->table); tryGetIdentifierNameInto(table, query->table);
query->cluster = cluster_str; query->cluster = cluster_str;
getIdentifierName(to_database, query->to_database); tryGetIdentifierNameInto(to_database, query->to_database);
getIdentifierName(to_table, query->to_table); tryGetIdentifierNameInto(to_table, query->to_table);
query->set(query->columns_list, columns_list); query->set(query->columns_list, columns_list);
query->set(query->storage, storage); query->set(query->storage, storage);
getIdentifierName(as_database, query->as_database); tryGetIdentifierNameInto(as_database, query->as_database);
getIdentifierName(as_table, query->as_table); tryGetIdentifierNameInto(as_table, query->as_table);
query->set(query->select, select); query->set(query->select, select);
return true; return true;

View File

@ -73,7 +73,7 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expect
&& type_parser.parse(pos, type, expected)) && type_parser.parse(pos, type, expected))
{ {
auto name_type_pair = std::make_shared<ASTNameTypePair>(); auto name_type_pair = std::make_shared<ASTNameTypePair>();
getIdentifierName(name, name_type_pair->name); tryGetIdentifierNameInto(name, name_type_pair->name);
name_type_pair->type = type; name_type_pair->type = type;
name_type_pair->children.push_back(type); name_type_pair->children.push_back(type);
node = name_type_pair; node = name_type_pair;
@ -189,7 +189,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
const auto column_declaration = std::make_shared<ASTColumnDeclaration>(); const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
node = column_declaration; node = column_declaration;
getIdentifierName(name, column_declaration->name); tryGetIdentifierNameInto(name, column_declaration->name);
if (type) if (type)
{ {

View File

@ -116,8 +116,8 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
query->if_exists = if_exists; query->if_exists = if_exists;
query->temporary = temporary; query->temporary = temporary;
getIdentifierName(database, query->database); tryGetIdentifierNameInto(database, query->database);
getIdentifierName(table, query->table); tryGetIdentifierNameInto(table, query->table);
query->cluster = cluster_str; query->cluster = cluster_str;

View File

@ -147,11 +147,11 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
} }
else else
{ {
getIdentifierName(database, query->database); tryGetIdentifierNameInto(database, query->database);
getIdentifierName(table, query->table); tryGetIdentifierNameInto(table, query->table);
} }
getIdentifierName(format, query->format); tryGetIdentifierNameInto(format, query->format);
query->columns = columns; query->columns = columns;
query->select = select; query->select = select;

View File

@ -30,8 +30,8 @@ static bool parseDatabaseAndTable(
} }
db_and_table.database.clear(); db_and_table.database.clear();
getIdentifierName(database, db_and_table.database); tryGetIdentifierNameInto(database, db_and_table.database);
getIdentifierName(table, db_and_table.table); tryGetIdentifierNameInto(table, db_and_table.table);
return true; return true;
} }

View File

@ -31,7 +31,7 @@ static bool parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expec
if (!value_p.parse(pos, value, expected)) if (!value_p.parse(pos, value, expected))
return false; return false;
getIdentifierName(name, change.name); tryGetIdentifierNameInto(name, change.name);
change.value = value->as<ASTLiteral &>().value; change.value = value->as<ASTLiteral &>().value;
return true; return true;

View File

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

View File

@ -75,8 +75,8 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
} }
} }
getIdentifierName(database, query->database); tryGetIdentifierNameInto(database, query->database);
getIdentifierName(table, query->table); tryGetIdentifierNameInto(table, query->table);
node = query; node = query;

View File

@ -21,7 +21,7 @@ bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false; return false;
auto query = std::make_shared<ASTUseQuery>(); auto query = std::make_shared<ASTUseQuery>();
getIdentifierName(database, query->database); tryGetIdentifierNameInto(database, query->database);
node = query; node = query;
return true; return true;

View File

@ -29,13 +29,13 @@ bool parseDatabaseAndTableName(IParser::Pos & pos, Expected & expected, String &
return false; return false;
} }
getIdentifierName(database, database_str); tryGetIdentifierNameInto(database, database_str);
getIdentifierName(table, table_str); tryGetIdentifierNameInto(table, table_str);
} }
else else
{ {
database_str = ""; database_str = "";
getIdentifierName(database, table_str); tryGetIdentifierNameInto(database, table_str);
} }
return true; return true;

View File

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

View File

@ -64,7 +64,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type); command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type);
if (command_ast->column) if (command_ast->column)
command.after_column = *getIdentifierName(command_ast->column); command.after_column = getIdentifierName(command_ast->column);
if (ast_col_decl.ttl) if (ast_col_decl.ttl)
command.ttl = ast_col_decl.ttl; command.ttl = ast_col_decl.ttl;
@ -80,7 +80,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
AlterCommand command; AlterCommand command;
command.type = AlterCommand::DROP_COLUMN; command.type = AlterCommand::DROP_COLUMN;
command.column_name = *getIdentifierName(command_ast->column); command.column_name = getIdentifierName(command_ast->column);
command.if_exists = command_ast->if_exists; command.if_exists = command_ast->if_exists;
return command; return command;
} }
@ -123,7 +123,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
{ {
AlterCommand command; AlterCommand command;
command.type = COMMENT_COLUMN; command.type = COMMENT_COLUMN;
command.column_name = *getIdentifierName(command_ast->column); command.column_name = getIdentifierName(command_ast->column);
const auto & ast_comment = command_ast->comment->as<ASTLiteral &>(); const auto & ast_comment = command_ast->comment->as<ASTLiteral &>();
command.comment = ast_comment.value.get<String>(); command.comment = ast_comment.value.get<String>();
command.if_exists = command_ast->if_exists; command.if_exists = command_ast->if_exists;

View File

@ -91,7 +91,7 @@ void ColumnDescription::writeText(WriteBuffer & buf) const
void ColumnDescription::readText(ReadBuffer & buf) void ColumnDescription::readText(ReadBuffer & buf)
{ {
ParserColumnDeclaration column_parser(true); ParserColumnDeclaration column_parser(/* require type */ true);
String column_line; String column_line;
readEscapedStringUntilEOL(column_line, buf); readEscapedStringUntilEOL(column_line, buf);
ASTPtr ast = parseQuery(column_parser, column_line, "column parser", 0); ASTPtr ast = parseQuery(column_parser, column_line, "column parser", 0);

View File

@ -62,7 +62,7 @@ void MergeTreeWhereOptimizer::calculateColumnSizes(const MergeTreeData & data, c
static void collectIdentifiersNoSubqueries(const ASTPtr & ast, NameSet & set) static void collectIdentifiersNoSubqueries(const ASTPtr & ast, NameSet & set)
{ {
if (auto opt_name = getIdentifierName(ast)) if (auto opt_name = tryGetIdentifierNameInto(ast))
return (void)set.insert(*opt_name); return (void)set.insert(*opt_name);
if (ast->as<ASTSubquery>()) if (ast->as<ASTSubquery>())

View File

@ -45,13 +45,13 @@ static Names extractColumnNames(const ASTPtr & node)
Names res; Names res;
res.reserve(elements.size()); res.reserve(elements.size());
for (const auto & elem : elements) for (const auto & elem : elements)
res.push_back(*getIdentifierName(elem)); res.push_back(getIdentifierName(elem));
return res; return res;
} }
else else
{ {
return { *getIdentifierName(node) }; return { getIdentifierName(node) };
} }
} }
@ -502,7 +502,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing) if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
{ {
if (!getIdentifierName(engine_args.back(), merging_params.sign_column)) if (!tryGetIdentifierNameInto(engine_args.back(), merging_params.sign_column))
throw Exception( throw Exception(
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);
@ -514,7 +514,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 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() && !engine_args.back()->as<ASTLiteral>()) if (!engine_args.empty() && !engine_args.back()->as<ASTLiteral>())
{ {
if (!getIdentifierName(engine_args.back(), merging_params.version_column)) if (!tryGetIdentifierNameInto(engine_args.back(), merging_params.version_column))
throw Exception( throw Exception(
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);
@ -552,14 +552,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
} }
else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
{ {
if (!getIdentifierName(engine_args.back(), merging_params.version_column)) if (!tryGetIdentifierNameInto(engine_args.back(), merging_params.version_column))
throw Exception( throw Exception(
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);
engine_args.pop_back(); engine_args.pop_back();
if (!getIdentifierName(engine_args.back(), merging_params.sign_column)) if (!tryGetIdentifierNameInto(engine_args.back(), merging_params.sign_column))
throw Exception( throw Exception(
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);
@ -616,7 +616,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// Now only three parameters remain - date (or partitioning expression), primary_key, index_granularity. /// Now only three parameters remain - date (or partitioning expression), primary_key, index_granularity.
if (!getIdentifierName(engine_args[0], date_column_name)) if (!tryGetIdentifierNameInto(engine_args[0], date_column_name))
throw Exception( throw Exception(
"Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);

View File

@ -65,7 +65,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
PartitionCommand res; PartitionCommand res;
res.type = CLEAR_COLUMN; res.type = CLEAR_COLUMN;
res.partition = command_ast->partition; res.partition = command_ast->partition;
res.column_name = *getIdentifierName(command_ast->column); res.column_name = getIdentifierName(command_ast->column);
return res; return res;
} }
else if (command_ast->type == ASTAlterCommand::FREEZE_ALL) else if (command_ast->type == ASTAlterCommand::FREEZE_ALL)

View File

@ -303,7 +303,7 @@ void registerStorageFile(StorageFactory & factory)
{ {
/// Will use FD if engine_args[1] is int literal or identifier with std* name /// Will use FD if engine_args[1] is int literal or identifier with std* name
if (auto opt_name = getIdentifierName(engine_args[1])) if (auto opt_name = tryGetIdentifierName(engine_args[1]))
{ {
if (*opt_name == "stdin") if (*opt_name == "stdin")
source_fd = STDIN_FILENO; source_fd = STDIN_FILENO;

View File

@ -90,7 +90,7 @@ void registerStorageJoin(StorageFactory & factory)
"Storage Join requires at least 3 parameters: Join(ANY|ALL, LEFT|INNER, keys...).", "Storage Join requires at least 3 parameters: Join(ANY|ALL, LEFT|INNER, keys...).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
auto opt_strictness_id = getIdentifierName(engine_args[0]); auto opt_strictness_id = tryGetIdentifierName(engine_args[0]);
if (!opt_strictness_id) if (!opt_strictness_id)
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS); throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
@ -103,7 +103,7 @@ void registerStorageJoin(StorageFactory & factory)
else else
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS); throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
auto opt_kind_id = getIdentifierName(engine_args[1]); auto opt_kind_id = tryGetIdentifierName(engine_args[1]);
if (!opt_kind_id) if (!opt_kind_id)
throw Exception("Second parameter of storage Join must be LEFT or INNER (without quotes).", ErrorCodes::BAD_ARGUMENTS); throw Exception("Second parameter of storage Join must be LEFT or INNER (without quotes).", ErrorCodes::BAD_ARGUMENTS);
@ -124,7 +124,7 @@ void registerStorageJoin(StorageFactory & factory)
key_names.reserve(engine_args.size() - 2); key_names.reserve(engine_args.size() - 2);
for (size_t i = 2, size = engine_args.size(); i < size; ++i) for (size_t i = 2, size = engine_args.size(); i < size; ++i)
{ {
auto opt_key = getIdentifierName(engine_args[i]); auto opt_key = tryGetIdentifierName(engine_args[i]);
if (!opt_key) if (!opt_key)
throw Exception("Parameter №" + toString(i + 1) + " of storage Join don't look like column name.", ErrorCodes::BAD_ARGUMENTS); throw Exception("Parameter №" + toString(i + 1) + " of storage Join don't look like column name.", ErrorCodes::BAD_ARGUMENTS);

View File

@ -67,7 +67,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
} }
else else
{ {
if (!getIdentifierName(args[arg_num], cluster_name)) if (!tryGetIdentifierNameInto(args[arg_num], cluster_name))
cluster_description = getStringLiteral(*args[arg_num], "Hosts pattern"); cluster_description = getStringLiteral(*args[arg_num], "Hosts pattern");
} }
++arg_num; ++arg_num;