mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #4056 from 4ertus2/ast
Helpers for ASTIdentifier (based on 'QueryNormalizer refactoring' changes)
This commit is contained in:
commit
8a251753be
@ -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()))
|
||||
{
|
||||
|
@ -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>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <sstream>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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 {};
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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 {};
|
||||
}
|
||||
};
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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());
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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>
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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()))
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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())
|
||||
|
Loading…
Reference in New Issue
Block a user