Skip to content

Commit

Permalink
Merge pull request #57662 from ClickHouse/vdimir/analyzer_join_on_tru…
Browse files Browse the repository at this point in the history
…e_nullable

Analyzer: Fix JOIN ON true with join_use_nulls
  • Loading branch information
vdimir committed Dec 11, 2023
2 parents e443567 + f3fada0 commit efb82a9
Show file tree
Hide file tree
Showing 3 changed files with 43 additions and 14 deletions.
28 changes: 14 additions & 14 deletions src/Planner/PlannerJoinTree.cpp
Expand Up @@ -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,
Expand Down Expand Up @@ -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)
Expand All @@ -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))
{
Expand All @@ -1124,6 +1112,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_

auto table_join = std::make_shared<TableJoin>(settings, query_context->getGlobalTemporaryVolume());
table_join->getTableJoin() = join_node.toASTTableJoin()->as<ASTTableJoin &>();

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)
Expand Down
4 changes: 4 additions & 0 deletions tests/queries/0_stateless/02000_join_on_const.reference
Expand Up @@ -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)
25 changes: 25 additions & 0 deletions tests/queries/0_stateless/02000_join_on_const.sql
Expand Up @@ -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;

0 comments on commit efb82a9

Please sign in to comment.