提交 5289c133 编写于 作者: C chertus

child free ASTIdentifier

上级 b50fa226
......@@ -99,8 +99,7 @@ private:
/// @note It expects that only table (not column) identifiers are visited.
void visit(const ASTIdentifier & identifier, ASTPtr & ast) const
{
DatabaseAndTableWithAlias db_and_name(identifier);
if (db_and_name.database.empty())
if (identifier.name_parts.empty())
ast = createTableIdentifier(database_name, identifier.name);
}
......
......@@ -13,7 +13,7 @@ namespace DB
/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left.
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'.
void stripIdentifier(DB::ASTPtr & ast, size_t num_qualifiers_to_strip)
void stripIdentifier(const DB::ASTPtr & ast, size_t num_qualifiers_to_strip)
{
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(ast.get());
......@@ -22,29 +22,15 @@ void stripIdentifier(DB::ASTPtr & ast, size_t num_qualifiers_to_strip)
if (num_qualifiers_to_strip)
{
size_t num_components = identifier->children.size();
/// plain column
if (num_components - num_qualifiers_to_strip == 1)
{
DB::String node_alias = identifier->tryGetAlias();
ast = identifier->children.back();
if (!node_alias.empty())
ast->setAlias(node_alias);
}
else
/// nested column
identifier->name_parts.erase(identifier->name_parts.begin(), identifier->name_parts.begin() + num_qualifiers_to_strip);
DB::String new_name;
for (const auto & part : identifier->name_parts)
{
identifier->children.erase(identifier->children.begin(), identifier->children.begin() + num_qualifiers_to_strip);
DB::String new_name;
for (const auto & child : identifier->children)
{
if (!new_name.empty())
new_name += '.';
new_name += static_cast<const ASTIdentifier &>(*child.get()).name;
}
identifier->name = new_name;
if (!new_name.empty())
new_name += '.';
new_name += part;
}
identifier->name.swap(new_name);
}
}
......@@ -71,13 +57,13 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident
table = identifier.name;
alias = identifier.tryGetAlias();
if (!identifier.children.empty())
if (!identifier.name_parts.empty())
{
if (identifier.children.size() != 2)
throw Exception("Logical error: number of components in table expression not equal to two", ErrorCodes::LOGICAL_ERROR);
if (identifier.name_parts.size() != 2)
throw Exception("Logical error: 2 components expected in table expression '" + identifier.name + "'", ErrorCodes::LOGICAL_ERROR);
getIdentifierName(identifier.children[0], database);
getIdentifierName(identifier.children[1], table);
database = identifier.name_parts[0];
table = identifier.name_parts[1];
}
}
......
......@@ -39,7 +39,7 @@ struct DatabaseAndTableWithAlias
bool satisfies(const DatabaseAndTableWithAlias & table, bool table_may_be_an_alias);
};
void stripIdentifier(DB::ASTPtr & ast, size_t num_qualifiers_to_strip);
void stripIdentifier(const DB::ASTPtr & ast, size_t num_qualifiers_to_strip);
size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier,
const DatabaseAndTableWithAlias & names);
......
......@@ -86,20 +86,17 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
String database_name;
String table_name;
auto identifier = table_expression->database_and_table_name;
if (identifier->children.size() > 2)
auto identifier = typeid_cast<const ASTIdentifier *>(table_expression->database_and_table_name.get());
if (identifier->name_parts.size() > 2)
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
if (identifier->children.size() > 1)
if (identifier->name_parts.size() > 1)
{
auto database_ptr = identifier->children[0];
auto table_ptr = identifier->children[1];
getIdentifierName(database_ptr, database_name);
getIdentifierName(table_ptr, table_name);
database_name = identifier->name_parts[0];
table_name = identifier->name_parts[1];
}
else
getIdentifierName(identifier, table_name);
table_name = identifier->name;
table = context.getTable(database_name, table_name);
}
......
#include <Common/typeid_cast.h>
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/SemanticSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
......@@ -19,6 +18,7 @@ namespace ErrorCodes
extern const int TOO_DEEP_AST;
}
#if 0
/// Attach additional semantic info to generated select.
struct AppendSemanticVisitorData
{
......@@ -35,6 +35,7 @@ struct AppendSemanticVisitorData
done = true;
}
};
#endif
/// Replaces one table element with pair.
struct RewriteTablesVisitorData
......@@ -124,7 +125,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast
if (!left)
return;
SemanticSelectQuery::hideNames(select, hidden_names, subquery_name);
//SemanticSelectQuery::hideNames(select, hidden_names, subquery_name);
}
select.tables = std::make_shared<ASTTablesInSelectQuery>();
......@@ -135,11 +136,15 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast
data.done = true;
}
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTSelectQuery & select, ASTPtr ast_left, ASTPtr ast_right, const String & subquery_alias)
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTSelectQuery &, ASTPtr ast_left, ASTPtr ast_right, const String & subquery_alias)
{
#if 0
using RewriteMatcher = LinkedMatcher<
OneTypeMatcher<RewriteTablesVisitorData>,
OneTypeMatcher<AppendSemanticVisitorData>>;
#else
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
#endif
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
auto left = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_left.get());
......@@ -160,8 +165,12 @@ ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTSelectQuery & select, ASTP
if (!res)
throw Exception("Cannot parse rewrite query", ErrorCodes::LOGICAL_ERROR);
#if 0
RewriteVisitor::Data visitor_data =
std::make_pair<RewriteTablesVisitorData, AppendSemanticVisitorData>({ast_left, ast_right}, {select.semantic});
#else
RewriteVisitor::Data visitor_data{ast_left, ast_right};
#endif
RewriteVisitor(visitor_data).visit(res);
return res;
}
......
......@@ -133,8 +133,12 @@ void PredicateExpressionsOptimizer::getDependenciesAndQualifiedOfExpression(cons
{
if (const auto identifier = typeid_cast<ASTIdentifier *>(expression.get()))
{
if (!identifier->children.empty())
dependencies_and_qualified.emplace_back(std::pair(identifier, expression->getAliasOrColumnName()));
String table_alias;
if (!identifier->name_parts.empty())
{
if (!tables_with_aliases.empty())
table_alias = tables_with_aliases[0].getQualifiedNamePrefix();
}
else
{
size_t best_table_pos = 0;
......@@ -153,9 +157,11 @@ void PredicateExpressionsOptimizer::getDependenciesAndQualifiedOfExpression(cons
}
}
String qualified_name = tables_with_aliases[best_table_pos].getQualifiedNamePrefix() + expression->getAliasOrColumnName();
dependencies_and_qualified.emplace_back(std::pair(identifier, qualified_name));
table_alias = tables_with_aliases[best_table_pos].getQualifiedNamePrefix();
}
String qualified_name = table_alias + expression->getAliasOrColumnName();
dependencies_and_qualified.emplace_back(std::pair(identifier, qualified_name));
}
else
{
......@@ -356,31 +362,17 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
if (qualified_asterisk->children.size() != 1)
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
ASTIdentifier * ident = typeid_cast<ASTIdentifier *>(qualified_asterisk->children[0].get());
if (!ident)
throw Exception("Logical error: qualified asterisk must have identifier as its child", ErrorCodes::LOGICAL_ERROR);
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
size_t num_components = ident->children.size();
if (num_components > 2)
throw Exception("Qualified asterisk cannot have more than two qualifiers", ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
for (auto it = tables_expression.begin(); it != tables_expression.end(); ++it)
for (auto it = tables_expression.begin(); it != tables_expression.end();)
{
const ASTTableExpression * table_expression = *it;
DatabaseAndTableWithAlias database_and_table_with_alias(*table_expression, context.getCurrentDatabase());
/// database.table.*
if (num_components == 2 && !database_and_table_with_alias.database.empty()
&& static_cast<const ASTIdentifier &>(*ident->children[0]).name == database_and_table_with_alias.database
&& static_cast<const ASTIdentifier &>(*ident->children[1]).name == database_and_table_with_alias.table)
continue;
/// table.* or alias.*
else if (num_components == 0
&& ((!database_and_table_with_alias.table.empty() && ident->name == database_and_table_with_alias.table)
|| (!database_and_table_with_alias.alias.empty() && ident->name == database_and_table_with_alias.alias)))
continue;
if (ident_db_and_name.satisfies(database_and_table_with_alias, true))
++it;
else
/// It's not a required table
tables_expression.erase(it);
it = tables_expression.erase(it); /// It's not a required table
}
}
......
......@@ -168,18 +168,11 @@ void QueryNormalizer::visit(ASTExpressionList & node, const ASTPtr &, Data & dat
}
else if (const auto * qualified_asterisk = typeid_cast<const ASTQualifiedAsterisk *>(child.get()))
{
const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(qualified_asterisk->children[0].get());
size_t num_components = identifier->children.size();
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
for (const auto & [table_name, table_columns] : tables_with_columns)
{
if ((num_components == 2 /// database.table.*
&& !table_name.database.empty() /// This is normal (not a temporary) table.
&& static_cast<const ASTIdentifier &>(*identifier->children[0]).name == table_name.database
&& static_cast<const ASTIdentifier &>(*identifier->children[1]).name == table_name.table)
|| (num_components == 0 /// t.*
&& ((!table_name.table.empty() && identifier->name == table_name.table) /// table.*
|| (!table_name.alias.empty() && identifier->name == table_name.alias)))) /// alias.*
if (ident_db_and_name.satisfies(table_name, true))
{
for (const auto & column_name : table_columns)
node.children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
......
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTSelectQuery.h>
namespace DB
{
/// Additional information for ASTSelectQuery
class SemanticSelectQuery : public ISemantic
{
public:
SemanticPtr clone() const override { return std::make_shared<SemanticSelectQuery>(*this); }
std::vector<String> getPossibleNames(const String & name) const
{
std::vector<String> res;
res.push_back(name);
for (auto it = hidings.find(name); it != hidings.end(); it = hidings.find(it->second))
res.push_back(it->second);
return res;
}
static void hideNames(ASTSelectQuery & select, const std::vector<String> & hidden, const String & new_name)
{
if (!select.semantic)
select.semantic = std::make_shared<SemanticSelectQuery>();
auto & sema = static_cast<SemanticSelectQuery &>(*select.semantic);
sema.hideNames(hidden, new_name);
}
private:
std::unordered_map<String, String> hidings;
void hideNames(const std::vector<String> & hidden, const String & new_name)
{
for (auto & name : hidden)
hidings.emplace(name, new_name);
}
};
}
......@@ -18,14 +18,14 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
/// A simple or compound identifier?
if (children.size() > 1)
if (name_parts.size() > 1)
{
for (size_t i = 0, size = children.size(); i < size; ++i)
for (size_t i = 0, size = name_parts.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << '.';
format_element(static_cast<const ASTIdentifier &>(*children[i].get()).name);
format_element(name_parts[i]);
}
}
else
......@@ -44,11 +44,7 @@ ASTPtr createTableIdentifier(const String & database_name, const String & table_
if (database_name.empty())
return ASTIdentifier::createSpecial(table_name);
ASTPtr database = ASTIdentifier::createSpecial(database_name);
ASTPtr table = ASTIdentifier::createSpecial(table_name);
ASTPtr database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name);
database_and_table->children = {database, table};
ASTPtr database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name, {database_name, table_name});
return database_and_table;
}
......@@ -121,30 +117,30 @@ void addIdentifierQualifier(ASTIdentifier & identifier, const String & database,
{
if (!alias.empty())
{
identifier.children.emplace_back(std::make_shared<ASTIdentifier>(alias));
identifier.name_parts.emplace_back(alias);
}
else
{
if (!database.empty())
identifier.children.emplace_back(std::make_shared<ASTIdentifier>(database));
identifier.children.emplace_back(std::make_shared<ASTIdentifier>(table));
identifier.name_parts.emplace_back(database);
identifier.name_parts.emplace_back(table);
}
}
bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table)
{
size_t num_components = identifier.children.size();
size_t num_components = identifier.name_parts.size();
if (num_components >= 3)
return *getIdentifierName(identifier.children[0]) == database &&
*getIdentifierName(identifier.children[1]) == table;
return identifier.name_parts[0] == database &&
identifier.name_parts[1] == table;
return false;
}
bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table)
{
size_t num_components = identifier.children.size();
size_t num_components = identifier.name_parts.size();
if (num_components >= 2)
return *getIdentifierName(identifier.children[0]) == table;
return identifier.name_parts[0] == table;
return false;
}
......
......@@ -12,11 +12,14 @@ namespace DB
class ASTIdentifier : public ASTWithAlias
{
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.
/// The composite identifier will have a concatenated name (of the form a.b.c),
/// and individual components will be available inside the name_parts.
String name;
std::vector<String> name_parts;
ASTIdentifier(const String & name_)
ASTIdentifier(const String & name_, std::vector<String> && name_parts_ = {})
: name(name_)
, name_parts(name_parts_)
, special(false)
{
range = StringRange(name.data(), name.data() + name.size());
......@@ -37,11 +40,13 @@ protected:
void appendColumnNameImpl(WriteBuffer & ostr) const override;
private:
using ASTWithAlias::children; /// ASTIdentifier is child free
bool special; /// TODO: it would be ptr to semantic here
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name_)
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name, std::vector<String> && name_parts = {})
{
auto ret = std::make_shared<ASTIdentifier>(name_);
auto ret = std::make_shared<ASTIdentifier>(name, std::move(name_parts));
ret->special = true;
return ret;
}
......
......@@ -51,8 +51,6 @@ ASTPtr ASTSelectQuery::clone() const
#undef CLONE
if (semantic)
res->semantic = semantic->clone();
return res;
}
......
......@@ -169,19 +169,19 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
return false;
String name;
std::vector<String> parts;
const ASTExpressionList & list = static_cast<const ASTExpressionList &>(*id_list.get());
for (const auto & child : list.children)
{
if (!name.empty())
name += '.';
name += *getIdentifierName(child);
parts.emplace_back(*getIdentifierName(child));
name += parts.back();
}
node = std::make_shared<ASTIdentifier>(name);
/// In `children`, remember the identifiers-components, if there are more than one.
if (list.children.size() > 1)
node->children.insert(node->children.end(), list.children.begin(), list.children.end());
if (parts.size() == 1)
parts.clear();
node = std::make_shared<ASTIdentifier>(name, std::move(parts));
return true;
}
......
......@@ -31,20 +31,6 @@ class IAST;
using ASTPtr = std::shared_ptr<IAST>;
using ASTs = std::vector<ASTPtr>;
class ISemantic;
using SemanticPtr = std::shared_ptr<ISemantic>;
/// Interfase to set additional information to IAST. Derived classes should be named according to their AST nodes' types:
/// ASTIdentifier => SemanticIdentifer, ASTSome => SemanticSome, ...
class ISemantic
{
public:
virtual ~ISemantic() = default;
ISemantic() = default;
ISemantic(const ISemantic &) = default;
virtual SemanticPtr clone() const = 0;
};
class WriteBuffer;
......@@ -58,7 +44,6 @@ public:
/// This pointer does not allow it to be deleted while the range refers to it.
StringPtr owned_string;
SemanticPtr semantic;
virtual ~IAST() = default;
IAST() = default;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册