JoinToSubqueryTransform with short names

This commit is contained in:
chertus 2019-02-20 15:12:36 +03:00
parent 1fc6c60668
commit 48ee3e83fa
8 changed files with 300 additions and 101 deletions

View File

@ -0,0 +1,38 @@
#pragma once
#include <unordered_map>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
namespace DB
{
struct AsteriskSemanticImpl
{
using RevertedAliases = std::unordered_map<String, std::vector<String>>;
using RevertedAliasesPtr = std::shared_ptr<RevertedAliases>;
RevertedAliasesPtr aliases; /// map of aliases that should be set in phase of * expanding.
};
struct AsteriskSemantic
{
using RevertedAliases = AsteriskSemanticImpl::RevertedAliases;
using RevertedAliasesPtr = AsteriskSemanticImpl::RevertedAliasesPtr;
static void setAliases(ASTAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
static void setAliases(ASTQualifiedAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
static RevertedAliasesPtr getAliases(const ASTAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
static RevertedAliasesPtr getAliases(const ASTQualifiedAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
private:
static std::shared_ptr<AsteriskSemanticImpl> makeSemantic(const RevertedAliasesPtr & aliases)
{
return std::make_shared<AsteriskSemanticImpl>(AsteriskSemanticImpl{aliases});
}
};
}

View File

@ -126,15 +126,18 @@ void IdentifierSemantic::setColumnNormalName(ASTIdentifier & identifier, const D
identifier.semantic->can_be_alias = false; identifier.semantic->can_be_alias = false;
if (identifier.semantic->need_long_name) if (identifier.semantic->need_long_name)
setColumnLongName(identifier, db_and_table);
}
void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
String prefix = db_and_table.getQualifiedNamePrefix();
if (!prefix.empty())
{ {
String prefix = db_and_table.getQualifiedNamePrefix(); String short_name = identifier.shortName();
if (!prefix.empty()) identifier.name = prefix + short_name;
{ prefix.resize(prefix.size() - 1); /// crop dot
String short_name = identifier.shortName(); identifier.name_parts = {prefix, short_name};
identifier.name = prefix + short_name;
prefix.resize(prefix.size() - 1); /// crop dot
identifier.name_parts = {prefix, short_name};
}
} }
} }
@ -145,4 +148,9 @@ String IdentifierSemantic::columnNormalName(const ASTIdentifier & identifier, co
return copy->getAliasOrColumnName(); return copy->getAliasOrColumnName();
} }
String IdentifierSemantic::columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
return db_and_table.getQualifiedNamePrefix() + identifier.shortName();
}
} }

View File

@ -28,7 +28,9 @@ struct IdentifierSemantic
static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setNeedLongName(ASTIdentifier & identifier, bool); /// if set setColumnNormalName makes qualified name static void setNeedLongName(ASTIdentifier & identifier, bool); /// if set setColumnNormalName makes qualified name
static bool canBeAlias(const ASTIdentifier & identifier); static bool canBeAlias(const ASTIdentifier & identifier);
static void setMembership(ASTIdentifier & identifier, size_t table_no); static void setMembership(ASTIdentifier & identifier, size_t table_no);

View File

