mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #3183 from 4ertus2/refactoring
refactoring: hide ASTIdentifier kind
This commit is contained in:
commit
a2d8ee1caa
@ -135,7 +135,7 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
|
||||
{
|
||||
ASTPtr database_name_identifier_node;
|
||||
if (!table.database_name.empty())
|
||||
database_name_identifier_node = std::make_shared<ASTIdentifier>(table.database_name, ASTIdentifier::Column);
|
||||
database_name_identifier_node = std::make_shared<ASTIdentifier>(table.database_name);
|
||||
|
||||
ASTPtr table_name_identifier_node;
|
||||
String table_name_or_alias;
|
||||
@ -146,9 +146,9 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
|
||||
table_name_or_alias = table.alias;
|
||||
|
||||
if (!table_name_or_alias.empty())
|
||||
table_name_identifier_node = std::make_shared<ASTIdentifier>(table_name_or_alias, ASTIdentifier::Column);
|
||||
table_name_identifier_node = std::make_shared<ASTIdentifier>(table_name_or_alias);
|
||||
|
||||
ASTPtr column_identifier_node = std::make_shared<ASTIdentifier>(column, ASTIdentifier::Column);
|
||||
ASTPtr column_identifier_node = std::make_shared<ASTIdentifier>(column);
|
||||
|
||||
String compound_name;
|
||||
if (database_name_identifier_node)
|
||||
@ -157,7 +157,7 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
|
||||
compound_name += table_name_or_alias + ".";
|
||||
compound_name += column;
|
||||
|
||||
auto elem = std::make_shared<ASTIdentifier>(compound_name, ASTIdentifier::Column);
|
||||
auto elem = std::make_shared<ASTIdentifier>(compound_name);
|
||||
|
||||
if (database_name_identifier_node)
|
||||
elem->children.emplace_back(std::move(database_name_identifier_node));
|
||||
|
@ -308,7 +308,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const std::ve
|
||||
{
|
||||
if (auto * identifier = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
{
|
||||
if (identifier->kind == ASTIdentifier::Column)
|
||||
if (identifier->general())
|
||||
{
|
||||
/// Select first table name with max number of qualifiers which can be stripped.
|
||||
size_t max_num_qualifiers_to_strip = 0;
|
||||
@ -646,7 +646,7 @@ void ExpressionAnalyzer::findExternalTables(ASTPtr & ast)
|
||||
StoragePtr external_storage;
|
||||
|
||||
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
if (node->kind == ASTIdentifier::Table)
|
||||
if (node->special())
|
||||
if ((external_storage = context.tryGetExternalTable(node->name)))
|
||||
external_tables[node->name] = external_storage;
|
||||
}
|
||||
@ -830,7 +830,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t
|
||||
* instead of doing a subquery, you just need to read it.
|
||||
*/
|
||||
|
||||
auto database_and_table_name = std::make_shared<ASTIdentifier>(external_table_name, ASTIdentifier::Table);
|
||||
auto database_and_table_name = ASTIdentifier::createSpecial(external_table_name);
|
||||
|
||||
if (auto ast_table_expr = typeid_cast<ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
|
||||
{
|
||||
@ -1659,7 +1659,7 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast)
|
||||
|
||||
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
{
|
||||
if (node->kind == ASTIdentifier::Column)
|
||||
if (node->general())
|
||||
{
|
||||
auto splitted = Nested::splitName(node->name); /// ParsedParams, Key1
|
||||
|
||||
@ -2884,7 +2884,7 @@ void ExpressionAnalyzer::collectJoinedColumnsFromJoinOnExpr()
|
||||
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
|
||||
if (identifier)
|
||||
{
|
||||
if (identifier->kind == ASTIdentifier::Column)
|
||||
if (identifier->general())
|
||||
{
|
||||
auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names);
|
||||
auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names);
|
||||
@ -2923,7 +2923,7 @@ void ExpressionAnalyzer::collectJoinedColumnsFromJoinOnExpr()
|
||||
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
|
||||
if (identifier)
|
||||
{
|
||||
if (identifier->kind == ASTIdentifier::Column)
|
||||
if (identifier->general())
|
||||
{
|
||||
auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names);
|
||||
stripIdentifier(ast, num_components);
|
||||
@ -3091,7 +3091,7 @@ void ExpressionAnalyzer::getRequiredSourceColumnsImpl(const ASTPtr & ast,
|
||||
|
||||
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
{
|
||||
if (node->kind == ASTIdentifier::Column
|
||||
if (node->general()
|
||||
&& !ignored_names.count(node->name)
|
||||
&& !ignored_names.count(Nested::extractTableName(node->name)))
|
||||
{
|
||||
|
@ -102,18 +102,18 @@ StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & contex
|
||||
|
||||
void replaceDatabaseAndTable(ASTPtr & database_and_table, const String & database_name, const String & table_name)
|
||||
{
|
||||
ASTPtr table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
ASTPtr table = ASTIdentifier::createSpecial(table_name);
|
||||
|
||||
if (!database_name.empty())
|
||||
{
|
||||
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
|
||||
ASTPtr database = ASTIdentifier::createSpecial(database_name);
|
||||
|
||||
database_and_table = std::make_shared<ASTIdentifier>(database_name + "." + table_name, ASTIdentifier::Table);
|
||||
database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name);
|
||||
database_and_table->children = {database, table};
|
||||
}
|
||||
else
|
||||
{
|
||||
database_and_table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
database_and_table = ASTIdentifier::createSpecial(table_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -85,7 +85,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
|
||||
if (functionIsInOrGlobalInOperator(func_node->name))
|
||||
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(func_node->arguments->children.at(1).get()))
|
||||
if (!aliases.count(right->name))
|
||||
right->kind = ASTIdentifier::Table;
|
||||
right->setSpecial();
|
||||
|
||||
/// Special cases for count function.
|
||||
String func_name_lowercase = Poco::toLower(func_node->name);
|
||||
@ -108,7 +108,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
|
||||
}
|
||||
else if ((identifier_node = typeid_cast<ASTIdentifier *>(ast.get())))
|
||||
{
|
||||
if (identifier_node->kind == ASTIdentifier::Column)
|
||||
if (identifier_node->general())
|
||||
{
|
||||
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
|
||||
auto it_alias = aliases.find(identifier_node->name);
|
||||
@ -122,7 +122,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
|
||||
{
|
||||
/// Avoid infinite recursion here
|
||||
auto replace_to_identifier = typeid_cast<ASTIdentifier *>(it_alias->second.get());
|
||||
bool is_cycle = replace_to_identifier && replace_to_identifier->kind == ASTIdentifier::Column
|
||||
bool is_cycle = replace_to_identifier && replace_to_identifier->general()
|
||||
&& replace_to_identifier->name == identifier_node->name;
|
||||
|
||||
if (!is_cycle)
|
||||
@ -164,9 +164,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
|
||||
if (database_and_table_name)
|
||||
{
|
||||
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(database_and_table_name.get()))
|
||||
{
|
||||
right->kind = ASTIdentifier::Table;
|
||||
}
|
||||
right->setSpecial();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -10,22 +10,17 @@ namespace DB
|
||||
*/
|
||||
class ASTIdentifier : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
enum Kind /// TODO This is semantic, not syntax. Remove it.
|
||||
{
|
||||
Column,
|
||||
Database,
|
||||
Table,
|
||||
Format,
|
||||
General,
|
||||
Special, // Database, Table, Format
|
||||
};
|
||||
|
||||
public:
|
||||
/// name. The composite identifier here will have a concatenated name (of the form a.b.c), and individual components will be available inside the children.
|
||||
String name;
|
||||
|
||||
/// what this identifier identifies
|
||||
Kind kind;
|
||||
|
||||
ASTIdentifier(const String & name_, const Kind kind_ = Column)
|
||||
ASTIdentifier(const String & name_, const Kind kind_ = General)
|
||||
: name(name_), kind(kind_) { range = StringRange(name.data(), name.data() + name.size()); }
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
@ -38,9 +33,21 @@ 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);
|
||||
}
|
||||
|
||||
protected:
|
||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
||||
|
||||
private:
|
||||
Kind kind;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -345,11 +345,11 @@ void ASTSelectQuery::setDatabaseIfNeeded(const String & database_name)
|
||||
|
||||
if (table_expression->database_and_table_name->children.empty())
|
||||
{
|
||||
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
|
||||
ASTPtr database = ASTIdentifier::createSpecial(database_name);
|
||||
ASTPtr table = table_expression->database_and_table_name;
|
||||
|
||||
const String & old_name = static_cast<ASTIdentifier &>(*table_expression->database_and_table_name).name;
|
||||
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(database_name + "." + old_name, ASTIdentifier::Table);
|
||||
table_expression->database_and_table_name = ASTIdentifier::createSpecial(database_name + "." + old_name);
|
||||
table_expression->database_and_table_name->children = {database, table};
|
||||
}
|
||||
else if (table_expression->database_and_table_name->children.size() != 2)
|
||||
@ -376,18 +376,18 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
|
||||
table_expression = table_expr.get();
|
||||
}
|
||||
|
||||
ASTPtr table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
ASTPtr table = ASTIdentifier::createSpecial(table_name);
|
||||
|
||||
if (!database_name.empty())
|
||||
{
|
||||
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
|
||||
ASTPtr database = ASTIdentifier::createSpecial(database_name);
|
||||
|
||||
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(database_name + "." + table_name, ASTIdentifier::Table);
|
||||
table_expression->database_and_table_name = ASTIdentifier::createSpecial(database_name + "." + table_name);
|
||||
table_expression->database_and_table_name->children = {database, table};
|
||||
}
|
||||
else
|
||||
{
|
||||
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
table_expression->database_and_table_name = ASTIdentifier::createSpecial(table_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -69,7 +69,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)).kind = ASTIdentifier::Format;
|
||||
typeid_cast<ASTIdentifier &>(*(query_with_output.format)).setSpecial();
|
||||
|
||||
query_with_output.children.push_back(query_with_output.format);
|
||||
}
|
||||
|
@ -384,7 +384,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
|
||||
else if (const auto identifier_ptr = typeid_cast<const ASTIdentifier *>(ptr))
|
||||
{
|
||||
/// disallow moving result of ARRAY JOIN to PREWHERE
|
||||
if (identifier_ptr->kind == ASTIdentifier::Column)
|
||||
if (identifier_ptr->general())
|
||||
if (array_joined_names.count(identifier_ptr->name) ||
|
||||
array_joined_names.count(Nested::extractTableName(identifier_ptr->name)))
|
||||
return true;
|
||||
|
@ -595,7 +595,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl
|
||||
insert->columns = list_of_columns;
|
||||
list_of_columns->children.reserve(columns_intersection.size());
|
||||
for (const String & column : columns_intersection)
|
||||
list_of_columns->children.push_back(std::make_shared<ASTIdentifier>(column, ASTIdentifier::Column));
|
||||
list_of_columns->children.push_back(std::make_shared<ASTIdentifier>(column));
|
||||
|
||||
InterpreterInsertQuery interpreter{insert, context, allow_materialized};
|
||||
|
||||
|
@ -97,7 +97,7 @@ static bool isValidFunction(const ASTPtr & expression, const NameSet & columns)
|
||||
|
||||
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(&*expression))
|
||||
{
|
||||
if (identifier->kind == ASTIdentifier::Kind::Column)
|
||||
if (identifier->general())
|
||||
return columns.count(identifier->name);
|
||||
}
|
||||
return true;
|
||||
|
@ -292,7 +292,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
|
||||
/// 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->kind = ASTIdentifier::Table;
|
||||
id->setSpecial();
|
||||
|
||||
ClusterPtr cluster;
|
||||
if (!cluster_name.empty())
|
||||
|
@ -66,7 +66,7 @@ StoragePtr TableFunctionShardByHash::executeImpl(const ASTPtr & ast_function, co
|
||||
/// Similar to other TableFunctions.
|
||||
for (auto & arg : args)
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
|
||||
id->kind = ASTIdentifier::Table;
|
||||
id->setSpecial();
|
||||
|
||||
auto cluster = context.getCluster(cluster_name);
|
||||
size_t shard_index = sipHash64(key) % cluster->getShardCount();
|
||||
|
Loading…
Reference in New Issue
Block a user