#include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int AMBIGUOUS_COLUMN_NAME; extern const int NOT_IMPLEMENTED; extern const int UNKNOWN_IDENTIFIER; } namespace { /// @note we use `--` prefix for unique short names and `--.` for subqueries. /// It expects that user do not use names starting with `--` and column names starting with dot. ASTPtr makeSubqueryTemplate(const String & table_alias) { ParserTablesInSelectQueryElement parser(true); String query_template = "(select * from _t)"; if (!table_alias.empty()) query_template += " as " + table_alias; ASTPtr subquery_template = parseQuery(parser, query_template, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); if (!subquery_template) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot parse subquery template"); return subquery_template; } ASTPtr makeSubqueryQualifiedAsterisk() { auto asterisk = std::make_shared(); asterisk->qualifier = std::make_shared("--.s"); asterisk->children.push_back(asterisk->qualifier); return asterisk; } /// Replace asterisks in select_expression_list with column identifiers class ExtractAsterisksMatcher { public: struct Data { std::unordered_map table_columns; std::unordered_map table_name_alias; std::vector tables_order; std::shared_ptr new_select_expression_list; explicit Data(const std::vector & tables) { tables_order.reserve(tables.size()); for (const auto & table : tables) { String table_name = table.table.getQualifiedNamePrefix(false); NamesAndTypesList columns = table.columns; tables_order.push_back(table_name); table_name_alias.emplace(table.table.table /* table_name */, table_name /* alias_name */); table_columns.emplace(std::move(table_name), std::move(columns)); } } using ShouldAddColumnPredicate = std::function; /// Add columns from table with table_name into select expression list /// Use should_add_column_predicate for check if column name should be added /// By default should_add_column_predicate returns true for any column name void addTableColumns( const String & table_name, ASTs & columns, ShouldAddColumnPredicate should_add_column_predicate = [](const String &) { return true; }) { String name = table_name; auto it = table_columns.find(name); if (it == table_columns.end()) { auto table_name_it = table_name_alias.find(table_name); if (table_name_it != table_name_alias.end()) { name = table_name_it->second; it = table_columns.find(table_name_it->second); if (it == table_columns.end()) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown qualified identifier: {}", table_name); } else throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown qualified identifier: {}", table_name); } for (const auto & column : it->second) { if (should_add_column_predicate(column.name)) { ASTPtr identifier; if (it->first.empty()) /// We want tables from JOIN to have aliases. /// But it is possible to set joined_subquery_requires_alias = 0, /// and write a query like `select * FROM (SELECT 1), (SELECT 1), (SELECT 1)`. /// If so, table name will be empty here. /// /// We cannot create compound identifier with empty part (there is an assert). /// So, try our luck and use only column name. /// (Rewriting AST for JOIN is not an efficient design). identifier = std::make_shared(column.name); else identifier = std::make_shared(std::vector{it->first, column.name}); columns.emplace_back(std::move(identifier)); } } } }; static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return false; } static void visit(const ASTPtr & ast, Data & data) { if (auto * t = ast->as()) visit(*t, ast, data); } private: static void visit(const ASTExpressionList & node, const ASTPtr &, Data & data) { bool has_asterisks = false; data.new_select_expression_list = std::make_shared(); data.new_select_expression_list->children.reserve(node.children.size()); for (const auto & child : node.children) { ASTs columns; if (const auto * asterisk = child->as()) { has_asterisks = true; for (auto & table_name : data.tables_order) data.addTableColumns(table_name, columns); if (asterisk->transformers) { for (const auto & transformer : asterisk->transformers->children) IASTColumnsTransformer::transform(transformer, columns); } } else if (const auto * qualified_asterisk = child->as()) { has_asterisks = true; if (!qualified_asterisk->qualifier) throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must have a qualifier"); auto & identifier = qualified_asterisk->qualifier->as(); data.addTableColumns(identifier.name(), columns); if (qualified_asterisk->transformers) { for (const auto & transformer : qualified_asterisk->transformers->children) { if (transformer->as() || transformer->as() || transformer->as()) IASTColumnsTransformer::transform(transformer, columns); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must only have children of IASTColumnsTransformer type"); } } } else if (const auto * columns_list_matcher = child->as()) { has_asterisks = true; for (const auto & ident : columns_list_matcher->column_list->children) columns.emplace_back(ident->clone()); if (columns_list_matcher->transformers) { for (const auto & transformer : columns_list_matcher->transformers->children) IASTColumnsTransformer::transform(transformer, columns); } } else if (const auto * columns_regexp_matcher = child->as()) { has_asterisks = true; for (auto & table_name : data.tables_order) data.addTableColumns( table_name, columns, [&](const String & column_name) { return columns_regexp_matcher->isColumnMatching(column_name); }); if (columns_regexp_matcher->transformers) { for (const auto & transformer : columns_regexp_matcher->transformers->children) IASTColumnsTransformer::transform(transformer, columns); } } else data.new_select_expression_list->children.push_back(child); data.new_select_expression_list->children.insert( data.new_select_expression_list->children.end(), std::make_move_iterator(columns.begin()), std::make_move_iterator(columns.end())); } if (!has_asterisks) data.new_select_expression_list.reset(); } }; /// Replaces table elements with pair. struct RewriteTablesVisitorData { using TypeToVisit = ASTTablesInSelectQuery; ASTs new_tables; bool done = false; /// @note Do not change ASTTablesInSelectQuery itself. No need to change select.tables. void visit(ASTTablesInSelectQuery &, ASTPtr & ast) { if (done) return; ast->children.swap(new_tables); done = true; } }; bool needRewrite(ASTSelectQuery & select, std::vector & table_expressions) { if (!select.tables()) return false; const auto * tables = select.tables()->as(); if (!tables) return false; size_t num_tables = tables->children.size(); if (num_tables <= 2) return false; size_t num_array_join = 0; table_expressions.reserve(num_tables); for (size_t i = 0; i < num_tables; ++i) { const auto * table = tables->children[i]->as(); if (!table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expected"); if (table->table_expression) if (const auto * expression = table->table_expression->as()) table_expressions.push_back(expression); if (!i) continue; if (!table->table_join && !table->array_join) throw Exception(ErrorCodes::LOGICAL_ERROR, "Joined table expected"); if (table->array_join) { ++num_array_join; continue; } const auto & join = table->table_join->as(); if (join.kind == JoinKind::Comma) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query"); } if (num_tables - num_array_join <= 2) return false; /// it's not trivial to support mix of JOIN ON & JOIN USING cause of short names if (num_array_join) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Multiple JOIN does not support mix with ARRAY JOINs"); return true; } using RewriteMatcher = OneTypeMatcher; using RewriteVisitor = InDepthNodeVisitor; using ExtractAsterisksVisitor = ConstInDepthNodeVisitor; /// V2 specific visitors struct CollectColumnIdentifiersMatcher { using Visitor = ConstInDepthNodeVisitor; struct Data { std::vector & identifiers; std::vector> ignored; explicit Data(std::vector & identifiers_) : identifiers(identifiers_) {} void addIdentifier(const ASTIdentifier & ident) { for (const auto & aliases : ignored) if (aliases.contains(ident.name())) return; identifiers.push_back(const_cast(&ident)); } void pushIgnored(const Names & names) { ignored.emplace_back(std::unordered_set(names.begin(), names.end())); } void popIgnored() { ignored.pop_back(); } }; static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { /// "lambda" visit children itself. if (const auto * f = node->as()) if (f->name == "lambda") return false; /// Do not go into subqueries. Do not collect table identifiers. Do not get identifier from 't.*'. return !node->as() && !node->as() && !node->as(); } static void visit(const ASTPtr & ast, Data & data) { if (auto * t = ast->as()) visit(*t, ast, data); else if (auto * f = ast->as()) visit(*f, ast, data); } static void visit(const ASTIdentifier & ident, const ASTPtr &, Data & data) { data.addIdentifier(ident); } static void visit(const ASTFunction & func, const ASTPtr &, Data & data) { if (func.name == "lambda") { data.pushIgnored(RequiredSourceColumnsMatcher::extractNamesFromLambda(func)); Visitor(data).visit(func.arguments->children[1]); data.popIgnored(); } } }; using CollectColumnIdentifiersVisitor = CollectColumnIdentifiersMatcher::Visitor; struct CheckAliasDependencyVisitorData { using TypeToVisit = ASTIdentifier; const Aliases & aliases; const ASTIdentifier * dependency = nullptr; void visit(ASTIdentifier & ident, ASTPtr &) { if (!dependency && aliases.contains(ident.name())) dependency = &ident; } }; using CheckAliasDependencyMatcher = OneTypeMatcher; using CheckAliasDependencyVisitor = InDepthNodeVisitor; struct RewriteWithAliasMatcher { using Data = std::unordered_map; static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { return !node->as(); } static void visit(ASTPtr & ast, Data & data) { String alias = ast->tryGetAlias(); if (!alias.empty()) { auto it = data.find(alias); if (it != data.end() && it->second.get() == ast.get()) ast = std::make_shared(alias); } } }; using RewriteWithAliasVisitor = InDepthNodeVisitor; class SubqueryExpressionsRewriteMatcher { public: struct Data { ASTPtr expression_list; bool done = false; }; static bool needChildVisit(ASTPtr & node, ASTPtr &) { return !node->as(); } static void visit(ASTPtr & ast, Data & data) { if (auto * t = ast->as()) visit(*t, ast, data); } private: static void visit(ASTSelectQuery & select, ASTPtr &, Data & data) { if (!data.done) { if (data.expression_list->children.empty()) data.expression_list->children.emplace_back(std::make_shared()); select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(data.expression_list)); } data.done = true; } }; using SubqueryExpressionsRewriteVisitor = InDepthNodeVisitor; struct TableNeededColumns { const DatabaseAndTableWithAlias & table; NameSet no_clashes = {}; NameSet alias_clashes = {}; std::unordered_map column_clashes = {}; void fillExpressionList(ASTExpressionList & expression_list) const { size_t columns_count = no_clashes.size() + column_clashes.size() + alias_clashes.size(); expression_list.children.reserve(expression_list.children.size() + columns_count); String table_name = table.getQualifiedNamePrefix(false); for (const auto & column : no_clashes) addShortName(column, expression_list); for (const auto & column : alias_clashes) addShortName(column, expression_list); for (const auto & [column, alias] : column_clashes) addAliasedName(table_name, column, alias, expression_list); } static void addShortName(const String & column, ASTExpressionList & expression_list) { auto ident = std::make_shared(column); expression_list.children.emplace_back(std::move(ident)); } /// t.x as `some` static void addAliasedName(const String & table, const String & column, const String & alias, ASTExpressionList & expression_list) { std::vector name_parts; if (!table.empty()) name_parts.push_back(table); name_parts.push_back(column); auto ident = std::make_shared(std::move(name_parts)); ident->setAlias(alias); expression_list.children.emplace_back(std::move(ident)); } }; class UniqueShortNames { public: /// We know that long names are unique (do not clashes with others). /// So we could make unique names base on this knolage by adding some unused prefix. /// Add a heading underscore to make unique names valid for `isValidIdentifierBegin` static constexpr const char * pattern = "_--"; String longToShort(const String & long_name) { auto it = long_to_short.find(long_name); if (it != long_to_short.end()) return it->second; String short_name = generateUniqueName(long_name); long_to_short.emplace(long_name, short_name); return short_name; } private: std::unordered_map long_to_short; static String generateUniqueName(const String & long_name) { return String(pattern) + long_name; } }; size_t countTablesWithColumn(const std::vector & tables, const String & short_name) { size_t count = 0; for (const auto & table : tables) if (table.hasColumn(short_name)) ++count; return count; } /// 'select `--t.x`, `--t.x`, ...' -> 'select `--t.x` as `t.x`, `t.x`, ...' void restoreName(ASTIdentifier & ident, const String & original_name, NameSet & restored_names) { if (!ident.tryGetAlias().empty()) return; if (original_name.empty()) return; if (!restored_names.contains(original_name)) { ident.setAlias(original_name); restored_names.emplace(original_name); } else { ident.setShortName(original_name); } } /// Find clashes and normalize names /// 1. If column name has no clashes make all its occurrences short: 'table.column' -> 'column', 'table_alias.column' -> 'column'. /// 2. If column name can't be short cause of alias with same name generate and use unique name for it. /// 3. If column clashes with another column generate and use unique names for them. /// 4. If column clashes with another column and it's short - it's 'ambiguous column' error. /// 5. If column clashes with alias add short column name to select list. It would be removed later if not needed. std::vector normalizeColumnNamesExtractNeeded( const std::vector & tables, const Aliases & aliases, const std::vector & identifiers, const std::unordered_set & public_identifiers, UniqueShortNames & unique_names) { std::vector needed_columns; needed_columns.reserve(tables.size()); for (const auto & table : tables) needed_columns.push_back(TableNeededColumns{table.table}); NameSet restored_names; for (ASTIdentifier * ident : identifiers) { bool got_alias = aliases.contains(ident->name()); bool allow_ambiguous = got_alias; /// allow ambiguous column overridden by an alias if (auto table_pos = IdentifierSemantic::chooseTableColumnMatch(*ident, tables, allow_ambiguous)) { if (!ident->isShort()) { if (got_alias) { auto alias = aliases.find(ident->name())->second; auto alias_ident = alias->clone(); if (auto * alias_ident_typed = alias_ident->as()) { alias_ident_typed->restoreTable(); bool alias_equals_column_name = alias_ident->getColumnNameWithoutAlias() == ident->getColumnNameWithoutAlias(); if (!alias_equals_column_name) throw Exception(ErrorCodes::AMBIGUOUS_COLUMN_NAME, "Alias clashes with qualified column '{}'", ident->name()); } } String short_name = ident->shortName(); String original_long_name; if (public_identifiers.contains(ident)) original_long_name = ident->name(); size_t count = countTablesWithColumn(tables, short_name); const auto & table = tables[*table_pos]; /// isValidIdentifierBegin retuired to be consistent with TableJoin::deduplicateAndQualifyColumnNames if (count > 1 || aliases.contains(short_name) || !isValidIdentifierBegin(short_name.at(0))) { IdentifierSemantic::setColumnLongName(*ident, table.table); /// table.column -> table_alias.column const auto & unique_long_name = ident->name(); /// For tables moved into subselects we need unique short names for clashed names String unique_short_name = unique_names.longToShort(unique_long_name); ident->setShortName(unique_short_name); needed_columns[*table_pos].column_clashes.emplace(short_name, unique_short_name); } else { if (!table.hasColumn(short_name)) { throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "There's no column '{}' in table '{}'", ident->name(), table.table.getQualifiedNamePrefix(false)); } ident->setShortName(short_name); /// table.column -> column needed_columns[*table_pos].no_clashes.emplace(short_name); } restoreName(*ident, original_long_name, restored_names); } else if (got_alias) { String short_name = ident->shortName(); if (!isValidIdentifierBegin(short_name.at(0))) { String original_long_name; if (public_identifiers.contains(ident)) original_long_name = ident->name(); const auto & table = tables[*table_pos]; IdentifierSemantic::setColumnLongName(*ident, table.table); /// table.column -> table_alias.column const auto & unique_long_name = ident->name(); String unique_short_name = unique_names.longToShort(unique_long_name); ident->setShortName(unique_short_name); needed_columns[*table_pos].column_clashes.emplace(short_name, unique_short_name); restoreName(*ident, original_long_name, restored_names); } else needed_columns[*table_pos].alias_clashes.emplace(ident->shortName()); } else { String short_name = ident->shortName(); if (!isValidIdentifierBegin(short_name.at(0))) { String original_long_name; if (public_identifiers.contains(ident)) original_long_name = ident->name(); const auto & table = tables[*table_pos]; IdentifierSemantic::setColumnLongName(*ident, table.table); /// table.column -> table_alias.column const auto & unique_long_name = ident->name(); String unique_short_name = unique_names.longToShort(unique_long_name); ident->setShortName(unique_short_name); needed_columns[*table_pos].column_clashes.emplace(short_name, unique_short_name); restoreName(*ident, original_long_name, restored_names); } else needed_columns[*table_pos].no_clashes.emplace(ident->shortName()); } } } return needed_columns; } /// Make expression list for current subselect std::shared_ptr subqueryExpressionList( size_t table_pos, const std::vector & needed_columns, const std::vector> & alias_pushdown) { auto expression_list = std::make_shared(); /// First time extract needed left table columns manually. /// Next times extract left table columns via QualifiedAsterisk: `--s`.* if (table_pos == 1) needed_columns[0].fillExpressionList(*expression_list); else expression_list->children.emplace_back(makeSubqueryQualifiedAsterisk()); /// Add needed right table columns needed_columns[table_pos].fillExpressionList(*expression_list); for (const auto & expr : alias_pushdown[table_pos]) expression_list->children.emplace_back(expr); return expression_list; } } /// namelesspace bool JoinToSubqueryTransformMatcher::needChildVisit(ASTPtr & node, const ASTPtr &) { return !node->as(); } void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) { if (auto * t = ast->as()) visit(*t, ast, data); } /// The reason for V2: not to alias columns without clashes. /// It allows USING and 'select *' for queries with subselects. It doesn't need AsterisksSemantic and related stuff. /// 1. Expand asterisks in select expression list. /// 2. Normalize column names and find name clashes /// 3. Rewrite multiple JOINs with subqueries: /// SELECT ... FROM (SELECT `--.s`.*, ... FROM (...) AS `--.s` JOIN tableY ON ...) AS `--.s` JOIN tableZ ON ...' /// 4. Push down expressions of aliases used in ON section into expression list of first reletad subquery void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data) { std::vector table_expressions; if (!needRewrite(select, table_expressions)) return; auto & src_tables = select.tables()->children; size_t tables_count = src_tables.size(); if (table_expressions.size() != data.tables.size() || tables_count != data.tables.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent tables count in JOIN rewriter"); /// Replace * and t.* with columns in select expression list. { ExtractAsterisksVisitor::Data asterisks_data(data.tables); ExtractAsterisksVisitor(asterisks_data).visit(select.select()); if (asterisks_data.new_select_expression_list) select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(asterisks_data.new_select_expression_list)); } /// Collect column identifiers std::vector identifiers; CollectColumnIdentifiersVisitor::Data data_identifiers(identifiers); CollectColumnIdentifiersVisitor(data_identifiers).visit(ast); std::vector using_identifiers; std::vector> alias_pushdown(tables_count); std::unordered_map on_aliases; /// Collect columns from JOIN sections. Detect if we have aliases there (they need pushdown). for (size_t table_pos = 0; table_pos < tables_count; ++table_pos) { auto * table = src_tables[table_pos]->as(); if (table->table_join) { auto & join = table->table_join->as(); if (join.on_expression) { std::vector on_identifiers; CollectColumnIdentifiersVisitor::Data data_on_identifiers(on_identifiers); CollectColumnIdentifiersVisitor(data_on_identifiers).visit(join.on_expression); identifiers.insert(identifiers.end(), on_identifiers.begin(), on_identifiers.end()); /// Extract aliases used in ON section for pushdown. Exclude the last table. if (table_pos < tables_count - 1) { for (auto * ident : on_identifiers) { auto it = data.aliases.find(ident->name()); if (!on_aliases.contains(ident->name()) && it != data.aliases.end()) { auto alias_expression = it->second; alias_pushdown[table_pos].push_back(alias_expression); on_aliases[ident->name()] = alias_expression; } } } } else if (join.using_expression_list) { CollectColumnIdentifiersVisitor::Data data_using_identifiers(using_identifiers); CollectColumnIdentifiersVisitor(data_using_identifiers).visit(join.using_expression_list); } } } /// Check if alias expression is too complex to push it down. for (auto & expr : on_aliases) { CheckAliasDependencyVisitor::Data check{data.aliases}; CheckAliasDependencyVisitor(check).visit(expr.second); if (check.dependency) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot rewrite JOINs. " "Alias '{}' used in ON section depends on another alias '{}'", expr.first, check.dependency->name()); } /// Check same name in aliases, USING and ON sections. Cannot push down alias to ON through USING cause of name masquerading. for (auto * ident : using_identifiers) if (on_aliases.contains(ident->name())) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot rewrite JOINs. Alias '{}' appears both in ON and USING", ident->name()); using_identifiers.clear(); /// Replace pushdowned expressions with aliases names in original expression lists. RewriteWithAliasVisitor(on_aliases).visit(ast); on_aliases.clear(); /// We need to know if identifier is public. If so we have too keep its output name. std::unordered_set public_identifiers; for (auto & top_level_child : select.select()->children) if (auto * ident = top_level_child->as()) { if (!data.try_to_keep_original_names || startsWith(ident->name(), UniqueShortNames::pattern)) public_identifiers.insert(ident); } UniqueShortNames unique_names; std::vector needed_columns = normalizeColumnNamesExtractNeeded(data.tables, data.aliases, identifiers, public_identifiers, unique_names); /// Rewrite JOINs with subselects ASTPtr left_table = src_tables[0]; static ASTPtr subquery_template = makeSubqueryTemplate("`--.s`"); for (size_t i = 1; i < src_tables.size() - 1; ++i) { auto expression_list = subqueryExpressionList(i, needed_columns, alias_pushdown); ASTPtr subquery = subquery_template->clone(); SubqueryExpressionsRewriteVisitor::Data expr_rewrite_data{std::move(expression_list)}; SubqueryExpressionsRewriteVisitor(expr_rewrite_data).visit(subquery); left_table = replaceJoin(left_table, src_tables[i], subquery); } // expand the last table into a subselect, to resolve alias clashes inside it static ASTPtr last_select_template = makeSubqueryTemplate("`--.t`"); auto last_select = last_select_template->clone(); { auto expression_list = std::make_shared(); needed_columns[src_tables.size() - 1].fillExpressionList(*expression_list); SubqueryExpressionsRewriteVisitor::Data expr_rewrite_data{std::move(expression_list)}; SubqueryExpressionsRewriteVisitor(expr_rewrite_data).visit(last_select); // move ASTTableJoin out of subquery auto * last_table_elem = src_tables.back()->as(); auto * last_select_elem = last_select->as(); if (!last_table_elem || !last_select_elem) throw Exception(ErrorCodes::LOGICAL_ERROR, "Two TablesInSelectQueryElements expected"); if (!last_table_elem->table_join) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table join expected"); last_select_elem->table_join = std::move(last_table_elem->table_join); last_select_elem->children.emplace_back(last_select_elem->table_join); last_table_elem->children.erase( std::remove(last_table_elem->children.begin(), last_table_elem->children.end(), last_select_elem->table_join), last_table_elem->children.end()); RewriteVisitor::Data visitor_data{{src_tables.back()}}; RewriteVisitor(visitor_data).visit(last_select); } RewriteVisitor::Data visitor_data{{left_table, last_select}}; RewriteVisitor(visitor_data).visit(select.refTables()); data.done = true; } ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right, ASTPtr subquery_template) { const auto * left = ast_left->as(); const auto * right = ast_right->as(); if (!left || !right) throw Exception(ErrorCodes::LOGICAL_ERROR, "Two TablesInSelectQueryElements expected"); if (!right->table_join) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table join expected"); /// replace '_t' with pair of joined tables RewriteVisitor::Data visitor_data{{ast_left, ast_right}}; RewriteVisitor(visitor_data).visit(subquery_template); return subquery_template; } }