@ -1,5 +1,8 @@
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Interpreters/JoinToSubqueryTransformVisitor.h> #include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/AsteriskSemantic.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
@ -9,6 +12,7 @@
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
namespace DB namespace DB
{ {
@ -16,48 +20,143 @@ namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int TOO_DEEP_AST; extern const int TOO_DEEP_AST;
extern const int AMBIGUOUS_COLUMN_NAME;
} }
#if 0 namespace
/// Attach additional semantic info to generated select. {
/// Find columns with aliases to push them into rewritten subselects.
/// Normalize table aliases: table_name.column_name -> table_alias.column_name
/// Make aliases maps (alias -> column_name, column_name -> alias)
struct ColumnAliasesVisitorData
{
using TypeToVisit = ASTIdentifier;
const std::vector<DatabaseAndTableWithAlias> tables;
AsteriskSemantic::RevertedAliases rev_aliases;
std::unordered_map<String, String> aliases;
std::vector<ASTIdentifier *> short_identifiers;
std::vector<ASTIdentifier *> compound_identifiers;
ColumnAliasesVisitorData(std::vector<DatabaseAndTableWithAlias> && tables_)
: tables(tables_)
{}
void visit(ASTIdentifier & node, ASTPtr &)
{
if (node.isShort())
{
short_identifiers.push_back(&node);
return;
}
bool last_table = false;
String long_name;
for (auto & table : tables)
{
if (IdentifierSemantic::canReferColumnToTable(node, table))
{
if (!long_name.empty())
throw Exception("Cannot refer column '" + node.name + "' to one table", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name
long_name = node.name;
if (&table == &tables.back())
last_table = true;
}
}
if (long_name.empty())
throw Exception("Cannot refer column '" + node.name + "' to table", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
String alias = node.tryGetAlias();
if (!alias.empty())
{
aliases[alias] = long_name;
rev_aliases[long_name].push_back(alias);
if (!last_table)
{
node.setShortName(alias);
node.setAlias("");
}
}
else
compound_identifiers.push_back(&node);
}
void replaceIdentifiersWithAliases()
{
for (auto * identifier : short_identifiers)
if (!aliases.count(identifier->name))
throw Exception("Short column name '" + identifier->name + "' is not an alias", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
for (auto * identifier : compound_identifiers)
{
auto it = rev_aliases.find(identifier->name);
if (it == rev_aliases.end())
{
bool last_table = IdentifierSemantic::canReferColumnToTable(*identifier, tables.back());
if (!last_table)
throw Exception("Column name without alias '" + identifier->name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
}
else
{
if (it->second.empty())
throw Exception("No alias for '" + identifier->name + "'", ErrorCodes::LOGICAL_ERROR);
identifier->setShortName(it->second[0]);
}
}
}
};
/// Attach additional semantic info to generated selects.
struct AppendSemanticVisitorData struct AppendSemanticVisitorData
{ {
using TypeToVisit = ASTSelectQuery; using TypeToVisit = ASTSelectQuery;
const SemanticPtr & semantic; AsteriskSemantic::RevertedAliasesPtr rev_aliases = {};
bool done = false; bool done = false;
void visit(ASTSelectQuery & select, ASTPtr &) void visit(ASTSelectQuery & select, ASTPtr &)
{ {
if (done || !semantic) if (done || !rev_aliases || !select.select_expression_list)
return; return;
select.semantic = semantic->clone();
for (auto & child : select.select_expression_list->children)
{
if (auto * node = typeid_cast<ASTAsterisk *>(child.get()))
AsteriskSemantic::setAliases(*node, rev_aliases);
if (auto * node = typeid_cast<ASTQualifiedAsterisk *>(child.get()))
AsteriskSemantic::setAliases(*node, rev_aliases);
}
done = true; done = true;
} }
}; };
#endif
/// Replaces one table element with pair.
/// Replaces table elements with pair.
struct RewriteTablesVisitorData struct RewriteTablesVisitorData
{ {
using TypeToVisit = ASTTablesInSelectQuery; using TypeToVisit = ASTTablesInSelectQuery;
const ASTPtr & left; ASTPtr left;
const ASTPtr & right; ASTPtr right;
bool done = false; bool done = false;
/// @note Do not change ASTTablesInSelectQuery itself. No need to change select.tables.
void visit(ASTTablesInSelectQuery &, ASTPtr & ast) void visit(ASTTablesInSelectQuery &, ASTPtr & ast)
{ {
if (done) if (done)
return; return;
ast->children.clear(); std::vector<ASTPtr> new_tables{left, right};
ast->children.push_back(left); ast->children.swap(new_tables);
ast->children.push_back(right);
done = true; done = true;
} }
}; };
static bool needRewrite(ASTSelectQuery & select) bool needRewrite(ASTSelectQuery & select)
{ {
if (!select.tables) if (!select.tables)
return false; return false;
@ -70,28 +169,32 @@ static bool needRewrite(ASTSelectQuery & select)
if (num_tables <= 2) if (num_tables <= 2)
return false; return false;
for (size_t i = 1; i < tables->children.size(); ++i)
{
auto table = typeid_cast<const ASTTablesInSelectQueryElement *>(tables->children[i].get());
if (!table || !table->table_join)
throw Exception("Multiple JOIN expects joined tables", ErrorCodes::LOGICAL_ERROR);
auto join = typeid_cast<const ASTTableJoin *>(table->table_join.get());
if (join->kind == ASTTableJoin::Kind::Comma)
throw Exception("Multiple COMMA JOIN is not supported", ErrorCodes::LOGICAL_ERROR);
/// it's not trivial to support mix of JOIN ON & JOIN USING cause of short names
if (!join || !join->on_expression)
throw Exception("Multiple JOIN expects JOIN with ON section", ErrorCodes::LOGICAL_ERROR);
}
return true; return true;
} }
static void appendTableNameAndAlias(std::vector<String> & hidden, const ASTPtr & table_element) using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
{ using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
auto element = typeid_cast<const ASTTablesInSelectQueryElement *>(table_element.get()); using ColumnAliasesMatcher = OneTypeMatcher<ColumnAliasesVisitorData>;
if (!element || element->children.empty()) using ColumnAliasesVisitor = InDepthNodeVisitor<ColumnAliasesMatcher, true>;
throw Exception("Expected TablesInSelectQueryElement with at least one child", ErrorCodes::LOGICAL_ERROR); using AppendSemanticMatcher = OneTypeMatcher<AppendSemanticVisitorData>;
using AppendSemanticVisitor = InDepthNodeVisitor<AppendSemanticMatcher, true>;
auto table_expression = typeid_cast<const ASTTableExpression *>(element->children[0].get()); } /// namelesspace
if (!table_expression || table_expression->children.empty())
throw Exception("Expected TableExpression with at least one child", ErrorCodes::LOGICAL_ERROR);
String alias = table_expression->children[0]->tryGetAlias();
if (!alias.empty())
hidden.push_back(alias);
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);
}
std::vector<ASTPtr *> JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) std::vector<ASTPtr *> JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
@ -101,79 +204,84 @@ std::vector<ASTPtr *> JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data &
return {}; return {};
} }
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data) void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & data)
{ {
static String alias_prefix = "__join"; /// FIXME using RevertedAliases = AsteriskSemantic::RevertedAliases;
if (!needRewrite(select)) if (!needRewrite(select))
return; return;
auto tables = typeid_cast<const ASTTablesInSelectQuery *>(select.tables.get()); ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, ""));
if (!tables) if (select.select_expression_list)
throw Exception("TablesInSelectQuery expected", ErrorCodes::LOGICAL_ERROR); ColumnAliasesVisitor(aliases_data).visit(select.select_expression_list);
if (select.where_expression)
ColumnAliasesVisitor(aliases_data).visit(select.where_expression);
if (select.prewhere_expression)
ColumnAliasesVisitor(aliases_data).visit(select.prewhere_expression);
if (select.having_expression)
ColumnAliasesVisitor(aliases_data).visit(select.having_expression);
size_t num_tables = tables->children.size(); /// JOIN sections
ASTPtr left = tables->children[0]; for (auto & child : select.tables->children)
for (size_t i = 1; i < num_tables - 1; ++i)
{ {
ASTPtr right = tables->children[i]; auto table = typeid_cast<ASTTablesInSelectQueryElement *>(child.get());
std::vector<String> hidden_names; if (table->table_join)
appendTableNameAndAlias(hidden_names, left); {
appendTableNameAndAlias(hidden_names, right); auto * join = typeid_cast<ASTTableJoin *>(table->table_join.get());
ColumnAliasesVisitor(aliases_data).visit(join->on_expression);
String subquery_name = alias_prefix + toString(i); }
left = replaceJoin(select, left, right, subquery_name);
if (!left)
return;
//SemanticSelectQuery::hideNames(select, hidden_names, subquery_name);
} }
select.tables = std::make_shared<ASTTablesInSelectQuery>(); aliases_data.replaceIdentifiersWithAliases();
select.tables->children.push_back(left);
select.tables->children.push_back(tables->children.back()); auto rev_aliases = std::make_shared<RevertedAliases>();
rev_aliases->swap(aliases_data.rev_aliases);
auto & src_tables = select.tables->children;
ASTPtr left_table = src_tables[0];
for (size_t i = 1; i < src_tables.size() - 1; ++i)
{
left_table = replaceJoin(left_table, src_tables[i]);
if (!left_table)
throw Exception("Cannot replace tables with subselect", ErrorCodes::LOGICAL_ERROR);
/// attach data to generated asterisk
AppendSemanticVisitor::Data semantic_data{rev_aliases, false};
AppendSemanticVisitor(semantic_data).visit(left_table);
}
/// replace tables in select with generated two-table join
RewriteVisitor::Data visitor_data{left_table, src_tables.back()};
RewriteVisitor(visitor_data).visit(select.tables);
ast = ast->clone(); /// rewrite AST in right manner
data.done = true; data.done = true;
} }
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTSelectQuery &, ASTPtr ast_left, ASTPtr ast_right, const String & subquery_alias) static ASTPtr makeSubqueryTemplate()
{ {
#if 0 ParserTablesInSelectQueryElement parser(true);
using RewriteMatcher = LinkedMatcher< ASTPtr subquery_template = parseQuery(parser, "(select * from _t)", 0);
OneTypeMatcher<RewriteTablesVisitorData>, if (!subquery_template)
OneTypeMatcher<AppendSemanticVisitorData>>; throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR);
#else return subquery_template;
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>; }
#endif
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right)
{
auto left = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_left.get()); auto left = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_left.get());
auto right = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_right.get()); auto right = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_right.get());
if (!left || !right) if (!left || !right)
throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR); throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR);
if (!right->table_join || right->array_join) if (!right->table_join)
return {}; throw Exception("Table join expected", ErrorCodes::LOGICAL_ERROR);
auto table_join = typeid_cast<const ASTTableJoin *>(right->table_join.get()); static ASTPtr subquery_template = makeSubqueryTemplate();
if (table_join->kind != ASTTableJoin::Kind::Inner)
return {};
ParserTablesInSelectQueryElement parser(true); /// replace '_t' with pair of joined tables
String subquery = "(select * from _t) as " + subquery_alias; ASTPtr res = subquery_template->clone();
ASTPtr res = parseQuery(parser, subquery, 0);
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}; RewriteVisitor::Data visitor_data{ast_left, ast_right};
#endif
RewriteVisitor(visitor_data).visit(res); RewriteVisitor(visitor_data).visit(res);
return res; return res;
} }

