mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #43141 from kitaisreal/analyzer-join-with-constants-improvements
Analyzer improve JOIN with constants
This commit is contained in:
commit
ab5e1ab650
@ -227,7 +227,11 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node,
|
||||
JoinClausesAndActions join_clauses_and_actions;
|
||||
JoinKind join_kind = join_node.getKind();
|
||||
|
||||
auto join_constant = tryExtractConstantFromJoinNode(join_tree_node);
|
||||
std::optional<bool> join_constant;
|
||||
|
||||
if (join_node.getStrictness() == JoinStrictness::All)
|
||||
join_constant = tryExtractConstantFromJoinNode(join_tree_node);
|
||||
|
||||
if (join_constant)
|
||||
{
|
||||
/** If there is JOIN with always true constant, we transform it to cross.
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Functions/FunctionsConversion.h>
|
||||
#include <Functions/CastOverloadResolver.h>
|
||||
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/TableNode.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
#include <Analyzer/JoinNode.h>
|
||||
@ -76,6 +77,23 @@ void JoinClause::dump(WriteBuffer & buffer) const
|
||||
|
||||
if (!right_filter_condition_nodes.empty())
|
||||
buffer << " right_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes);
|
||||
|
||||
if (!asof_conditions.empty())
|
||||
{
|
||||
buffer << " asof_conditions: ";
|
||||
size_t asof_conditions_size = asof_conditions.size();
|
||||
|
||||
for (size_t i = 0; i < asof_conditions_size; ++i)
|
||||
{
|
||||
const auto & asof_condition = asof_conditions[i];
|
||||
|
||||
buffer << "key_index: " << asof_condition.key_index;
|
||||
buffer << "inequality: " << toString(asof_condition.asof_inequality);
|
||||
|
||||
if (i + 1 != asof_conditions_size)
|
||||
buffer << ',';
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String JoinClause::dump() const
|
||||
@ -249,9 +267,7 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag,
|
||||
join_node);
|
||||
|
||||
if (!expression_side_optional)
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
"JOIN {} with constants is not supported",
|
||||
join_node.formatASTForErrorMessage());
|
||||
expression_side_optional = JoinTableSide::Right;
|
||||
|
||||
auto expression_side = *expression_side_optional;
|
||||
join_clause.addCondition(expression_side, join_expressions_actions_node);
|
||||
@ -277,6 +293,22 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
for (const auto & node : join_expression_actions_nodes)
|
||||
join_expression_dag_input_nodes.insert(&node);
|
||||
|
||||
auto * function_node = join_node.getJoinExpression()->as<FunctionNode>();
|
||||
if (!function_node)
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
"JOIN {} join expression expected function",
|
||||
join_node.formatASTForErrorMessage());
|
||||
|
||||
/** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction.
|
||||
* If we do not ignore it, this function will be replaced by underlying constant.
|
||||
* For example ASOF JOIN does not support JOIN with constants, and we should process it like ordinary JOIN.
|
||||
*
|
||||
* Example: SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2
|
||||
* ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value);
|
||||
*/
|
||||
auto constant_value = function_node->getConstantValueOrNull();
|
||||
function_node->performConstantFolding({});
|
||||
|
||||
PlannerActionsVisitor join_expression_visitor(planner_context);
|
||||
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_node.getJoinExpression());
|
||||
if (join_expression_dag_node_raw_pointers.size() != 1)
|
||||
@ -284,6 +316,8 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
"JOIN {} ON clause contains multiple expressions",
|
||||
join_node.formatASTForErrorMessage());
|
||||
|
||||
function_node->performConstantFolding(std::move(constant_value));
|
||||
|
||||
const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0];
|
||||
if (!join_expressions_actions_root_node->function)
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
|
@ -0,0 +1,15 @@
|
||||
1 1
|
||||
--
|
||||
--
|
||||
--
|
||||
1 2
|
||||
--
|
||||
1 1 1 1
|
||||
--
|
||||
1 1 0 0
|
||||
--
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
@ -0,0 +1,27 @@
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 1 AS id) AS t2 ON t1.id = t2.id AND 1;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 2 AS id) AS t2 ON t1.id = t2.id AND 1;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 1 AS id) AS t2 ON t1.id = t2.id AND 0;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 2 AS id) AS t2 ON t1.id = t2.id OR 1;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2 ON (t1.id = t2.id) AND 1 == 1 AND (t1.value >= t2.value);
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2 ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t2.value);
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt);
|
Loading…
Reference in New Issue
Block a user