Merge pull request #3183 from 4ertus2/refactoring

refactoring: hide ASTIdentifier kind
This commit is contained in:
alexey-milovidov 2018-09-20 20:38:06 +03:00 committed by GitHub
commit a2d8ee1caa
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 47 additions and 42 deletions

View File

@ -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));

View File

@ -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)))
{

View File

@ -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);
}
}

View File

@ -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();
}
}
}

View File

@ -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;
};
}

View File

@ -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);
}
}

View File

@ -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);
}

View File

@ -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;

View File

@ -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};

View File

@ -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;

View File

@ -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())

View File

@ -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();