From 0f6a6453fee773d4d38d3f85ba6b360958fc9435 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 30 Sep 2020 23:11:49 +0300 Subject: [PATCH] Remove multiple_joins_rewriter v1 (#15472) --- src/Core/Settings.h | 2 +- src/Interpreters/AsteriskSemantic.h | 41 --- src/Interpreters/InterpreterSelectQuery.cpp | 11 +- .../JoinToSubqueryTransformVisitor.cpp | 261 +--------------- .../JoinToSubqueryTransformVisitor.h | 6 +- .../TranslateQualifiedNamesVisitor.cpp | 25 +- src/Parsers/ASTAsterisk.h | 8 - src/Parsers/ASTQualifiedAsterisk.h | 8 - .../00847_multiple_join_same_column.reference | 12 - .../00847_multiple_join_same_column.sql | 22 -- .../00849_multiple_comma_join.reference | 292 ------------------ .../0_stateless/00849_multiple_comma_join.sql | 69 ----- .../00849_multiple_comma_join_2.sql | 1 - .../00854_multiple_join_asterisks.sql | 8 - .../01116_cross_count_asterisks.reference | 2 - .../01116_cross_count_asterisks.sql | 17 - ...rewrite_with_ambiguous_column_and_view.sql | 1 - ...multiple_joins_rewriter_v2_and_lambdas.sql | 2 - 18 files changed, 14 insertions(+), 774 deletions(-) delete mode 100644 src/Interpreters/AsteriskSemantic.h delete mode 100644 tests/queries/0_stateless/00849_multiple_comma_join.reference delete mode 100644 tests/queries/0_stateless/00849_multiple_comma_join.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 19738d826c6..bc974433073 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -368,7 +368,6 @@ class IColumn; \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ - M(UInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \ @@ -399,6 +398,7 @@ class IColumn; M(UInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ M(Bool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \ M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ + M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing. Will be removed after 2021-03-31", 0) \ \ M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ diff --git a/src/Interpreters/AsteriskSemantic.h b/src/Interpreters/AsteriskSemantic.h deleted file mode 100644 index 1bd9ecedddd..00000000000 --- a/src/Interpreters/AsteriskSemantic.h +++ /dev/null @@ -1,41 +0,0 @@ -#pragma once - -#include - -#include -#include -#include - -namespace DB -{ - -struct AsteriskSemanticImpl -{ - using RevertedAliases = std::unordered_map>; - using RevertedAliasesPtr = std::shared_ptr; - - 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 void setAliases(ASTColumnsMatcher & 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; } - static RevertedAliasesPtr getAliases(const ASTColumnsMatcher & node) { return node.semantic ? node.semantic->aliases : nullptr; } - -private: - static std::shared_ptr makeSemantic(const RevertedAliasesPtr & aliases) - { - return std::make_shared(AsteriskSemanticImpl{aliases}); - } -}; - -} diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 823808759a2..556070d0360 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -93,7 +93,6 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_WITH_FILL_EXPRESSION; - extern const int INVALID_SETTING_VALUE; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -190,7 +189,7 @@ static Context getSubqueryContext(const Context & context) return subquery_context; } -static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings) +static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database) { ASTSelectQuery & select = query->as(); @@ -202,11 +201,7 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table CrossToInnerJoinVisitor::Data cross_to_inner{tables, aliases, database}; CrossToInnerJoinVisitor(cross_to_inner).visit(query); - size_t rewriter_version = settings.multiple_joins_rewriter_version; - if (!rewriter_version || rewriter_version > 2) - throw Exception("Bad multiple_joins_rewriter_version setting value: " + settings.multiple_joins_rewriter_version.toString(), - ErrorCodes::INVALID_SETTING_VALUE); - JoinToSubqueryTransformVisitor::Data join_to_subs_data{tables, aliases, rewriter_version}; + JoinToSubqueryTransformVisitor::Data join_to_subs_data{tables, aliases}; JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query); } @@ -271,7 +266,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Rewrite JOINs if (!has_input && joined_tables.tablesCount() > 1) { - rewriteMultipleJoins(query_ptr, joined_tables.tablesWithColumns(), context->getCurrentDatabase(), settings); + rewriteMultipleJoins(query_ptr, joined_tables.tablesWithColumns(), context->getCurrentDatabase()); joined_tables.reset(getSelectQuery()); joined_tables.resolveTables(); diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 5f38f410e04..cdd7ec3ebf9 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include @@ -11,6 +10,8 @@ #include #include #include +#include +#include #include #include #include @@ -127,169 +128,6 @@ private: } }; -/// 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 ColumnAliasesMatcher -{ - using Visitor = ConstInDepthNodeVisitor; - - struct Data - { - const std::vector tables; - bool public_names; - AsteriskSemantic::RevertedAliases rev_aliases; /// long_name -> aliases - std::unordered_map aliases; /// alias -> long_name - std::vector> compound_identifiers; - std::set allowed_long_names; /// original names allowed as aliases '--t.x as t.x' (select expressions only). - bool inside_function = false; - - explicit Data(const std::vector && tables_) - : tables(tables_) - , public_names(false) - {} - - void replaceIdentifiersWithAliases() - { - String hide_prefix = "--"; /// @note restriction: user should not use aliases like `--table.column` - - for (auto & [identifier, is_public] : compound_identifiers) - { - String long_name = identifier->name; - - auto it = rev_aliases.find(long_name); - if (it == rev_aliases.end()) - { - bool last_table = false; - { - if (auto best_table_pos = IdentifierSemantic::chooseTable(*identifier, tables)) - last_table = (*best_table_pos + 1 == tables.size()); - } - - if (!last_table) - { - String alias = hide_prefix + long_name; - aliases[alias] = long_name; - rev_aliases[long_name].push_back(alias); - - IdentifierSemantic::coverName(*identifier, alias); - if (is_public) - { - identifier->setAlias(long_name); - allowed_long_names.insert(long_name); - } - } - else if (is_public) - identifier->setAlias(long_name); /// prevent crop long to short name - } - else - { - if (it->second.empty()) - throw Exception("No alias for '" + long_name + "'", ErrorCodes::LOGICAL_ERROR); - - if (is_public && allowed_long_names.count(long_name)) - ; /// leave original name unchanged for correct output - else - IdentifierSemantic::coverName(*identifier, it->second[0]); - } - } - } - }; - - static bool needChildVisit(const ASTPtr & node, const ASTPtr &) - { - /// Do not go into subqueries. Function visits children itself. - if (node->as() || - node->as()) - return false; - return !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); - - /// Do not allow asterisks but ignore them inside functions. I.e. allow 'count(*)'. - if (!data.inside_function && (ast->as() || ast->as())) - throw Exception("Multiple JOIN do not support asterisks for complex queries yet", ErrorCodes::NOT_IMPLEMENTED); - } - - static void visit(const ASTFunction &, const ASTPtr & ast, Data & data) - { - /// Grandchild case: Function -> (ExpressionList) -> Asterisk - data.inside_function = true; - Visitor visitor(data); - for (auto & child : ast->children) - visitor.visit(child); - data.inside_function = false; - } - - static void visit(const ASTIdentifier & const_node, const ASTPtr &, Data & data) - { - ASTIdentifier & node = const_cast(const_node); /// we know it's not const - if (node.isShort()) - return; - - bool last_table = false; - String long_name; - - if (auto table_pos = IdentifierSemantic::chooseTable(node, data.tables)) - { - const auto & table = data.tables[*table_pos]; - IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name - long_name = node.name; - if (&table == &data.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()) - { - data.aliases[alias] = long_name; - data.rev_aliases[long_name].push_back(alias); - - if (!last_table) - { - IdentifierSemantic::coverName(node, alias); - node.setAlias({}); - } - } - else if (node.compound()) - data.compound_identifiers.emplace_back(&node, data.public_names); - } -}; - -/// Attach additional semantic info to generated selects. -struct AppendSemanticVisitorData -{ - using TypeToVisit = ASTSelectQuery; - - AsteriskSemantic::RevertedAliasesPtr rev_aliases = {}; - bool done = false; - - void visit(ASTSelectQuery & select, ASTPtr &) - { - if (done || !rev_aliases || !select.select()) - return; - - for (auto & child : select.select()->children) - { - if (auto * node = child->as()) - AsteriskSemantic::setAliases(*node, rev_aliases); - if (auto * node = child->as()) - AsteriskSemantic::setAliases(*node, rev_aliases); - } - - done = true; - } -}; - /// Replaces table elements with pair. struct RewriteTablesVisitorData { @@ -371,9 +209,6 @@ bool needRewrite(ASTSelectQuery & select, std::vector; using RewriteVisitor = InDepthNodeVisitor; using ExtractAsterisksVisitor = ConstInDepthNodeVisitor; -using ColumnAliasesVisitor = ColumnAliasesMatcher::Visitor; -using AppendSemanticMatcher = OneTypeMatcher; -using AppendSemanticVisitor = InDepthNodeVisitor; /// V2 specific visitors @@ -718,12 +553,7 @@ bool JoinToSubqueryTransformMatcher::needChildVisit(ASTPtr & node, const ASTPtr void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) { if (auto * t = ast->as()) - { - if (data.version == 1) - visitV1(*t, ast, data); - else - visitV2(*t, ast, data); - } + visit(*t, ast, data); } /// The reason for V2: not to alias columns without clashes. @@ -733,7 +563,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) /// 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::visitV2(ASTSelectQuery & select, ASTPtr & ast, Data & data) +void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data) { std::vector table_expressions; if (!needRewrite<2>(select, table_expressions)) @@ -855,89 +685,6 @@ void JoinToSubqueryTransformMatcher::visitV2(ASTSelectQuery & select, ASTPtr & a data.done = true; } -void JoinToSubqueryTransformMatcher::visitV1(ASTSelectQuery & select, ASTPtr &, Data & data) -{ - using RevertedAliases = AsteriskSemantic::RevertedAliases; - - std::vector table_expressions; - if (!needRewrite(select, table_expressions)) - return; - - if (table_expressions.size() != data.tables.size()) - throw Exception("Inconsistent tables count in JOIN rewriter", ErrorCodes::LOGICAL_ERROR); - - bool has_subquery = false; - for (const auto & expr : table_expressions) - if (expr->subquery) - has_subquery = true; - - if (!has_subquery) - { - 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)); - } - - ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, "")); - if (select.select()) - { - /// TODO: there's a bug here. We need to publish only top-level ASTIdentifiers but visitor extracts all. - aliases_data.public_names = true; - ColumnAliasesVisitor(aliases_data).visit(select.select()); - aliases_data.public_names = false; - } - if (select.where()) - ColumnAliasesVisitor(aliases_data).visit(select.where()); - if (select.prewhere()) - ColumnAliasesVisitor(aliases_data).visit(select.prewhere()); - if (select.orderBy()) - ColumnAliasesVisitor(aliases_data).visit(select.orderBy()); - if (select.groupBy()) - ColumnAliasesVisitor(aliases_data).visit(select.groupBy()); - if (select.having()) - ColumnAliasesVisitor(aliases_data).visit(select.having()); - - /// JOIN sections - for (auto & child : select.tables()->children) - { - auto * table = child->as(); - if (table->table_join) - { - auto & join = table->table_join->as(); - if (join.on_expression) - ColumnAliasesVisitor(aliases_data).visit(join.on_expression); - } - } - - aliases_data.replaceIdentifiersWithAliases(); - - auto rev_aliases = std::make_shared(); - rev_aliases->swap(aliases_data.rev_aliases); - - auto & src_tables = select.tables()->children; - ASTPtr left_table = src_tables[0]; - - static ASTPtr subquery_template = makeSubqueryTemplate(); - - for (size_t i = 1; i < src_tables.size() - 1; ++i) - { - left_table = replaceJoin(left_table, src_tables[i], subquery_template->clone()); - 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.refTables()); - - data.done = true; -} - ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right, ASTPtr subquery_template) { const auto * left = ast_left->as(); diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.h b/src/Interpreters/JoinToSubqueryTransformVisitor.h index 643eb19b365..a024a168509 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.h +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.h @@ -20,7 +20,6 @@ public: { const std::vector & tables; const Aliases & aliases; - size_t version = 1; bool done = false; }; @@ -43,10 +42,7 @@ private: /// TablesInSelectQueryElement [source1] /// TablesInSelectQueryElement [source2] /// - static void visitV1(ASTSelectQuery & select, ASTPtr & ast, Data & data); - - /// V2 uses information about tables' columns to rewrite queries. - static void visitV2(ASTSelectQuery & select, ASTPtr & ast, Data & data); + static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data); /// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite static ASTPtr replaceJoin(ASTPtr left, ASTPtr right, ASTPtr subquery_template); diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 74622c72865..9e0b6fdd196 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -174,25 +173,11 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr Visitor(data).visit(select.refHaving()); } -static void addIdentifier(ASTs & nodes, const DatabaseAndTableWithAlias & table, const String & column_name, - AsteriskSemantic::RevertedAliasesPtr aliases) +static void addIdentifier(ASTs & nodes, const DatabaseAndTableWithAlias & table, const String & column_name) { String table_name = table.getQualifiedNamePrefix(false); auto identifier = std::make_shared(std::vector{table_name, column_name}); - - bool added = false; - if (aliases && aliases->count(identifier->name)) - { - for (const String & alias : (*aliases)[identifier->name]) - { - nodes.push_back(identifier->clone()); - nodes.back()->setAlias(alias); - added = true; - } - } - - if (!added) - nodes.emplace_back(identifier); + nodes.emplace_back(identifier); } /// Replace *, alias.*, database.table.* with a list of columns. @@ -237,7 +222,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { if (first_table || !data.join_using_columns.count(column.name)) { - addIdentifier(node.children, table.table, column.name, AsteriskSemantic::getAliases(*asterisk)); + addIdentifier(node.children, table.table, column.name); } } @@ -264,7 +249,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { if (asterisk_pattern->isColumnMatching(column.name) && (first_table || !data.join_using_columns.count(column.name))) { - addIdentifier(node.children, table.table, column.name, AsteriskSemantic::getAliases(*asterisk_pattern)); + addIdentifier(node.children, table.table, column.name); } } @@ -287,7 +272,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { for (const auto & column : table.columns) { - addIdentifier(node.children, table.table, column.name, AsteriskSemantic::getAliases(*qualified_asterisk)); + addIdentifier(node.children, table.table, column.name); } break; } diff --git a/src/Parsers/ASTAsterisk.h b/src/Parsers/ASTAsterisk.h index 9c4c9a2df6d..027758ba48c 100644 --- a/src/Parsers/ASTAsterisk.h +++ b/src/Parsers/ASTAsterisk.h @@ -6,9 +6,6 @@ namespace DB { -struct AsteriskSemantic; -struct AsteriskSemanticImpl; - /** SELECT * is expanded to all visible columns of the source table. * Optional transformers can be attached to further manipulate these expanded columns. */ @@ -21,11 +18,6 @@ public: protected: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; - -private: - std::shared_ptr semantic; /// pimpl - - friend struct AsteriskSemantic; }; } diff --git a/src/Parsers/ASTQualifiedAsterisk.h b/src/Parsers/ASTQualifiedAsterisk.h index 2c3689d0ace..1b644532f53 100644 --- a/src/Parsers/ASTQualifiedAsterisk.h +++ b/src/Parsers/ASTQualifiedAsterisk.h @@ -6,9 +6,6 @@ namespace DB { -struct AsteriskSemantic; -struct AsteriskSemanticImpl; - /** Something like t.* * It will have qualifier as its child ASTIdentifier. * Optional transformers can be attached to further manipulate these expanded columns. @@ -27,11 +24,6 @@ public: protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - -private: - std::shared_ptr semantic; /// pimpl - - friend struct AsteriskSemantic; }; } diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.reference b/tests/queries/0_stateless/00847_multiple_join_same_column.reference index 4e813d5a677..91bd62ca5a3 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.reference +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.reference @@ -31,18 +31,6 @@ y.b: 0 │ 1 │ 1 │ 1 │ │ 2 │ 2 │ 2 │ └─────┴─────┴─────┘ -┌─s.a─┬─s.a─┬─s_b─┬─s_b─┐ -│ 1 │ 1 │ 1 │ 1 │ -│ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┘ -┌─y.a─┬─y.a─┬─y_b─┬─y_b─┐ -│ 1 │ 1 │ 1 │ 1 │ -│ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┘ -┌─t_a─┬─t_a─┬─s_a─┬─s_a─┬─y_a─┬─y_a─┐ -│ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ -│ 2 │ 2 │ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┴─────┴─────┘ ┌─s.a─┬─s.a─┬─s_b─┬─s.b─┐ │ 1 │ 1 │ 1 │ 1 │ │ 0 │ 0 │ 0 │ 0 │ diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/tests/queries/0_stateless/00847_multiple_join_same_column.sql index eae18dba7f3..c7f0c6383c2 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -39,28 +39,6 @@ left join y on y.b = s.b order by t.a format PrettyCompactNoEscapes; -set multiple_joins_rewriter_version = 1; - -select s.a, s.a, s.b as s_b, s.b from t -left join s on s.a = t.a -left join y on s.b = y.b -order by t.a -format PrettyCompactNoEscapes; - -select y.a, y.a, y.b as y_b, y.b from t -left join s on s.a = t.a -left join y on y.b = s.b -order by t.a -format PrettyCompactNoEscapes; - -select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t -left join s on t.a = s.a -left join y on y.b = s.b -order by t.a -format PrettyCompactNoEscapes; - -set multiple_joins_rewriter_version = 2; - select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a left join y on s.b = y.b diff --git a/tests/queries/0_stateless/00849_multiple_comma_join.reference b/tests/queries/0_stateless/00849_multiple_comma_join.reference deleted file mode 100644 index f4db2238dd1..00000000000 --- a/tests/queries/0_stateless/00849_multiple_comma_join.reference +++ /dev/null @@ -1,292 +0,0 @@ -SELECT a -FROM t1_00849 -CROSS JOIN t2_00849 -SELECT a -FROM t1_00849 -ALL INNER JOIN t2_00849 ON a = t2_00849.a -WHERE a = t2_00849.a -SELECT a -FROM t1_00849 -ALL INNER JOIN t2_00849 ON b = t2_00849.b -WHERE b = t2_00849.b -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` -) AS `--.s` -ALL INNER JOIN t3_00849 ON `--t1_00849.a` = a -WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - a AS `--t1_00849.a`, - b AS `--t1_00849.b`, - t2_00849.a, - t2_00849.b AS `--t2_00849.b` - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b` -) AS `--.s` -ALL INNER JOIN t3_00849 ON `--t1_00849.b` = b -WHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` - ) AS `--.s` - ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a` -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t1_00849.a` = a -WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - `--t1_00849.b`, - `t2_00849.a`, - `--t2_00849.b`, - a, - b AS `--t3_00849.b` - FROM - ( - SELECT - a AS `--t1_00849.a`, - b AS `--t1_00849.b`, - t2_00849.a, - t2_00849.b AS `--t2_00849.b` - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b` - ) AS `--.s` - ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b` -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t1_00849.b` = b -WHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a` - ) AS `--.s` - ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a` -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t2_00849.a` = a -WHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 - ) AS `--.s` - ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t3_00849.a` = a -WHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 - ) AS `--.s` - CROSS JOIN t3_00849 -) AS `--.s` -ALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) -WHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` - ) AS `--.s` - ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a` -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t3_00849.a` = a -WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `t2_00849.a`, - `t2_00849.b`, - a, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 - ) AS `--.s` - CROSS JOIN t3_00849 -) AS `--.s` -CROSS JOIN t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `t2_00849.a`, - `t2_00849.b`, - a, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 - ) AS `--.s` - CROSS JOIN t3_00849 -) AS `--.s` -CROSS JOIN t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 -) AS `--.s` -CROSS JOIN t3_00849 -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` -) AS `--.s` -CROSS JOIN t3_00849 -SELECT * FROM t1, t2 -1 1 1 1 -1 1 1 \N -2 2 1 1 -2 2 1 \N -3 3 1 1 -3 3 1 \N -4 4 1 1 -4 4 1 \N -SELECT * FROM t1, t2 WHERE t1.a = t2.a -1 1 1 1 -1 1 1 \N -SELECT t1.a, t2.a FROM t1, t2 WHERE t1.b = t2.b -1 1 -SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a -1 1 1 -1 1 \N -1 \N 1 -1 \N \N -SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b -1 1 1 -SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a -1 1 1 1 -1 1 1 \N -1 1 \N 1 -1 1 \N \N -1 \N 1 1 -1 \N 1 \N -1 \N \N 1 -1 \N \N \N -SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b -1 1 1 1 -SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a -1 1 1 1 -1 1 1 \N -1 1 \N 1 -1 1 \N \N -1 \N 1 1 -1 \N 1 \N -1 \N \N 1 -1 \N \N \N diff --git a/tests/queries/0_stateless/00849_multiple_comma_join.sql b/tests/queries/0_stateless/00849_multiple_comma_join.sql deleted file mode 100644 index 5a8962c2277..00000000000 --- a/tests/queries/0_stateless/00849_multiple_comma_join.sql +++ /dev/null @@ -1,69 +0,0 @@ -SET enable_debug_queries = 1; -SET enable_optimize_predicate_expression = 0; -SET multiple_joins_rewriter_version = 1; - -DROP TABLE IF EXISTS t1_00849; -DROP TABLE IF EXISTS t2_00849; -DROP TABLE IF EXISTS t3_00849; -DROP TABLE IF EXISTS t4_00849; - -CREATE TABLE t1_00849 (a UInt32, b Nullable(Int32)) ENGINE = Memory; -CREATE TABLE t2_00849 (a UInt32, b Nullable(Int32)) ENGINE = Memory; -CREATE TABLE t3_00849 (a UInt32, b Nullable(Int32)) ENGINE = Memory; -CREATE TABLE t4_00849 (a UInt32, b Nullable(Int32)) ENGINE = Memory; - -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849 WHERE t1_00849.a = t2_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849 WHERE t1_00849.b = t2_00849.b; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849 WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849 WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a AND t1_00849.a = t4_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b AND t1_00849.b = t4_00849.b; - -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t2_00849.a = t1_00849.a AND t2_00849.a = t3_00849.a AND t2_00849.a = t4_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t3_00849.a = t1_00849.a AND t3_00849.a = t2_00849.a AND t3_00849.a = t4_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t4_00849.a = t1_00849.a AND t4_00849.a = t2_00849.a AND t4_00849.a = t3_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.a = t2_00849.a AND t2_00849.a = t3_00849.a AND t3_00849.a = t4_00849.a; - -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849; -ANALYZE SELECT t1_00849.a FROM t1_00849 CROSS JOIN t2_00849 CROSS JOIN t3_00849 CROSS JOIN t4_00849; - -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849 CROSS JOIN t3_00849; -ANALYZE SELECT t1_00849.a FROM t1_00849 JOIN t2_00849 USING a CROSS JOIN t3_00849; -- { serverError 48 } -ANALYZE SELECT t1_00849.a FROM t1_00849 JOIN t2_00849 ON t1_00849.a = t2_00849.a CROSS JOIN t3_00849; - -INSERT INTO t1_00849 values (1,1), (2,2), (3,3), (4,4); -INSERT INTO t2_00849 values (1,1), (1, Null); -INSERT INTO t3_00849 values (1,1), (1, Null); -INSERT INTO t4_00849 values (1,1), (1, Null); - -SELECT 'SELECT * FROM t1, t2'; -SELECT * FROM t1_00849, t2_00849 -ORDER BY t1_00849.a, t2_00849.b; -SELECT 'SELECT * FROM t1, t2 WHERE t1.a = t2.a'; -SELECT * FROM t1_00849, t2_00849 WHERE t1_00849.a = t2_00849.a -ORDER BY t1_00849.a, t2_00849.b; -SELECT 'SELECT t1.a, t2.a FROM t1, t2 WHERE t1.b = t2.b'; -SELECT t1_00849.a, t2_00849.b FROM t1_00849, t2_00849 WHERE t1_00849.b = t2_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b FROM t1_00849, t2_00849, t3_00849 -WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a -ORDER BY t2_00849.b, t3_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b FROM t1_00849, t2_00849, t3_00849 WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 -WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a AND t1_00849.a = t4_00849.a -ORDER BY t2_00849.b, t3_00849.b, t4_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 -WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b AND t1_00849.b = t4_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 -WHERE t1_00849.a = t2_00849.a AND t2_00849.a = t3_00849.a AND t3_00849.a = t4_00849.a -ORDER BY t2_00849.b, t3_00849.b, t4_00849.b; - -DROP TABLE t1_00849; -DROP TABLE t2_00849; -DROP TABLE t3_00849; -DROP TABLE t4_00849; diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql index de09bcc0b4d..710327ec0e4 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql +++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql @@ -1,6 +1,5 @@ SET enable_debug_queries = 1; SET enable_optimize_predicate_expression = 0; -SET multiple_joins_rewriter_version = 2; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/00854_multiple_join_asterisks.sql b/tests/queries/0_stateless/00854_multiple_join_asterisks.sql index 43aef45440c..c01ea721000 100644 --- a/tests/queries/0_stateless/00854_multiple_join_asterisks.sql +++ b/tests/queries/0_stateless/00854_multiple_join_asterisks.sql @@ -4,14 +4,6 @@ select t1.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join sy select t2.*, t3.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; select t1.dummy, t2.*, t3.dummy from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; -set multiple_joins_rewriter_version = 1; - -select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1 -join system.one t2 on t1.dummy = t2.dummy -join system.one t3 ON t1.dummy = t3.dummy; -- { serverError 48 } - -set multiple_joins_rewriter_version = 2; - select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; diff --git a/tests/queries/0_stateless/01116_cross_count_asterisks.reference b/tests/queries/0_stateless/01116_cross_count_asterisks.reference index 8347b144a35..5f1d0ecea5d 100644 --- a/tests/queries/0_stateless/01116_cross_count_asterisks.reference +++ b/tests/queries/0_stateless/01116_cross_count_asterisks.reference @@ -1,4 +1,2 @@ 2 1 -2 -1 diff --git a/tests/queries/0_stateless/01116_cross_count_asterisks.sql b/tests/queries/0_stateless/01116_cross_count_asterisks.sql index 1fb8b0b0e66..aa5adaddae5 100644 --- a/tests/queries/0_stateless/01116_cross_count_asterisks.sql +++ b/tests/queries/0_stateless/01116_cross_count_asterisks.sql @@ -1,20 +1,3 @@ -SET multiple_joins_rewriter_version = 2; - -SELECT count(*) -FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 -WHERE (n1.number = n2.number) AND (n2.number = n3.number); - -SELECT count(*) c FROM ( - SELECT count(*), count(*) as c - FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 - WHERE (n1.number = n2.number) AND (n2.number = n3.number) - AND (SELECT count(*) FROM numbers(1)) = 1 -) -WHERE (SELECT count(*) FROM numbers(2)) = 2 -HAVING c IN(SELECT count(*) c FROM numbers(1)); - -SET multiple_joins_rewriter_version = 1; - SELECT count(*) FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 WHERE (n1.number = n2.number) AND (n2.number = n3.number); diff --git a/tests/queries/0_stateless/01144_join_rewrite_with_ambiguous_column_and_view.sql b/tests/queries/0_stateless/01144_join_rewrite_with_ambiguous_column_and_view.sql index c90d01ff76d..ae844888407 100644 --- a/tests/queries/0_stateless/01144_join_rewrite_with_ambiguous_column_and_view.sql +++ b/tests/queries/0_stateless/01144_join_rewrite_with_ambiguous_column_and_view.sql @@ -11,7 +11,6 @@ INSERT INTO t1 (id, value1) VALUES (1, 'val11'); INSERT INTO t2 (id, value2) VALUES (1, 'val21'); INSERT INTO t3 (id, value3) VALUES (1, 'val31'); -SET multiple_joins_rewriter_version = 2; SET enable_optimize_predicate_expression = 1; SELECT t1.id, t2.id as id, t3.id as value diff --git a/tests/queries/0_stateless/01144_multiple_joins_rewriter_v2_and_lambdas.sql b/tests/queries/0_stateless/01144_multiple_joins_rewriter_v2_and_lambdas.sql index daf93649f89..6a5ba042c63 100644 --- a/tests/queries/0_stateless/01144_multiple_joins_rewriter_v2_and_lambdas.sql +++ b/tests/queries/0_stateless/01144_multiple_joins_rewriter_v2_and_lambdas.sql @@ -1,5 +1,3 @@ -set multiple_joins_rewriter_version = 2; - select arrayMap(x, y -> floor((y - x) / x, 3), l, r) diff_percent, test, query