diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 95ea260e727d..abcf971b8320 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -988,18 +988,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_strictness == JoinStrictness::All || join_strictness == JoinStrictness::Semi || join_strictness == JoinStrictness::Anti) join_constant = tryExtractConstantFromJoinNode(join_table_expression); - if (join_constant) - { - /** If there is JOIN with always true constant, we transform it to cross. - * If there is JOIN with always false constant, we do not process JOIN keys. - * It is expected by join algorithm to handle such case. - * - * Example: SELECT * FROM test_table AS t1 INNER JOIN test_table AS t2 ON 1; - */ - if (*join_constant) - join_kind = JoinKind::Cross; - } - else if (join_node.isOnJoinExpression()) + if (!join_constant && join_node.isOnJoinExpression()) { join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, right_plan_output_columns, @@ -1079,7 +1068,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); - bool join_use_nulls = settings.join_use_nulls; auto to_nullable_function = FunctionFactory::instance().get("toNullable", query_context); auto join_cast_plan_columns_to_nullable = [&](QueryPlan & plan_to_add_cast) @@ -1105,7 +1093,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ plan_to_add_cast.addStep(std::move(cast_join_columns_step)); }; - if (join_use_nulls) + if (settings.join_use_nulls) { if (isFull(join_kind)) { @@ -1124,6 +1112,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto table_join = std::make_shared(settings, query_context->getGlobalTemporaryVolume()); table_join->getTableJoin() = join_node.toASTTableJoin()->as(); + + if (join_constant) + { + /** If there is JOIN with always true constant, we transform it to cross. + * If there is JOIN with always false constant, we do not process JOIN keys. + * It is expected by join algorithm to handle such case. + * + * Example: SELECT * FROM test_table AS t1 INNER JOIN test_table AS t2 ON 1; + */ + if (*join_constant) + join_kind = JoinKind::Cross; + } table_join->getTableJoin().kind = join_kind; if (join_kind == JoinKind::Comma) diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index e9d1c685fdd5..848ecedf9e3a 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -65,3 +65,7 @@ SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 1 ('',0) +2 +4 2 Nullable(UInt64) UInt8 +4 2 UInt64 Nullable(UInt8) +4 2 Nullable(UInt64) Nullable(UInt8) diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index 3205c084672b..a68e75443d87 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -90,6 +90,31 @@ SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 -- { echoOff } +SELECT a + 1 +FROM (SELECT 1 as x) as t1 +LEFT JOIN ( SELECT 1 AS a ) AS t2 +ON TRUE +SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; + +SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) +FROM (SELECT 1 as x) as t1 +LEFT JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 +ON TRUE +SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; + +SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) +FROM (SELECT 1 as x) as t1 +RIGHT JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 +ON TRUE +SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; + +SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) +FROM (SELECT 1 as x) as t1 +FULL JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 +ON TRUE +SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; + + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2;