View File

@ -42,7 +42,7 @@ private:
static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data); static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data);
/// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite /// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite
static ASTPtr replaceJoin(ASTSelectQuery & select, ASTPtr left, ASTPtr right, const String & subquery_alias); static ASTPtr replaceJoin(ASTPtr left, ASTPtr right);
}; };
using JoinToSubqueryTransformVisitor = InDepthNodeVisitor<JoinToSubqueryTransformMatcher, true>; using JoinToSubqueryTransformVisitor = InDepthNodeVisitor<JoinToSubqueryTransformMatcher, true>;

View File

@ -2,6 +2,7 @@
#include <Interpreters/TranslateQualifiedNamesVisitor.h> #include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/IdentifierSemantic.h> #include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/AsteriskSemantic.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Core/Names.h> #include <Core/Names.h>
@ -141,6 +142,32 @@ std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & sel
return out; return out;
} }
/// qualifed names for duplicates
static std::shared_ptr<ASTIdentifier> makeIdentifier(const String & short_name, const String & long_name, bool need_long_name)
{
if (need_long_name)
return std::make_shared<ASTIdentifier>(long_name);
return std::make_shared<ASTIdentifier>(short_name);
}
static void addIdentifier(ASTs & nodes, std::shared_ptr<ASTIdentifier> identifier, const String & long_name,
AsteriskSemantic::RevertedAliasesPtr aliases)
{
bool added = false;
if (aliases && aliases->count(long_name))
{
for (const String & alias : (*aliases)[long_name])
{
nodes.push_back(identifier->clone());
nodes.back()->setAlias(alias);
added = true;
}
}
if (!added)
nodes.emplace_back(identifier);
}
/// Replace *, alias.*, database.table.* with a list of columns. /// Replace *, alias.*, database.table.* with a list of columns.
void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPtr &, Data & data) void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPtr &, Data & data)
{ {
@ -177,20 +204,21 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
for (const auto & child : old_children) for (const auto & child : old_children)
{ {
if (typeid_cast<const ASTAsterisk *>(child.get())) if (const auto * asterisk = typeid_cast<const ASTAsterisk *>(child.get()))
{ {
bool first_table = true; bool first_table = true;
for (const auto & [table_name, table_columns] : tables_with_columns) for (const auto & [table_name, table_columns] : tables_with_columns)
{ {
for (const auto & column_name : table_columns) for (const auto & column_name : table_columns)
{
if (first_table || !data.join_using_columns.count(column_name)) if (first_table || !data.join_using_columns.count(column_name))
{ {
/// qualifed names for duplicates bool need_prefix = !first_table && source_columns.count(column_name);
if (!first_table && source_columns.count(column_name)) String long_name = table_name.getQualifiedNamePrefix() + column_name;
node.children.emplace_back(std::make_shared<ASTIdentifier>(table_name.getQualifiedNamePrefix() + column_name)); auto identifier = makeIdentifier(column_name, long_name, need_prefix);
else addIdentifier(node.children, identifier, long_name, AsteriskSemantic::getAliases(*asterisk));
node.children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
} }
}
first_table = false; first_table = false;
} }
@ -206,11 +234,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
{ {
for (const auto & column_name : table_columns) for (const auto & column_name : table_columns)
{ {
/// qualifed names for duplicates bool need_prefix = !first_table && source_columns.count(column_name);
if (!first_table && source_columns.count(column_name)) String long_name = table_name.getQualifiedNamePrefix() + column_name;
node.children.emplace_back(std::make_shared<ASTIdentifier>(table_name.getQualifiedNamePrefix() + column_name)); auto identifier = makeIdentifier(column_name, long_name, need_prefix);
else addIdentifier(node.children, identifier, long_name, AsteriskSemantic::getAliases(*qualified_asterisk));
node.children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
} }
break; break;
} }

View File

@ -6,6 +6,9 @@
namespace DB namespace DB
{ {
struct AsteriskSemantic;
struct AsteriskSemanticImpl;
class ASTAsterisk : public IAST class ASTAsterisk : public IAST
{ {
public: public:
@ -15,6 +18,11 @@ public:
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private:
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
friend struct AsteriskSemantic;
}; };
} }

View File

@ -6,6 +6,9 @@
namespace DB namespace DB
{ {
struct AsteriskSemantic;
struct AsteriskSemanticImpl;
/** Something like t.* /** Something like t.*
* It will have qualifier as its child ASTIdentifier. * It will have qualifier as its child ASTIdentifier.
*/ */
@ -23,6 +26,11 @@ public:
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
private:
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
friend struct AsteriskSemantic;
}; };
} }