From ca90cd56cc3d006045ff73bdcb4d4787cc852ec1 Mon Sep 17 00:00:00 2001 From: Stepan Stepanishchev <116617250+SteveStevenpoor@users.noreply.github.com> Date: Tue, 24 Feb 2026 23:04:42 +0700 Subject: [PATCH 1/4] [FLINK-38916][table-planner] MultiJoin produces incorrect results for OR join conditions when parallelism is greater than 1 This closes #27498. --- .../planner/plan/utils/MultiJoinUtil.java | 10 +- .../planner/plan/stream/sql/MultiJoinTest.xml | 2480 ++++++++++++++--- 2 files changed, 2097 insertions(+), 393 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java index 83bf9be50c1a3..59a69599381d9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java @@ -71,9 +71,13 @@ private static void extractEqualityConditions( final SqlKind kind = call.getOperator().getKind(); if (kind != SqlKind.EQUALS) { - for (final RexNode operand : call.getOperands()) { - extractEqualityConditions( - operand, inputOffsets, inputFieldCounts, joinAttributeMap); + // Only conjunctions (AND) can contain equality conditions that are valid for multijoin. + // All other condition types are deferred to the postJoinFilter. + if (kind == SqlKind.AND) { + for (final RexNode operand : call.getOperands()) { + extractEqualityConditions( + operand, inputOffsets, inputFieldCounts, joinAttributeMap); + } } return; } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index 56bf1998204f0..3a5e888ca25c4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -17,123 +17,370 @@ limitations under the License. --> + + 5 THEN 1 END) as bulk_orders +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN OrderItems oi + ON o.order_id = oi.order_id +LEFT JOIN ProductCategories pc + ON oi.product_name = pc.category_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +GROUP BY u.user_id, u.name, pc.category_name +HAVING COUNT(DISTINCT o.order_id) > 0]]> + + + ($3, 0)]) ++- LogicalAggregate(group=[{0, 1, 2}], order_count=[COUNT(DISTINCT $3)], total_items=[SUM($4)], total_value=[SUM($5)], avg_item_price=[AVG($6)], max_payment=[MAX($7)], min_payment=[MIN($7)], bulk_orders=[COUNT($8)]) + +- LogicalProject(user_id=[$0], name=[$1], category_name=[$12], order_id=[$3], quantity=[$9], $f5=[*($9, $10)], unit_price=[$10], price=[$15], $f8=[CASE(>($9, 5), 1, null:INTEGER)]) + +- LogicalJoin(condition=[=($0, $16)], joinType=[left]) + :- LogicalJoin(condition=[=($8, $11)], joinType=[left]) + : :- LogicalJoin(condition=[=($3, $7)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, OrderItems]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + (order_count, 0)]) +- GroupAggregate(groupBy=[user_id, name, category_name], select=[user_id, name, category_name, COUNT_RETRACT(DISTINCT order_id) AS order_count, SUM_RETRACT(quantity) AS total_items, SUM_RETRACT($f5) AS total_value, AVG_RETRACT(unit_price) AS avg_item_price, MAX_RETRACT(price) AS max_payment, MIN_RETRACT(price) AS min_payment, COUNT_RETRACT($f8) AS bulk_orders]) +- Exchange(distribution=[hash[user_id, name, category_name]]) +- Calc(select=[user_id, name, category_name, order_id, quantity, *(quantity, unit_price) AS $f5, unit_price, price, CASE(>(quantity, 5), 1, null:INTEGER) AS $f8]) - +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($8, $11), =($0, $16)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:8;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:8;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price,category_id,category_name,parent_category,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id1)], select=[user_id,name,order_id,quantity,unit_price,category_name,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_name, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($3, $7)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;], 1=[LeftInputId:0;LeftFieldIndex:3;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) - : :- Exchange(distribution=[hash[order_id]]) - : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) - : : :- Exchange(distribution=[hash[user_id]]) - : : : +- ChangelogNormalize(key=[user_id]) - : : : +- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[order_id]]) - : +- ChangelogNormalize(key=[item_id]) - : +- Exchange(distribution=[hash[item_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) - :- Exchange(distribution=[hash[category_id]]) - : +- ChangelogNormalize(key=[category_id]) - : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, parent_category]) + : +- Calc(select=[user_id, name, order_id, quantity, unit_price, category_name]) + : +- MultiJoin(commonJoinKey=[product_name], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (category_id)], joinConditions=[=(product_name, category_id)], select=[user_id,name,order_id,product_name,quantity,unit_price,category_id,category_name], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name)]) + : :- Exchange(distribution=[hash[product_name]]) + : : +- Calc(select=[user_id, name, order_id, product_name, quantity, unit_price]) + : : +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) + : : :- Exchange(distribution=[hash[order_id]]) + : : : +- Calc(select=[user_id, name, order_id]) + : : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) + : : : :- Exchange(distribution=[hash[user_id]]) + : : : : +- ChangelogNormalize(key=[user_id]) + : : : : +- Exchange(distribution=[hash[user_id]]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : : +- Exchange(distribution=[hash[order_id]]) + : : +- Calc(select=[order_id, product_name, quantity, unit_price]) + : : +- ChangelogNormalize(key=[item_id]) + : : +- Exchange(distribution=[hash[item_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) + : +- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories, project=[category_id, category_name], metadata=[]]], fields=[category_id, category_name]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) ]]> + + 0]]> + + + ($2, 0)]) ++- LogicalAggregate(group=[{0}], unique_users=[COUNT(DISTINCT $1)], total_sales=[COUNT($2)], total_revenue=[SUM($3)], avg_sale_amount=[AVG($3)], max_sale_amount=[MAX($3)]) + +- LogicalProject(category_name=[$7], user_id=[$0], sale_id=[$10], amount=[$13]) + +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) + :- LogicalJoin(condition=[AND(=($0, $9), =($5, $6))], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Categories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) +]]> + (total_sales, 0)]) +- GroupAggregate(groupBy=[category_name], select=[category_name, COUNT_RETRACT(DISTINCT user_id) AS unique_users, COUNT_RETRACT(sale_id) AS total_sales, SUM_RETRACT(amount) AS total_revenue, AVG_RETRACT(amount) AS avg_sale_amount, MAX_RETRACT(amount) AS max_sale_amount]) +- Exchange(distribution=[hash[category_name]]) +- Calc(select=[category_name, user_id, sale_id, amount]) - +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $9), =($5, $6)), =($0, $11)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:3;, LeftInputId:1;LeftFieldIndex:2;RightInputId:2;RightFieldIndex:0;], 3=[LeftInputId:0;LeftFieldIndex:0;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,category_id,category_name,parent_category,user_id1,sale_id,user_id2,product_id,amount,sale_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) product_id, DOUBLE amount, DATE sale_date)]) + +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, (category_id), (sale_id)], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), =(product, category_id)), =(user_id, user_id2)], select=[user_id,user_id0,product,category_id,category_name,user_id1,sale_id,user_id2,amount], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, DOUBLE amount)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[category_id]) : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Categories]], fields=[category_id, category_name, parent_category, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Categories, project=[category_id, category_name, user_id], metadata=[]]], fields=[category_id, category_name, user_id]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[sale_id]) +- Exchange(distribution=[hash[sale_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, user_id, product_id, amount, sale_date]) + +- TableSourceScan(table=[[default_catalog, default_database, Sales, project=[sale_id, user_id, amount], metadata=[]]], fields=[sale_id, user_id, amount]) ]]> + + 600000 +), +active_projects AS ( + SELECT project_id, project_name, dept_id + FROM Projects + WHERE status = 'ACTIVE' +) +SELECT + u.user_id, + u.name, + o.order_id, + hbd.dept_name, + ap.project_name, + hbd.budget +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN high_budget_depts hbd + ON o.user_id = hbd.dept_id +LEFT JOIN active_projects ap + ON hbd.dept_id = ap.dept_id]]> + + + ($2, 600000)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Departments]]) + +- LogicalProject(project_id=[$0], project_name=[$1], dept_id=[$2]) + +- LogicalFilter(condition=[=($3, _UTF-16LE'ACTIVE')]) + +- LogicalTableScan(table=[[default_catalog, default_database, Projects]]) +]]> + (budget, 600000)]) : +- Exchange(distribution=[hash[dept_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Departments, filter=[]]], fields=[dept_id, dept_name, budget]) +- Exchange(distribution=[hash[dept_id]]) - +- Calc(select=[project_id, project_name, dept_id]) + +- Calc(select=[project_name, dept_id]) +- ChangelogNormalize(key=[project_id], condition=[=(status, 'ACTIVE')]) +- Exchange(distribution=[hash[project_id]]) +- TableSourceScan(table=[[default_catalog, default_database, Projects, filter=[]]], fields=[project_id, project_name, dept_id, status]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1000 THEN 'High-Value Premium' + WHEN pc.is_premium = true THEN 'Premium' + WHEN p.price > 500 THEN 'Standard High-Value' + ELSE 'Standard' + END AS product_tier, + CASE + WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended' + WHEN pr.rating >= 3 THEN 'Recommended' + WHEN pr.rating >= 2 THEN 'Average' + ELSE 'Not Recommended' + END AS recommendation_status, + CASE + WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate) + ELSE p.price + END AS final_price +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +LEFT JOIN ProductCategories pc + ON o.product = pc.category_id +LEFT JOIN ProductReviews pr + ON o.product = pr.product_id]]> + + + ($7, 1000)), _UTF-16LE'High-Value Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", $11, _UTF-16LE'Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Standard High-Value':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Standard':VARCHAR(19) CHARACTER SET "UTF-16LE")], recommendation_status=[CASE(AND(>=($15, 4), $16), _UTF-16LE'Highly Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 3), _UTF-16LE'Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 2), _UTF-16LE'Average':VARCHAR(18) CHARACTER SET "UTF-16LE", _UTF-16LE'Not Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE")], final_price=[CASE(>($12, 0.2:DECIMAL(2, 1)), *($7, -(1, $12)), CAST($7):DOUBLE)]) ++- LogicalJoin(condition=[=($5, $14)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, ProductReviews]]) +]]> + (price, 1000)), 'High-Value Premium', is_premium, 'Premium', >(price, 500), 'Standard High-Value', 'Standard') AS product_tier, CASE(AND(>=(rating, 4), is_verified), 'Highly Recommended', >=(rating, 3), 'Recommended', >=(rating, 2), 'Average', 'Not Recommended') AS recommendation_status, CASE(>(discount_rate, 0.2), *(price, -(1, discount_rate)), CAST(price AS DOUBLE)) AS final_price]) -+- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($5, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:5;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,category_id,category_name,is_premium,discount_rate,review_id,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) ++- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (category_id), noUniqueKey], joinConditions=[=(product, category_id), =(product, product_id)], select=[user_id,order_id,product,payment_id,price,category_id,category_name,is_premium,discount_rate,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) :- Exchange(distribution=[hash[product]]) - : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + : +- Calc(select=[user_id, order_id, product, payment_id, price]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) :- Exchange(distribution=[hash[category_id]]) : +- ChangelogNormalize(key=[category_id]) : +- Exchange(distribution=[hash[category_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, is_premium, discount_rate]) +- Exchange(distribution=[hash[product_id]]) - +- ChangelogNormalize(key=[review_id]) - +- Exchange(distribution=[hash[review_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) + +- Calc(select=[product_id, rating, is_verified]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) ]]> + + 1000 THEN 'Premium' + WHEN bo.price > 500 THEN 'Standard' + ELSE 'Basic' + END as order_tier + FROM base_orders bo + LEFT JOIN OrderMetrics om + ON bo.order_id = om.order_id +), +aggregated_metrics AS ( + SELECT + user_id, + name, + COUNT(DISTINCT order_id) as total_orders, + SUM(price) as total_spent, + AVG(price) as avg_order_value, + MAX(metric_value) as max_metric, + MIN(metric_value) as min_metric, + COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders + FROM enriched_orders + GROUP BY user_id, name +) +SELECT + user_id, + UPPER(name) as user_name, + total_orders, + ROUND(total_spent, 2) as total_spent_rounded, + ROUND(avg_order_value, 2) as avg_order_value_rounded, + CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary, + CASE + WHEN total_orders > 10 THEN 'Frequent Customer' + WHEN total_orders > 5 THEN 'Regular Customer' + ELSE 'Occasional Customer' + END as customer_type +FROM aggregated_metrics +WHERE total_spent > 0]]> + + + ($2, 10), _UTF-16LE'Frequent Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", >($2, 5), _UTF-16LE'Regular Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Occasional Customer':VARCHAR(19) CHARACTER SET "UTF-16LE")]) ++- LogicalFilter(condition=[>($3, 0)]) + +- LogicalAggregate(group=[{0, 1}], total_orders=[COUNT(DISTINCT $2)], total_spent=[SUM($3)], avg_order_value=[AVG($3)], max_metric=[MAX($4)], min_metric=[MIN($4)], premium_orders=[COUNT($5)]) + +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], price=[$4], metric_value=[$6], $f5=[CASE(=($7, _UTF-16LE'Premium'), 1, null:INTEGER)]) + +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], payment_id=[$3], price=[$4], metric_type=[$7], metric_value=[$8], order_tier=[CASE(>($4, 1000), _UTF-16LE'Premium':VARCHAR(8) CHARACTER SET "UTF-16LE", >($4, 500), _UTF-16LE'Standard':VARCHAR(8) CHARACTER SET "UTF-16LE", _UTF-16LE'Basic':VARCHAR(8) CHARACTER SET "UTF-16LE")]) + +- LogicalJoin(condition=[=($2, $6)], joinType=[left]) + :- LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], price=[$7]) + : +- LogicalJoin(condition=[=($0, $8)], joinType=[inner]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, OrderMetrics]]) +]]> + (total_orders, 10), 'Frequent Customer', >(total_orders, 5), 'Regular Customer', 'Occasional Customer') AS customer_type], where=[>(total_spent, 0)]) +- GroupAggregate(groupBy=[user_id, name], select=[user_id, name, COUNT_RETRACT(DISTINCT order_id) AS total_orders, SUM_RETRACT(price) AS total_spent, AVG_RETRACT(price) AS avg_order_value]) +- Exchange(distribution=[hash[user_id, name]]) - +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($2, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:2;], 1=[LeftInputId:0;LeftFieldIndex:2;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,order_id,payment_id,price,metric_id,order_id0,metric_type,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) metric_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) metric_type, DOUBLE metric_value)]) + +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,price,order_id0,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER price, VARCHAR(2147483647) order_id0, DOUBLE metric_value)]) :- Exchange(distribution=[hash[order_id]]) - : +- Calc(select=[user_id, name, order_id, payment_id, price]) - : +- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : +- Calc(select=[user_id, name, order_id, price]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, INTEGER price, VARCHAR(2147483647) user_id1)]) : :- Exchange(distribution=[hash[user_id]]) : : +- ChangelogNormalize(key=[user_id]) : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) +- Exchange(distribution=[hash[order_id]]) - +- ChangelogNormalize(key=[metric_id]) - +- Exchange(distribution=[hash[metric_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics]], fields=[metric_id, order_id, metric_type, metric_value]) + +- Calc(select=[order_id, metric_value]) + +- ChangelogNormalize(key=[metric_id]) + +- Exchange(distribution=[hash[metric_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics, project=[order_id, metric_value, metric_id], metadata=[]]], fields=[order_id, metric_value, metric_id]) ]]> + + + + + + + + = p.price OR p.price < 0) +LEFT JOIN Shipments s + ON p.user_id = s.user_id]]> + + + =($2, $7), <($7, 0)))], joinType=[inner]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + =($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -273,13 +735,13 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], location == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id, location]) -+- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -287,13 +749,13 @@ Calc(select=[user_id, name, order_id, payment_id, location]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id, location]) -+- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -302,16 +764,45 @@ Calc(select=[user_id, name, order_id, payment_id, location]) + + = p.price OR p.price < 0) +LEFT JOIN Shipments s + ON p.user_id = s.user_id]]> + + + =($2, $7), <($7, 0)))], joinType=[inner]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + =($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -320,69 +811,192 @@ Calc(select=[user_id, name, order_id, payment_id, location]) + + + + + + + + + + + + + + = FLOOR(p.price) OR p.price < 0) +LEFT JOIN Shipments s + ON p.payment_id = s.location]]> + + + =(FLOOR($2), FLOOR($8)), <($8, 0)))], joinType=[left]) + : :- LogicalProject(user_id=[$0], name=[$1], cash=[$2], order_id=[$3], user_id0=[$4], product=[$5], $f6=[UPPER($1)]) + : : +- LogicalJoin(condition=[=($4, $0)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalProject(payment_id=[$0], price=[$1], user_id=[$2], $f3=[UPPER($0)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + =(FLOOR($2), FLOOR($8)), <($8, 0)))]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:2;, LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:3;]}], select=[user_id,name,cash,order_id,user_id0,product,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- Calc(select=[user_id, name, cash, order_id, user_id0, product, UPPER(name) AS $f6]) - : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($4, $0)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) - : : :- Exchange(distribution=[hash[user_id]]) - : : : +- ChangelogNormalize(key=[user_id]) - : : : +- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - +- Exchange(distribution=[hash[location]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) +MultiJoin(commonJoinKey=[payment_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(payment_id, location)], select=[user_id,name,order_id,payment_id,location], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) location)]) +:- Exchange(distribution=[hash[payment_id]]) +: +- Calc(select=[user_id, name, order_id, payment_id]) +: +- MultiJoin(commonJoinKey=[user_id, $f6], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (payment_id)], joinConditions=[AND(=(user_id, user_id1), =($f6, $f3), OR(>=(FLOOR(cash), FLOOR(price)), <(price, 0)))], select=[user_id,name,cash,order_id,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) +: :- Exchange(distribution=[hash[user_id, $f6]]) +: : +- Calc(select=[user_id, name, cash, order_id, UPPER(name) AS $f6]) +: : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id0, user_id)], select=[user_id,name,cash,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) +: : :- Exchange(distribution=[hash[user_id]]) +: : : +- ChangelogNormalize(key=[user_id]) +: : : +- Exchange(distribution=[hash[user_id]]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) +: : +- Exchange(distribution=[hash[user_id]]) +: : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +: +- Exchange(distribution=[hash[user_id, $f3]]) +: +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) +: +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ++- Exchange(distribution=[hash[location]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments, project=[location], metadata=[]]], fields=[location]) ]]> + + + + + + + + 1000 THEN 'High' + WHEN p.price > 500 THEN 'Medium' + ELSE 'Low' + END as price_tier, + REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags, + TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created, + COALESCE(up.preferred_category, 'None') as user_preference, + CASE + WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates' + WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates' + ELSE 'Weekly Updates' + END as notification_frequency +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +LEFT JOIN ProductDetails pd + ON o.product = pd.product_id +LEFT JOIN UserPreferences up + ON u.user_id = up.user_id]]> + + + ($7, 1000), _UTF-16LE'High':VARCHAR(6) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Medium':VARCHAR(6) CHARACTER SET "UTF-16LE", _UTF-16LE'Low':VARCHAR(6) CHARACTER SET "UTF-16LE")], formatted_tags=[REGEXP_REPLACE($13, _UTF-16LE',', _UTF-16LE' | ')], product_created=[TO_TIMESTAMP_LTZ($12, 3)], user_preference=[COALESCE($15, _UTF-16LE'None')], notification_frequency=[CASE(=($16, _UTF-16LE'HIGH'), _UTF-16LE'Frequent Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", =($16, _UTF-16LE'MEDIUM'), _UTF-16LE'Daily Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", _UTF-16LE'Weekly Updates':VARCHAR(16) CHARACTER SET "UTF-16LE")]) ++- LogicalJoin(condition=[=($0, $14)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) + +- LogicalTableScan(table=[[default_catalog, default_database, UserPreferences]]) +]]> + (price, 1000), 'High', >(price, 500), 'Medium', 'Low') AS price_tier, REGEXP_REPLACE(tags, ',', ' | ') AS formatted_tags, TO_TIMESTAMP_LTZ(created_date, 3) AS product_created, COALESCE(preferred_category, 'None') AS user_preference, CASE(=(notification_level, 'HIGH'), 'Frequent Updates', =(notification_level, 'MEDIUM'), 'Daily Updates', 'Weekly Updates') AS notification_frequency]) -+- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($0, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,product_id,product_name,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (user_id)], joinConditions=[=(user_id, user_id2)], select=[user_id,name,product,price,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) :- Exchange(distribution=[hash[user_id]]) - : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - :- Exchange(distribution=[hash[product_id]]) - : +- ChangelogNormalize(key=[product_id]) - : +- Exchange(distribution=[hash[product_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, description, created_date, tags]) + : +- Calc(select=[user_id, name, product, price, description, created_date, tags]) + : +- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (product_id)], joinConditions=[=(product, product_id)], select=[user_id,name,product,price,product_id,description,created_date,tags], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) product_id, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags)]) + : :- Exchange(distribution=[hash[product]]) + : : +- Calc(select=[user_id, name, product, price]) + : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,user_id0,product,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) user_id1)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, description, created_date, tags], metadata=[]]], fields=[product_id, description, created_date, tags]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -427,30 +1092,92 @@ Calc(select=[user_id, UPPER(name) AS user_name_upper, LOWER(product) AS product_ + + = 4 THEN 'High Rating' + WHEN r.rating >= 3 THEN 'Medium Rating' + ELSE 'Low Rating' + END AS rating_category, + TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN ProductDetails pd + ON o.product = pd.product_id +LEFT JOIN Reviews r + ON pd.product_id = r.product_id]]> + + + =($13, 4), _UTF-16LE'High Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", >=($13, 3), _UTF-16LE'Medium Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", _UTF-16LE'Low Rating':VARCHAR(13) CHARACTER SET "UTF-16LE")], days_since_created=[CAST(/INT(Reinterpret(-(CURRENT_DATE, $10)), 86400000)):INTEGER]) ++- LogicalJoin(condition=[=($6, $12)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $6)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Reviews]]) +]]> + =(rating, 4), 'High Rating', >=(rating, 3), 'Medium Rating', 'Low Rating') AS rating_category, CAST(/INT(Reinterpret(-(CURRENT_DATE(), created_date)), 86400000) AS INTEGER) AS days_since_created]) -+- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $6), =($6, $12)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:1;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,product_id,product_name,price,weight,created_date,review_id,product_id0,rating,review_text,review_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DOUBLE weight, DATE created_date, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text, DATE review_date)]) ++- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (product_id), noUniqueKey], joinConditions=[=(product, product_id), =(product_id, product_id0)], select=[user_id,name,order_id,product,product_id,product_name,price,created_date,product_id0,rating,review_text], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DATE created_date, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text)]) :- Exchange(distribution=[hash[product]]) - : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Calc(select=[user_id, name, order_id, product]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[product_id]]) : +- ChangelogNormalize(key=[product_id]) : +- Exchange(distribution=[hash[product_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, price, weight, created_date]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, product_name, price, created_date], metadata=[]]], fields=[product_id, product_name, price, created_date]) +- Exchange(distribution=[hash[product_id]]) - +- ChangelogNormalize(key=[review_id]) - +- Exchange(distribution=[hash[review_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Reviews]], fields=[review_id, product_id, rating, review_text, review_date]) + +- Calc(select=[product_id, rating, review_text]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Reviews, project=[product_id, rating, review_text, review_id], metadata=[]]], fields=[product_id, rating, review_text, review_id]) ]]> + + + + + =($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner]) + :- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]]) + +- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]]) +]]> + + + + + + + + + 100) AS u +JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o + ON u.user_id = o.user_id +LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p + ON u.user_id = p.user_id +LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s + ON u.user_id = s.user_id]]> + + + ($2, 100)]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalProject(user_id=[$1], order_id=[$0], product=[$2]) + : : +- LogicalFilter(condition=[IS NOT NULL($2)]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalProject(user_id=[$2], price=[$1]) + : +- LogicalFilter(condition=[>($1, 50)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalProject(user_id=[$1], location=[$0]) + +- LogicalFilter(condition=[IS NOT NULL($0)]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + (cash, 100)]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[>(cash, 100)]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, cash], metadata=[]]], fields=[user_id, cash]) :- Exchange(distribution=[hash[user_id]]) : +- Calc(select=[user_id, order_id, product], where=[IS NOT NULL(product)]) : +- TableSourceScan(table=[[default_catalog, default_database, Orders, filter=[]]], fields=[order_id, user_id, product]) @@ -520,95 +1319,440 @@ Calc(select=[user_id, order_id, product, price, location]) + + 100 + ) AS p + ON o.user_id = p.user_id +) AS op +ON u.user_id = op.user_id]]> + + + ($1, 100)]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + (price, 100)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - - + + + + + =(price, 0)]) - : +- Exchange(distribution=[hash[payment_id, user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) - +- Exchange(distribution=[hash[user_id]]) - +- ChangelogNormalize(key=[user_id], condition=[IS NOT NULL(location)]) - +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, AddressPK, filter=[]]], fields=[user_id, location]) +LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) ++- LogicalJoin(condition=[=($0, $8)], joinType=[inner], joinHints=[[[MULTI_JOIN inheritPath:[0] options:[u, o, p]]]], stateTtlHints=[[[STATE_TTL inheritPath:[0] options:{p=1h, u=1d, o=2d}]]]) + :- LogicalJoin(condition=[=($0, $4)], joinType=[inner], joinHints=[[[MULTI_JOIN inheritPath:[0, 0] options:[u, o, p]]]], stateTtlHints=[[[STATE_TTL inheritPath:[0, 0] options:{p=1h, u=1d, o=2d}]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, Users]], hints=[[[ALIAS inheritPath:[] options:[u]]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]], hints=[[[ALIAS inheritPath:[] options:[o]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]], hints=[[[ALIAS inheritPath:[] options:[p]]]]) ]]> - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (b, 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) + +advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). +advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. + +related rel plan: +Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) ++- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a], metadata=[]]], fields=[a], changelogMode=[I,UB,UA,D]) + + +]]> + + + + + + + + + + + + + + + + = 0 +JOIN AddressPK a + ON u.user_id = a.user_id + AND a.location IS NOT NULL]]> + + + =($9, 0))], joinType=[inner]) + : :- LogicalJoin(condition=[AND(=($0, $5), IS NOT NULL($6))], joinType=[inner]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, UsersPK]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, OrdersPK]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, PaymentsPK]]) + +- LogicalTableScan(table=[[default_catalog, default_database, AddressPK]]) +]]> + + + =(price, 0)]) + : +- Exchange(distribution=[hash[payment_id, user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) + +- Exchange(distribution=[hash[user_id]]) + +- ChangelogNormalize(key=[user_id], condition=[IS NOT NULL(location)]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, AddressPK, filter=[]]], fields=[user_id, location]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -717,128 +2001,322 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -854,53 +2332,102 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) ]]> + + + + + + + + 10]]> + + + ($7, 10))]) + +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + 10)]) + +- Calc(select=[payment_id, user_id], where=[(price > 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> @@ -918,56 +2445,119 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) -+- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[payment_id, price, user_id], where=[>(price, 10)]) + +- Calc(select=[payment_id, user_id], where=[>(price, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) -+- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[payment_id, price, user_id], where=[(price > 10)]) + +- Calc(select=[payment_id, user_id], where=[(price > 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> + + 10]]> + + + ($7, 10))]) + +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + (price, 10)]) + +- Calc(select=[payment_id, user_id], where=[>(price, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + 100]]> + + + (-($1, $2), 100)]) + +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], promo_id=[$7], product_key0=[$8], promo_text=[$9]) + +- LogicalJoin(condition=[=($6, $8)], joinType=[left]) + :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], $f6=[-($1, $2)]) + : +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$4], product_key=[$5], quantity=[$6]) + : +- LogicalJoin(condition=[=($3, $5)], joinType=[left]) + : :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], $f3=[-($1, $2)]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Products]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Promotions]]) +]]> + (-(price, discount), 100)]) : : +- Exchange(distribution=[hash[product_id]]) : : +- TableSourceScan(table=[[default_catalog, default_database, Products, filter=[]]], fields=[product_id, price, discount]) : +- Exchange(distribution=[hash[product_key]]) - : +- ChangelogNormalize(key=[sale_id]) - : +- Exchange(distribution=[hash[sale_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) + : +- Calc(select=[product_key, quantity]) + : +- ChangelogNormalize(key=[sale_id]) + : +- Exchange(distribution=[hash[sale_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) +- Exchange(distribution=[hash[product_key]]) - +- ChangelogNormalize(key=[promo_id]) - +- Exchange(distribution=[hash[promo_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) + +- Calc(select=[product_key, promo_text]) + +- ChangelogNormalize(key=[promo_id]) + +- Exchange(distribution=[hash[promo_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) ]]> + + + + + + - - - - - (b, 100)]) - +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) - -advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). -advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. - -related rel plan: -Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) -+- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b], changelogMode=[I,UB,UA,D]) - - ]]> From 0ade9a4699a88152d69b8a6360f56d23abd9123b Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Wed, 24 Sep 2025 05:54:46 +0100 Subject: [PATCH 2/4] [FLINK-38417][table-planner] Ensure attribute ordering when managing join attributes in JoinKeyExtractor This closes #27033. --- .../exec/stream/MultiJoinSemanticTests.java | 5 +- .../exec/stream/MultiJoinTestPrograms.java | 777 +++++- .../planner/plan/stream/sql/MultiJoinTest.xml | 2406 +++-------------- .../AttributeBasedJoinKeyExtractor.java | 674 +++-- ...mingFourWayMixedOuterJoinOperatorTest.java | 12 +- 5 files changed, 1558 insertions(+), 2316 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java index d9f371bcd47e7..83eb11649ff22 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java @@ -52,6 +52,9 @@ public List programs() { MultiJoinTestPrograms.MULTI_JOIN_NULL_SAFE_JOIN_WITH_NULL_KEYS, MultiJoinTestPrograms.MULTI_JOIN_MIXED_CHANGELOG_MODES, MultiJoinTestPrograms - .MULTI_JOIN_WITH_TIME_ATTRIBUTES_IN_CONDITIONS_MATERIALIZATION); + .MULTI_JOIN_WITH_TIME_ATTRIBUTES_IN_CONDITIONS_MATERIALIZATION, + MultiJoinTestPrograms.MULTI_JOIN_TWO_WAY_INNER_JOIN_WITH_WHERE_IN, + MultiJoinTestPrograms.MULTI_JOIN_THREE_WAY_INNER_JOIN_MULTI_KEY_TYPES, + MultiJoinTestPrograms.MULTI_JOIN_FOUR_WAY_MIXED_JOIN_MULTI_KEY_TYPES_SHUFFLED); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java index 2137a1e5642d3..c19a5a280929a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.table.api.config.OptimizerConfigOptions; -import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.test.program.SinkTestStep; import org.apache.flink.table.test.program.SourceTestStep; import org.apache.flink.table.test.program.TableTestProgram; @@ -171,7 +171,7 @@ public class MultiJoinTestPrograms { "name STRING", "cash INT", "user_id_0 STRING PRIMARY KEY NOT ENFORCED") - .addOption("changelog-mode", "I,UA,D") + .addMode(ChangelogMode.upsert()) .producedValues( Row.ofKind(RowKind.INSERT, "Gus", 100, "1"), Row.ofKind(RowKind.INSERT, "Joe no order", 10, "8"), @@ -195,7 +195,7 @@ public class MultiJoinTestPrograms { "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING", "user_id_1 STRING") - .addOption("changelog-mode", "I,D") + .addMode(ChangelogMode.insertOnly()) .producedValues( Row.ofKind(RowKind.INSERT, "order0", "ProdB", "1"), Row.ofKind(RowKind.INSERT, "order6", "ProdF", "6"), @@ -212,7 +212,7 @@ public class MultiJoinTestPrograms { "payment_id STRING PRIMARY KEY NOT ENFORCED", "price INT", "user_id_2 STRING") - .addOption("changelog-mode", "I") + .addMode(ChangelogMode.insertOnly()) .producedValues( Row.ofKind(RowKind.INSERT, "payment1", 50, "1"), Row.ofKind(RowKind.INSERT, "payment5", -1, "5"), @@ -225,7 +225,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("Shipments") .addSchema("location STRING", "user_id_3 STRING") - .addOption("changelog-mode", "I,UA,UB,D") + .addMode(ChangelogMode.all()) .producedValues( Row.ofKind(RowKind.INSERT, "London", "1"), Row.ofKind(RowKind.INSERT, "Paris", "2"), @@ -242,7 +242,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addOption("changelog-mode", "I,UA,UB,D") + .addMode(ChangelogMode.all()) .consumedValues( "+I[3, Nomad, order3, payment3, New York]", "+I[1, Gus, order0, payment1, London]", @@ -276,6 +276,7 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-left-outer-join-with-restore", "three way left outer join with restore") + .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema("user_id STRING", "name STRING") @@ -329,6 +330,7 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-inner-join-with-restore", "three way inner join with restore") + .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema("user_id STRING", "name STRING") @@ -377,6 +379,61 @@ public class MultiJoinTestPrograms { + "INNER JOIN Payments p ON u.user_id = p.user_id") .build(); + public static final TableTestProgram + MULTI_JOIN_THREE_WAY_INNER_JOIN_WITH_TTL_HINTS_WITH_RESTORE = + TableTestProgram.of( + "three-way-inner-join-with-ttl-hints-with-restore", + "three way inner join with restore and ttl hints") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) + .setupTableSource( + SourceTestStep.newBuilder("Users") + .addSchema("user_id STRING", "name STRING") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, "1", "Gus"), + Row.ofKind(RowKind.INSERT, "2", "Bob")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, "3", "Alice")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Orders") + .addSchema("user_id STRING", "order_id STRING") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, "1", "order1"), + Row.ofKind(RowKind.INSERT, "2", "order2")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, "3", "order3")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Payments") + .addSchema("user_id STRING", "payment_id STRING") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, "1", "payment1"), + Row.ofKind(RowKind.INSERT, "2", "payment2")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, "3", "payment3")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addSchema( + "user_id STRING", + "name STRING", + "order_id STRING", + "payment_id STRING") + .consumedBeforeRestore( + "+I[1, Gus, order1, payment1]", + "+I[2, Bob, order2, payment2]") + .consumedAfterRestore("+I[3, Alice, order3, payment3]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO sink " + + "SELECT /*+ STATE_TTL('u' = '1d', 'p' = '2d') */u.user_id, u.name, o.order_id, p.payment_id " + + "FROM Users u " + + "INNER JOIN Orders o ON u.user_id = o.user_id " + + "INNER JOIN Payments p ON u.user_id = p.user_id") + .build(); + public static final TableTestProgram MULTI_JOIN_THREE_WAY_INNER_JOIN_NO_JOIN_KEY = TableTestProgram.of( "three-way-inner-join-no-join-key", @@ -454,14 +511,14 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "four-way-join-no-common-join-key-with-restore", "four way join no common join key with restore") - .setupConfig(TableConfigOptions.PLAN_FORCE_RECOMPILE, true) + .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema( "name STRING", "user_id_0 STRING PRIMARY KEY NOT ENFORCED", "cash INT") - .addOption("changelog-mode", "I,UA,D") + .addMode(ChangelogMode.upsert()) .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "Gus", "1", 100), Row.ofKind(RowKind.INSERT, "Nomad", "3", 50), @@ -486,7 +543,7 @@ public class MultiJoinTestPrograms { "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING", "user_id_1 STRING") - .addOption("changelog-mode", "I,D") + .addMode(ChangelogMode.insertOnly()) .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "order2", "ProdB", "2"), Row.ofKind(RowKind.INSERT, "order0", "ProdB", "1"), @@ -504,7 +561,7 @@ public class MultiJoinTestPrograms { "user_id_2 STRING", "payment_id STRING PRIMARY KEY NOT ENFORCED", "price INT") - .addOption("changelog-mode", "I") + .addMode(ChangelogMode.insertOnly()) .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "3", "3", 30), Row.ofKind(RowKind.INSERT, "1", "1", 50), @@ -518,7 +575,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("Shipments") .addSchema("location STRING", "user_id_3 STRING") - .addOption("changelog-mode", "I,UA,UB,D") + .addMode(ChangelogMode.all()) .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "Paris", "2"), Row.ofKind(RowKind.INSERT, "London", "1"), @@ -536,7 +593,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addOption("sink-changelog-mode-enforced", "I,UA,UB,D") + .addMode(ChangelogMode.all()) .consumedBeforeRestore( "+I[1, Gus, order0, 1, London]", "+I[1, Gus, order1, 1, London]", @@ -569,7 +626,7 @@ public class MultiJoinTestPrograms { "name STRING", "cash INT", "user_id_0 STRING PRIMARY KEY NOT ENFORCED") - .addOption("changelog-mode", "I,UA,D") + .addMode(ChangelogMode.upsert()) .producedValues( Row.ofKind(RowKind.INSERT, "Gus", 100, "1"), Row.ofKind(RowKind.INSERT, "Joe no order", 10, "8"), @@ -593,7 +650,7 @@ public class MultiJoinTestPrograms { "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING", "user_id_1 STRING") - .addOption("changelog-mode", "I,D") + .addMode(ChangelogMode.insertOnly()) .producedValues( Row.ofKind(RowKind.INSERT, "order0", "ProdB", "1"), Row.ofKind(RowKind.INSERT, "order6", "ProdF", "6"), @@ -610,7 +667,7 @@ public class MultiJoinTestPrograms { "payment_id STRING PRIMARY KEY NOT ENFORCED", "price INT", "user_id_2 STRING") - .addOption("changelog-mode", "I") + .addMode(ChangelogMode.insertOnly()) .producedValues( Row.ofKind(RowKind.INSERT, "1", 50, "1"), Row.ofKind(RowKind.INSERT, "5", -1, "5"), @@ -623,7 +680,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("Shipments") .addSchema("location STRING", "user_id_3 STRING") - .addOption("changelog-mode", "I,UA,UB,D") + .addMode(ChangelogMode.all()) .producedValues( Row.ofKind(RowKind.INSERT, "London", "1"), Row.ofKind(RowKind.INSERT, "Paris", "2"), @@ -640,7 +697,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addOption("sink-changelog-mode-enforced", "I,UA,UB,D") + .addMode(ChangelogMode.all()) .consumedValues( "+I[1, Gus, order0, 1, London]", "+I[1, Gus, order1, 1, London]", @@ -665,14 +722,14 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "four-way-complex-updating-join-with-restore", "four way complex updating join with restore") - .setupConfig(TableConfigOptions.PLAN_FORCE_RECOMPILE, true) + .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema( "name STRING", "user_id_0 STRING PRIMARY KEY NOT ENFORCED", "cash INT") - .addOption("changelog-mode", "I,UA,D") + .addMode(ChangelogMode.upsert()) .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "Gus", "1", 100), Row.ofKind(RowKind.INSERT, "Nomad", "3", 50), @@ -697,7 +754,7 @@ public class MultiJoinTestPrograms { "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING", "user_id_1 STRING") - .addOption("changelog-mode", "I,D") + .addMode(ChangelogMode.insertOnly()) .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "order2", "ProdB", "2"), Row.ofKind(RowKind.INSERT, "order0", "ProdB", "1"), @@ -715,7 +772,7 @@ public class MultiJoinTestPrograms { "user_id_2 STRING", "payment_id STRING PRIMARY KEY NOT ENFORCED", "price INT") - .addOption("changelog-mode", "I") + .addMode(ChangelogMode.insertOnly()) .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "3", "payment3", 30), Row.ofKind(RowKind.INSERT, "1", "payment1", 50), @@ -729,7 +786,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("Shipments") .addSchema("location STRING", "user_id_3 STRING") - .addOption("changelog-mode", "I,UA,UB,D") + .addMode(ChangelogMode.all()) .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "Paris", "2"), Row.ofKind(RowKind.INSERT, "London", "1"), @@ -747,7 +804,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addOption("sink-changelog-mode-enforced", "I,UA,UB,D") + .addMode(ChangelogMode.all()) .consumedBeforeRestore( "+I[1, Gus, order0, payment1, London]", "+I[1, Gus, order1, payment1, London]", @@ -769,6 +826,106 @@ public class MultiJoinTestPrograms { + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3") .build(); + public static final TableTestProgram + MULTI_JOIN_FOUR_WAY_COMPLEX_PRESERVES_UPSERT_KEY_WITH_RESTORE = + TableTestProgram.of( + "four-way-complex-preserves-upsert-key-with-restore", + "four way complex preserves upsert key with restore") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) + .setupTableSource( + SourceTestStep.newBuilder("Users") + .addSchema( + "user_id_0 STRING PRIMARY KEY NOT ENFORCED", + "name STRING") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, "1", "Gus"), + Row.ofKind(RowKind.INSERT, "3", "Joe")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, "2", "Bob")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Orders") + .addSchema( + "order_id STRING NOT NULL", + "user_id_1 STRING NOT NULL", + "product STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`order_id`, `user_id_1`) NOT ENFORCED") + .addMode(ChangelogMode.insertOnly()) + .producedBeforeRestore( + Row.ofKind( + RowKind.INSERT, "order1", "1", "ProdA")) + .producedAfterRestore( + Row.ofKind( + RowKind.INSERT, "order2", "2", "ProdB")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Payments") + .addSchema( + "payment_id STRING NOT NULL", + "user_id_2 STRING NOT NULL", + "price INT", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`payment_id`, `user_id_2`) NOT ENFORCED") + .addMode(ChangelogMode.insertOnly()) + .producedBeforeRestore( + Row.ofKind( + RowKind.INSERT, "payment1", "1", 100), + Row.ofKind( + RowKind.INSERT, "payment3", "3", 300)) + .producedAfterRestore( + Row.ofKind( + RowKind.INSERT, "payment2", "2", 200)) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Address") + .addSchema( + "user_id_3 STRING NOT NULL", + "location STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id_3`) NOT ENFORCED") + .addMode(ChangelogMode.insertOnly()) + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, "1", "London"), + Row.ofKind(RowKind.INSERT, "3", "Berlin")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, "2", "Paris")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addMode(ChangelogMode.upsert()) + .addSchema( + "user_id_0 STRING NOT NULL", + "order_id STRING NOT NULL", + "user_id_1 STRING NOT NULL", + "payment_id STRING NOT NULL", + "user_id_2 STRING NOT NULL", + "name STRING", + "location STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id_0`, `order_id`, `user_id_1`, `payment_id`, `user_id_2`) NOT ENFORCED") + .consumedBeforeRestore( + "+I[1, order1, 1, payment1, 1, Gus, London]") + .consumedAfterRestore( + "+I[2, order2, 2, payment2, 2, Bob, Paris]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO `sink`\n" + + "SELECT\n" + + " u.user_id_0,\n" + + " o.order_id,\n" + + " o.user_id_1,\n" + + " p.payment_id,\n" + + " p.user_id_2,\n" + + " u.name,\n" + + " a.location\n" + + "FROM Users u\n" + + "JOIN Orders o\n" + + " ON u.user_id_0 = o.user_id_1 AND o.product IS NOT NULL\n" + + "JOIN Payments p\n" + + " ON u.user_id_0 = p.user_id_2 AND p.price >= 0\n" + + "JOIN Address a\n" + + " ON u.user_id_0 = a.user_id_3 AND a.location IS NOT NULL") + .build(); + public static final TableTestProgram MULTI_JOIN_WITH_TIME_ATTRIBUTES_MATERIALIZATION = TableTestProgram.of( "three-way-join-with-time-attributes", @@ -896,6 +1053,65 @@ public class MultiJoinTestPrograms { + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2") .build(); + public static final TableTestProgram MULTI_JOIN_MIXED_CHANGELOG_MODES = + TableTestProgram.of( + "three-way-mixed-changelog-modes", + "three way join with mixed changelog modes and primary key configurations") + .setupTableSource( + SourceTestStep.newBuilder("AppendTable") + .addSchema("id STRING PRIMARY KEY NOT ENFORCED, val STRING") + .addMode(ChangelogMode.insertOnly()) + .producedValues( + Row.ofKind(RowKind.INSERT, "1", "append1"), + Row.ofKind(RowKind.INSERT, "2", "append2"), + Row.ofKind(RowKind.INSERT, "3", "append3")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("RetractTable") + .addSchema("ref_id STRING, data STRING") + .addMode(ChangelogMode.all()) + .producedValues( + Row.ofKind(RowKind.INSERT, "1", "retract1"), + Row.ofKind(RowKind.INSERT, "2", "retract2"), + Row.ofKind(RowKind.INSERT, "3", "retract3"), + Row.ofKind(RowKind.DELETE, "3", "retract3"), + Row.ofKind(RowKind.INSERT, "1", "retract1_new")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("UpsertTable") + .addSchema( + "key_id STRING PRIMARY KEY NOT ENFORCED, status STRING") + .addMode(ChangelogMode.upsert()) + .producedValues( + Row.ofKind(RowKind.INSERT, "1", "active"), + Row.ofKind(RowKind.INSERT, "2", "pending"), + Row.ofKind(RowKind.UPDATE_AFTER, "2", "active"), + Row.ofKind(RowKind.INSERT, "3", "inactive"), + Row.ofKind(RowKind.DELETE, "3", "inactive")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addSchema( + "id STRING", + "val STRING", + "data STRING", + "status STRING") + .addMode(ChangelogMode.all()) + .consumedValues( + "+I[1, append1, retract1, active]", + "+I[2, append2, retract2, active]", + "+I[1, append1, retract1_new, active]", + "+I[3, append3, null, null]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO sink " + + "SELECT a.id, a.val, r.data, u.status " + + "FROM AppendTable a " + + "LEFT JOIN RetractTable r ON a.id = r.ref_id " + + "LEFT JOIN UpsertTable u ON a.id = u.key_id") + .build(); + public static final TableTestProgram MULTI_JOIN_THREE_WAY_LEFT_OUTER_JOIN_WITH_CTE = TableTestProgram.of( "left-outer-join-with-cte", @@ -907,7 +1123,7 @@ public class MultiJoinTestPrograms { "user_id STRING", "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING") - .addOption("changelog-mode", "I, UA,D") + .addMode(ChangelogMode.upsert()) .producedValues( Row.ofKind(RowKind.INSERT, "2", "order2", "Product B"), Row.ofKind( @@ -1028,64 +1244,151 @@ public class MultiJoinTestPrograms { + "INNER JOIN OrdersNullSafe o ON u.user_id IS NOT DISTINCT FROM o.user_id") .build(); - public static final TableTestProgram MULTI_JOIN_MIXED_CHANGELOG_MODES = + static final TableTestProgram MULTI_JOIN_TWO_WAY_LEFT_JOIN_PRESERVES_UPSERT_KEY_WITH_RESTORE = TableTestProgram.of( - "three-way-mixed-changelog-modes", - "three way join with mixed changelog modes and primary key configurations") + "two-way-left-join-preserves-upsert-key-with-restore", + "validates upsert with non key filter with restore") .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( - SourceTestStep.newBuilder("AppendTable") - .addSchema("id STRING PRIMARY KEY NOT ENFORCED, val STRING") - .addOption("changelog-mode", "I") - .producedValues( - Row.ofKind(RowKind.INSERT, "1", "append1"), - Row.ofKind(RowKind.INSERT, "2", "append2"), - Row.ofKind(RowKind.INSERT, "3", "append3")) + SourceTestStep.newBuilder("Users") + .addMode(ChangelogMode.upsert()) + .addSchema( + "user_id INT NOT NULL", + "shard_id INT NOT NULL", + "description STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`) NOT ENFORCED") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, 1, 1, "shard_a")) + .producedAfterRestore( + Row.ofKind(RowKind.UPDATE_AFTER, 3, 1, "another shard")) .build()) .setupTableSource( - SourceTestStep.newBuilder("RetractTable") - .addSchema("ref_id STRING, data STRING") - .addOption("changelog-mode", "I,UA,UB,D") - .producedValues( - Row.ofKind(RowKind.INSERT, "1", "retract1"), - Row.ofKind(RowKind.INSERT, "2", "retract2"), - Row.ofKind(RowKind.INSERT, "3", "retract3"), - Row.ofKind(RowKind.DELETE, "3", "retract3"), - Row.ofKind(RowKind.INSERT, "1", "retract1_new")) + SourceTestStep.newBuilder("Orders") + .addMode(ChangelogMode.upsert()) + .addSchema( + "user_id INT NOT NULL", + "order_id BIGINT NOT NULL", + "product STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, 1, 1L, "a"), + Row.ofKind(RowKind.UPDATE_AFTER, 1, 1L, "a_updated")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, 1, 2L, "b"), + Row.ofKind(RowKind.DELETE, 1, 2L, "b"), + Row.ofKind(RowKind.INSERT, 3, 1L, "b"), + Row.ofKind(RowKind.INSERT, 9, 9L, "b")) .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addMode(ChangelogMode.upsert()) + .addSchema( + "`user_id` INT NOT NULL", + "`order_id` BIGINT NOT NULL", + "product STRING", + "user_shard_id INT", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED") + .consumedBeforeRestore("+I[1, 1, a_updated, 1]") + .consumedAfterRestore("+I[3, 1, b, 1]", "+I[9, 9, b, null]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO `sink`\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " o.product,\n" + + " u.shard_id\n" + + "FROM Orders o\n" + + "LEFT JOIN Users u\n" + + " ON u.user_id = o.user_id\n") + .build(); + + static final TableTestProgram MULTI_JOIN_THREE_WAY_JOIN_PRESERVES_UPSERT_KEY_WITH_RESTORE = + TableTestProgram.of( + "three-way-upsert-preserves-key-with-restore", + "validates upsert with non key filter with restore") + .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( - SourceTestStep.newBuilder("UpsertTable") + SourceTestStep.newBuilder("Users") + .addMode(ChangelogMode.upsert()) .addSchema( - "key_id STRING PRIMARY KEY NOT ENFORCED, status STRING") - .addOption("changelog-mode", "I,UA,D") - .producedValues( - Row.ofKind(RowKind.INSERT, "1", "active"), - Row.ofKind(RowKind.INSERT, "2", "pending"), - Row.ofKind(RowKind.UPDATE_AFTER, "2", "active"), - Row.ofKind(RowKind.INSERT, "3", "inactive"), - Row.ofKind(RowKind.DELETE, "3", "inactive")) + "user_id INT NOT NULL", + "shard_id INT NOT NULL", + "description STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`) NOT ENFORCED") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, 1, 1, "shard_a")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, 2, 1, "shard_b")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Orders") + .addMode(ChangelogMode.upsert()) + .addSchema( + "user_id INT NOT NULL", + "order_id BIGINT NOT NULL", + "product STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, 1, 1L, "a"), + Row.ofKind(RowKind.INSERT, 1, 2L, "b")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, 2, 1L, "c"), + Row.ofKind(RowKind.INSERT, 2, 2L, "d")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Payments") + .addMode(ChangelogMode.upsert()) + .addSchema( + "user_id INT NOT NULL", + "payment_id BIGINT NOT NULL", + "product STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `payment_id`) NOT ENFORCED") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, 1, 1L, "a"), + Row.ofKind(RowKind.INSERT, 1, 2L, "b")) + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, 2, 1L, "c"), + Row.ofKind(RowKind.INSERT, 2, 2L, "d")) .build()) .setupTableSink( SinkTestStep.newBuilder("sink") + .addMode(ChangelogMode.upsert()) .addSchema( - "id STRING", - "val STRING", - "data STRING", - "status STRING") - .addOption("sink-changelog-mode-enforced", "I,UA,UB,D") - .consumedValues( - "+I[1, append1, retract1, active]", - "+I[2, append2, retract2, active]", - "+I[1, append1, retract1_new, active]", - "+I[3, append3, null, null]") + "`user_id` INT NOT NULL", + "`order_id` BIGINT NOT NULL", + "`user_id2` INT NOT NULL", + "payment_id BIGINT NOT NULL", + "`user_id3` INT NOT NULL", + "description STRING", + "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`, `user_id2`, `payment_id`, `user_id3`) NOT ENFORCED") + .consumedBeforeRestore( + "+I[1, 1, 1, 2, 1, shard_a]", + "+I[1, 1, 1, 1, 1, shard_a]", + "+I[1, 2, 1, 1, 1, shard_a]", + "+I[1, 2, 1, 2, 1, shard_a]") + .consumedAfterRestore( + "+I[2, 2, 2, 1, 2, shard_b]", + "+I[2, 1, 2, 2, 2, shard_b]", + "+I[2, 2, 2, 2, 2, shard_b]", + "+I[2, 1, 2, 1, 2, shard_b]") .testMaterializedData() .build()) .runSql( - "INSERT INTO sink " - + "SELECT a.id, a.val, r.data, u.status " - + "FROM AppendTable a " - + "LEFT JOIN RetractTable r ON a.id = r.ref_id " - + "LEFT JOIN UpsertTable u ON a.id = u.key_id") + "INSERT INTO `sink`\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " p.user_id,\n" + + " p.payment_id,\n" + + " u.user_id,\n" + + " u.description\n" + + "FROM Users u\n" + + "JOIN `Orders` AS o\n" + + " ON o.user_id = u.user_id\n" + + "JOIN Payments p\n" + + " ON o.user_id = p.user_id") .build(); public static final TableTestProgram @@ -1104,7 +1407,7 @@ public class MultiJoinTestPrograms { "auctionDateTime AS TO_TIMESTAMP(auctionTimestamp)", "expires AS TO_TIMESTAMP(expiresTimestamp)", "WATERMARK FOR auctionDateTime AS auctionDateTime - INTERVAL '1' SECOND") - .addOption("changelog-mode", "I") + .addMode(ChangelogMode.insertOnly()) .producedValues( Row.ofKind( RowKind.INSERT, @@ -1133,7 +1436,7 @@ public class MultiJoinTestPrograms { "bidTimestamp STRING", "bidDateTime AS TO_TIMESTAMP(bidTimestamp)", "WATERMARK FOR bidDateTime AS bidDateTime - INTERVAL '1' SECOND") - .addOption("changelog-mode", "I") + .addMode(ChangelogMode.insertOnly()) .producedValues( Row.ofKind( RowKind.INSERT, @@ -1184,4 +1487,340 @@ public class MultiJoinTestPrograms { + ") Q " + "GROUP BY Q.category") .build(); + + public static final TableTestProgram MULTI_JOIN_TWO_WAY_INNER_JOIN_WITH_WHERE_IN = + TableTestProgram.of( + "two-way-inner-join-with-where-in", + "two way inner join with WHERE IN clause") + .setupTableSource( + SourceTestStep.newBuilder("Records") + .addSchema( + "`record_id` STRING PRIMARY KEY NOT ENFORCED", + "`user_id` INT") + .addMode(ChangelogMode.upsert()) + .producedValues( + Row.ofKind(RowKind.INSERT, "record_1", 1), + Row.ofKind(RowKind.INSERT, "record_2", 2), + Row.ofKind(RowKind.INSERT, "record_3", 3)) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Users") + .addSchema( + "`user_id` INT PRIMARY KEY NOT ENFORCED", "`id` STRING") + .addMode(ChangelogMode.upsert()) + .producedValues( + Row.ofKind(RowKind.INSERT, 1, "record_1"), + Row.ofKind(RowKind.INSERT, 2, "record_2"), + Row.ofKind(RowKind.INSERT, 3, "record_3")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addSchema("record_id STRING", "`user_id` INT", "id STRING") + .addMode(ChangelogMode.all()) + .consumedValues( + // Only records with user_id 1 and 2 should be + // included due to WHERE IN clause + "+I[record_1, 1, record_1]", + "+I[record_2, 2, record_2]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO sink " + + "SELECT r.`record_id`, r.`user_id`, u.`id` " + + "FROM Records r " + + "INNER JOIN Users u ON u.`user_id` = r.`user_id` AND u.`id` = r.`record_id` " + + "WHERE r.`user_id` IN (1, 2)") + .build(); + + public static final TableTestProgram MULTI_JOIN_THREE_WAY_INNER_JOIN_MULTI_KEY_TYPES = + TableTestProgram.of( + "three-way-inner-join-three-keys-shuffled", + "three way inner join with three keys in shuffled order") + .setupTableSource( + SourceTestStep.newBuilder("Users3K") + .addSchema("k1 STRING", "k2 INT", "k3 BOOLEAN", "name STRING") + .producedValues( + Row.ofKind(RowKind.INSERT, "K1", 100, true, "Gus"), + Row.ofKind(RowKind.INSERT, "A1", 200, false, "Bob")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Orders3K") + .addSchema( + // Shuffled key order: k2 first, then k3, then k1 + "k2 INT PRIMARY KEY NOT ENFORCED," + + "order_id STRING," + + "k3 BOOLEAN," + + "k1 STRING," + + "k4 STRING") + .producedValues( + // Matches Gus (Users3K: K1,100,true) + Row.ofKind( + RowKind.INSERT, + 100, + "order1", + true, + "K1", + "k4_val1"), + // Matches Bob (Users3K: A1,200,false) + Row.ofKind( + RowKind.INSERT, + 200, + "order2", + false, + "A1", + "k4_val2"), + // Partial matches that should NOT join (one key wrong) + Row.ofKind( + RowKind.INSERT, + 100, + "order_partial1", + true, + "WRONG", + "k4_u"), + Row.ofKind( + RowKind.INSERT, + 100, + "order_partial2", + false, + "K1", + "k4_u"), + Row.ofKind( + RowKind.INSERT, + 200, + "order_partial3", + false, + "WRONG", + "k4_u"), + Row.ofKind( + RowKind.INSERT, + 200, + "order_partial4", + true, + "A1", + "k4_u")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Payments3K") + .addSchema( + // Shuffled key order: k1 first, then k3, then k2 + "k1 STRING", + "payment_id STRING", + "k3 BOOLEAN", + "k2 INT") + .producedValues( + // Matches Gus (Users3K: K1,100,true) + Row.ofKind(RowKind.INSERT, "K1", "payment1", true, 100), + // Matches Bob (Users3K: A1,200,false) + Row.ofKind( + RowKind.INSERT, "A1", "payment2", false, 200), + // Partial matches that should NOT join (one key wrong) + Row.ofKind( + RowKind.INSERT, + "K1", + "payment_partial1", + false, + 100), + Row.ofKind( + RowKind.INSERT, + "A1", + "payment_partial2", + false, + 300)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addSchema( + "name STRING", + "order_id STRING", + "payment_id STRING", + "k1 STRING", + "k2 INT", + "k3 BOOLEAN") + .addMode(ChangelogMode.insertOnly()) + .consumedValues( + "+I[Gus, order1, payment1, K1, 100, true]", + "+I[Bob, order2, payment2, A1, 200, false]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO sink " + + "SELECT U.name, O.order_id, P.payment_id, U.k1, U.k2, U.k3 " + + "FROM Users3K AS U " + + "JOIN Orders3K AS O ON U.k1 = O.k1 AND U.k2 = O.k2 AND U.k3 = O.k3 " + + "JOIN Payments3K AS P ON U.k1 = P.k1 AND U.k2 = P.k2 AND U.k3 = P.k3") + .build(); + + public static final TableTestProgram MULTI_JOIN_FOUR_WAY_MIXED_JOIN_MULTI_KEY_TYPES_SHUFFLED = + TableTestProgram.of( + "four-way-mixed-join-multi-key-types-shuffled", + "four way mixed join with three keys in shuffled order and a final join with two conditions") + .setupTableSource( + SourceTestStep.newBuilder("Users4K") + .addSchema( + "k1 STRING PRIMARY KEY NOT ENFORCED", + "k2 INT", + "k3 BOOLEAN", + "k4 STRING", + "name STRING") + .producedValues( + Row.ofKind( + RowKind.INSERT, + "K1", + 100, + true, + "k4_val1", + "Gus"), + Row.ofKind( + RowKind.INSERT, + "A1", + 200, + false, + "k4_val2", + "Bob"), + Row.ofKind( + RowKind.INSERT, + "A1", + 200, + false, + "k4_val2", + "John"), + Row.ofKind( + RowKind.DELETE, + "A1", + 200, + false, + "k4_val2", + "John"), + Row.ofKind( + RowKind.INSERT, + "U_NO_MATCH", + 1, + true, + "k4_u", + "UserNoMatch")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Orders4K") + .addSchema( + // Shuffled key order: k2 first, then k3, then k1 + "k2 INT PRIMARY KEY NOT ENFORCED", + "order_id STRING", + "k3 BOOLEAN", + "k1 STRING", + "k4 STRING") + .producedValues( + // Matches Gus (Users4K: K1,100,true,k4_val1) + Row.ofKind( + RowKind.INSERT, + 100, + "order1", + true, + "K1", + "k4_val1"), + // Matches Bob (Users4K: A1,200,false,k4_val2) + Row.ofKind( + RowKind.INSERT, + 200, + "order2", + false, + "A1", + "k4_val2"), + // No match + Row.ofKind( + RowKind.INSERT, + 2, + "O_NO_MATCH", + true, + "K_O", + "k4_o")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Payments4K") + .addSchema( + // Shuffled key order: k1 first, then k3, then + // k2 + "k1 STRING PRIMARY KEY NOT ENFORCED," + + "payment_id STRING," + + "k3 BOOLEAN," + + "k2 INT," + + "k4 STRING") + .producedValues( + // Matches Gus (Users4K: K1,100,true,k4_val1) + Row.ofKind( + RowKind.INSERT, + "K1", + "payment1", + true, + 100, + "k4_val1"), + // Matches Bob (Users4K: A1,200,false,k4_val2) + Row.ofKind( + RowKind.INSERT, + "A1", + "payment2", + false, + 200, + "k4_val2"), + // No match + Row.ofKind( + RowKind.INSERT, + "P_NO_MATCH", + "P_NO_MATCH_ID", + true, + 3, + "k4_p")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("Shipments4K") + .addSchema( + "k3 BOOLEAN PRIMARY KEY NOT ENFORCED", + "k4 STRING", + "ship_id STRING") + .producedValues( + // Matches Gus + Row.ofKind(RowKind.INSERT, true, "k4_val1", "ship1"), + // Does not match anyone + Row.ofKind( + RowKind.INSERT, + false, + "k4_val_no_match", + "ship2"), + // No match + Row.ofKind( + RowKind.INSERT, + true, + "k4_val_another_no_match", + "ship3"), + // No match + Row.ofKind( + RowKind.INSERT, + false, + "k4_s_no_match", + "ship_no_match")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addSchema( + "name STRING", + "order_id STRING", + "payment_id STRING", + "ship_id STRING", + "k1 STRING", + "k2 INT", + "k3 BOOLEAN", + "k4 STRING") + .addMode(ChangelogMode.all()) + .consumedValues( + "+I[Bob, order2, payment2, null, A1, 200, false, k4_val2]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO sink " + + "SELECT U.name, O.order_id, P.payment_id, S.ship_id, U.k1, U.k2, U.k3, U.k4 " + + "FROM Users4K AS U " + + "JOIN Orders4K AS O ON O.k2 > 100 AND U.k2 = O.k2 AND U.k1 = O.k1 AND U.k3 = O.k3 AND U.k4 = O.k4 " + + "JOIN Payments4K AS P ON U.k1 = P.k1 AND O.k3 = P.k3 AND U.k4 = P.k4 AND O.k2 = P.k2 AND P.k2 > 100 " + + "LEFT JOIN Shipments4K AS S ON U.k3 = S.k3 AND U.k2 > 150 AND U.k4 = S.k4 " + + "WHERE U.k2 > 50") + .build(); } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index 3a5e888ca25c4..eb29e71a3b7b7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -17,370 +17,123 @@ limitations under the License. --> - - 5 THEN 1 END) as bulk_orders -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN OrderItems oi - ON o.order_id = oi.order_id -LEFT JOIN ProductCategories pc - ON oi.product_name = pc.category_id -LEFT JOIN Payments p - ON u.user_id = p.user_id -GROUP BY u.user_id, u.name, pc.category_name -HAVING COUNT(DISTINCT o.order_id) > 0]]> - - - ($3, 0)]) -+- LogicalAggregate(group=[{0, 1, 2}], order_count=[COUNT(DISTINCT $3)], total_items=[SUM($4)], total_value=[SUM($5)], avg_item_price=[AVG($6)], max_payment=[MAX($7)], min_payment=[MIN($7)], bulk_orders=[COUNT($8)]) - +- LogicalProject(user_id=[$0], name=[$1], category_name=[$12], order_id=[$3], quantity=[$9], $f5=[*($9, $10)], unit_price=[$10], price=[$15], $f8=[CASE(>($9, 5), 1, null:INTEGER)]) - +- LogicalJoin(condition=[=($0, $16)], joinType=[left]) - :- LogicalJoin(condition=[=($8, $11)], joinType=[left]) - : :- LogicalJoin(condition=[=($3, $7)], joinType=[left]) - : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, OrderItems]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) -]]> - (order_count, 0)]) +- GroupAggregate(groupBy=[user_id, name, category_name], select=[user_id, name, category_name, COUNT_RETRACT(DISTINCT order_id) AS order_count, SUM_RETRACT(quantity) AS total_items, SUM_RETRACT($f5) AS total_value, AVG_RETRACT(unit_price) AS avg_item_price, MAX_RETRACT(price) AS max_payment, MIN_RETRACT(price) AS min_payment, COUNT_RETRACT($f8) AS bulk_orders]) +- Exchange(distribution=[hash[user_id, name, category_name]]) +- Calc(select=[user_id, name, category_name, order_id, quantity, *(quantity, unit_price) AS $f5, unit_price, price, CASE(>(quantity, 5), 1, null:INTEGER) AS $f8]) - +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id1)], select=[user_id,name,order_id,quantity,unit_price,category_name,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_name, INTEGER price, VARCHAR(2147483647) user_id1)]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($8, $11), =($0, $16)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:8;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:8;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price,category_id,category_name,parent_category,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id, name, order_id, quantity, unit_price, category_name]) - : +- MultiJoin(commonJoinKey=[product_name], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (category_id)], joinConditions=[=(product_name, category_id)], select=[user_id,name,order_id,product_name,quantity,unit_price,category_id,category_name], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name)]) - : :- Exchange(distribution=[hash[product_name]]) - : : +- Calc(select=[user_id, name, order_id, product_name, quantity, unit_price]) - : : +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) - : : :- Exchange(distribution=[hash[order_id]]) - : : : +- Calc(select=[user_id, name, order_id]) - : : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) - : : : :- Exchange(distribution=[hash[user_id]]) - : : : : +- ChangelogNormalize(key=[user_id]) - : : : : +- Exchange(distribution=[hash[user_id]]) - : : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) - : : : +- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) - : : +- Exchange(distribution=[hash[order_id]]) - : : +- Calc(select=[order_id, product_name, quantity, unit_price]) - : : +- ChangelogNormalize(key=[item_id]) - : : +- Exchange(distribution=[hash[item_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) - : +- Exchange(distribution=[hash[category_id]]) - : +- ChangelogNormalize(key=[category_id]) - : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories, project=[category_id, category_name], metadata=[]]], fields=[category_id, category_name]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($3, $7)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;], 1=[LeftInputId:0;LeftFieldIndex:3;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) + : :- Exchange(distribution=[hash[order_id]]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[order_id]]) + : +- ChangelogNormalize(key=[item_id]) + : +- Exchange(distribution=[hash[item_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) + :- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, parent_category]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ]]> - - 0]]> - - - ($2, 0)]) -+- LogicalAggregate(group=[{0}], unique_users=[COUNT(DISTINCT $1)], total_sales=[COUNT($2)], total_revenue=[SUM($3)], avg_sale_amount=[AVG($3)], max_sale_amount=[MAX($3)]) - +- LogicalProject(category_name=[$7], user_id=[$0], sale_id=[$10], amount=[$13]) - +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) - :- LogicalJoin(condition=[AND(=($0, $9), =($5, $6))], joinType=[left]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Categories]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) -]]> - (total_sales, 0)]) +- GroupAggregate(groupBy=[category_name], select=[category_name, COUNT_RETRACT(DISTINCT user_id) AS unique_users, COUNT_RETRACT(sale_id) AS total_sales, SUM_RETRACT(amount) AS total_revenue, AVG_RETRACT(amount) AS avg_sale_amount, MAX_RETRACT(amount) AS max_sale_amount]) +- Exchange(distribution=[hash[category_name]]) +- Calc(select=[category_name, user_id, sale_id, amount]) - +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, (category_id), (sale_id)], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), =(product, category_id)), =(user_id, user_id2)], select=[user_id,user_id0,product,category_id,category_name,user_id1,sale_id,user_id2,amount], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, DOUBLE amount)]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $9), =($5, $6)), =($0, $11)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:3;, LeftInputId:1;LeftFieldIndex:2;RightInputId:2;RightFieldIndex:0;], 3=[LeftInputId:0;LeftFieldIndex:0;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,category_id,category_name,parent_category,user_id1,sale_id,user_id2,product_id,amount,sale_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) product_id, DOUBLE amount, DATE sale_date)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[category_id]) : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Categories, project=[category_id, category_name, user_id], metadata=[]]], fields=[category_id, category_name, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Categories]], fields=[category_id, category_name, parent_category, user_id]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[sale_id]) +- Exchange(distribution=[hash[sale_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Sales, project=[sale_id, user_id, amount], metadata=[]]], fields=[sale_id, user_id, amount]) + +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, user_id, product_id, amount, sale_date]) ]]> - - 600000 -), -active_projects AS ( - SELECT project_id, project_name, dept_id - FROM Projects - WHERE status = 'ACTIVE' -) -SELECT - u.user_id, - u.name, - o.order_id, - hbd.dept_name, - ap.project_name, - hbd.budget -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN high_budget_depts hbd - ON o.user_id = hbd.dept_id -LEFT JOIN active_projects ap - ON hbd.dept_id = ap.dept_id]]> - - - ($2, 600000)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Departments]]) - +- LogicalProject(project_id=[$0], project_name=[$1], dept_id=[$2]) - +- LogicalFilter(condition=[=($3, _UTF-16LE'ACTIVE')]) - +- LogicalTableScan(table=[[default_catalog, default_database, Projects]]) -]]> - (budget, 600000)]) : +- Exchange(distribution=[hash[dept_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Departments, filter=[]]], fields=[dept_id, dept_name, budget]) +- Exchange(distribution=[hash[dept_id]]) - +- Calc(select=[project_name, dept_id]) + +- Calc(select=[project_id, project_name, dept_id]) +- ChangelogNormalize(key=[project_id], condition=[=(status, 'ACTIVE')]) +- Exchange(distribution=[hash[project_id]]) +- TableSourceScan(table=[[default_catalog, default_database, Projects, filter=[]]], fields=[project_id, project_name, dept_id, status]) -]]> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 1000 THEN 'High-Value Premium' - WHEN pc.is_premium = true THEN 'Premium' - WHEN p.price > 500 THEN 'Standard High-Value' - ELSE 'Standard' - END AS product_tier, - CASE - WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended' - WHEN pr.rating >= 3 THEN 'Recommended' - WHEN pr.rating >= 2 THEN 'Average' - ELSE 'Not Recommended' - END AS recommendation_status, - CASE - WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate) - ELSE p.price - END AS final_price -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN Payments p - ON u.user_id = p.user_id -LEFT JOIN ProductCategories pc - ON o.product = pc.category_id -LEFT JOIN ProductReviews pr - ON o.product = pr.product_id]]> - - - ($7, 1000)), _UTF-16LE'High-Value Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", $11, _UTF-16LE'Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Standard High-Value':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Standard':VARCHAR(19) CHARACTER SET "UTF-16LE")], recommendation_status=[CASE(AND(>=($15, 4), $16), _UTF-16LE'Highly Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 3), _UTF-16LE'Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 2), _UTF-16LE'Average':VARCHAR(18) CHARACTER SET "UTF-16LE", _UTF-16LE'Not Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE")], final_price=[CASE(>($12, 0.2:DECIMAL(2, 1)), *($7, -(1, $12)), CAST($7):DOUBLE)]) -+- LogicalJoin(condition=[=($5, $14)], joinType=[left]) - :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) - : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) - +- LogicalTableScan(table=[[default_catalog, default_database, ProductReviews]]) -]]> - (price, 1000)), 'High-Value Premium', is_premium, 'Premium', >(price, 500), 'Standard High-Value', 'Standard') AS product_tier, CASE(AND(>=(rating, 4), is_verified), 'Highly Recommended', >=(rating, 3), 'Recommended', >=(rating, 2), 'Average', 'Not Recommended') AS recommendation_status, CASE(>(discount_rate, 0.2), *(price, -(1, discount_rate)), CAST(price AS DOUBLE)) AS final_price]) -+- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (category_id), noUniqueKey], joinConditions=[=(product, category_id), =(product, product_id)], select=[user_id,order_id,product,payment_id,price,category_id,category_name,is_premium,discount_rate,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($5, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:5;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,category_id,category_name,is_premium,discount_rate,review_id,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) :- Exchange(distribution=[hash[product]]) - : +- Calc(select=[user_id, order_id, product, payment_id, price]) - : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) :- Exchange(distribution=[hash[category_id]]) : +- ChangelogNormalize(key=[category_id]) : +- Exchange(distribution=[hash[category_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, is_premium, discount_rate]) +- Exchange(distribution=[hash[product_id]]) - +- Calc(select=[product_id, rating, is_verified]) - +- ChangelogNormalize(key=[review_id]) - +- Exchange(distribution=[hash[review_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) ]]> - - 1000 THEN 'Premium' - WHEN bo.price > 500 THEN 'Standard' - ELSE 'Basic' - END as order_tier - FROM base_orders bo - LEFT JOIN OrderMetrics om - ON bo.order_id = om.order_id -), -aggregated_metrics AS ( - SELECT - user_id, - name, - COUNT(DISTINCT order_id) as total_orders, - SUM(price) as total_spent, - AVG(price) as avg_order_value, - MAX(metric_value) as max_metric, - MIN(metric_value) as min_metric, - COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders - FROM enriched_orders - GROUP BY user_id, name -) -SELECT - user_id, - UPPER(name) as user_name, - total_orders, - ROUND(total_spent, 2) as total_spent_rounded, - ROUND(avg_order_value, 2) as avg_order_value_rounded, - CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary, - CASE - WHEN total_orders > 10 THEN 'Frequent Customer' - WHEN total_orders > 5 THEN 'Regular Customer' - ELSE 'Occasional Customer' - END as customer_type -FROM aggregated_metrics -WHERE total_spent > 0]]> - - - ($2, 10), _UTF-16LE'Frequent Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", >($2, 5), _UTF-16LE'Regular Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Occasional Customer':VARCHAR(19) CHARACTER SET "UTF-16LE")]) -+- LogicalFilter(condition=[>($3, 0)]) - +- LogicalAggregate(group=[{0, 1}], total_orders=[COUNT(DISTINCT $2)], total_spent=[SUM($3)], avg_order_value=[AVG($3)], max_metric=[MAX($4)], min_metric=[MIN($4)], premium_orders=[COUNT($5)]) - +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], price=[$4], metric_value=[$6], $f5=[CASE(=($7, _UTF-16LE'Premium'), 1, null:INTEGER)]) - +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], payment_id=[$3], price=[$4], metric_type=[$7], metric_value=[$8], order_tier=[CASE(>($4, 1000), _UTF-16LE'Premium':VARCHAR(8) CHARACTER SET "UTF-16LE", >($4, 500), _UTF-16LE'Standard':VARCHAR(8) CHARACTER SET "UTF-16LE", _UTF-16LE'Basic':VARCHAR(8) CHARACTER SET "UTF-16LE")]) - +- LogicalJoin(condition=[=($2, $6)], joinType=[left]) - :- LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], price=[$7]) - : +- LogicalJoin(condition=[=($0, $8)], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, OrderMetrics]]) -]]> - (total_orders, 10), 'Frequent Customer', >(total_orders, 5), 'Regular Customer', 'Occasional Customer') AS customer_type], where=[>(total_spent, 0)]) +- GroupAggregate(groupBy=[user_id, name], select=[user_id, name, COUNT_RETRACT(DISTINCT order_id) AS total_orders, SUM_RETRACT(price) AS total_spent, AVG_RETRACT(price) AS avg_order_value]) +- Exchange(distribution=[hash[user_id, name]]) - +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,price,order_id0,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER price, VARCHAR(2147483647) order_id0, DOUBLE metric_value)]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($2, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:2;], 1=[LeftInputId:0;LeftFieldIndex:2;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,order_id,payment_id,price,metric_id,order_id0,metric_type,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) metric_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) metric_type, DOUBLE metric_value)]) :- Exchange(distribution=[hash[order_id]]) - : +- Calc(select=[user_id, name, order_id, price]) - : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, INTEGER price, VARCHAR(2147483647) user_id1)]) + : +- Calc(select=[user_id, name, order_id, payment_id, price]) + : +- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) : :- Exchange(distribution=[hash[user_id]]) : : +- ChangelogNormalize(key=[user_id]) : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[order_id]]) - +- Calc(select=[order_id, metric_value]) - +- ChangelogNormalize(key=[metric_id]) - +- Exchange(distribution=[hash[metric_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics, project=[order_id, metric_value, metric_id], metadata=[]]], fields=[order_id, metric_value, metric_id]) + +- ChangelogNormalize(key=[metric_id]) + +- Exchange(distribution=[hash[metric_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics]], fields=[metric_id, order_id, metric_type, metric_value]) ]]> - - - - - - - - = p.price OR p.price < 0) -LEFT JOIN Shipments s - ON p.user_id = s.user_id]]> - - - =($2, $7), <($7, 0)))], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) -]]> - = price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -735,13 +273,13 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], location == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id, location]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -749,13 +287,13 @@ Calc(select=[user_id, name, order_id, payment_id, location]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id, location]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -764,45 +302,16 @@ Calc(select=[user_id, name, order_id, payment_id, location]) - - = p.price OR p.price < 0) -LEFT JOIN Shipments s - ON p.user_id = s.user_id]]> - - - =($2, $7), <($7, 0)))], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) -]]> - =(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -811,192 +320,69 @@ Calc(select=[user_id, name, order_id, payment_id, location]) - - - - - - - - - - - - - - = FLOOR(p.price) OR p.price < 0) -LEFT JOIN Shipments s - ON p.payment_id = s.location]]> - - - =(FLOOR($2), FLOOR($8)), <($8, 0)))], joinType=[left]) - : :- LogicalProject(user_id=[$0], name=[$1], cash=[$2], order_id=[$3], user_id0=[$4], product=[$5], $f6=[UPPER($1)]) - : : +- LogicalJoin(condition=[=($4, $0)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalProject(payment_id=[$0], price=[$1], user_id=[$2], $f3=[UPPER($0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) -]]> - =(FLOOR(cash), FLOOR(price)), <(price, 0)))], select=[user_id,name,cash,order_id,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) -: :- Exchange(distribution=[hash[user_id, $f6]]) -: : +- Calc(select=[user_id, name, cash, order_id, UPPER(name) AS $f6]) -: : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id0, user_id)], select=[user_id,name,cash,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) -: : :- Exchange(distribution=[hash[user_id]]) -: : : +- ChangelogNormalize(key=[user_id]) -: : : +- Exchange(distribution=[hash[user_id]]) -: : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) -: : +- Exchange(distribution=[hash[user_id]]) -: : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) -: +- Exchange(distribution=[hash[user_id, $f3]]) -: +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) -: +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) -+- Exchange(distribution=[hash[location]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments, project=[location], metadata=[]]], fields=[location]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($6, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[payment_id]]) + : +- Calc(select=[user_id, name, cash, order_id, user_id0, product, payment_id, price, user_id1]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, AND(=($0, $9), =($6, $10), OR(>=(FLOOR($2), FLOOR($8)), <($8, 0)))]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:2;, LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:3;]}], select=[user_id,name,cash,order_id,user_id0,product,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- Calc(select=[user_id, name, cash, order_id, user_id0, product, UPPER(name) AS $f6]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($4, $0)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[location]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> - - - - - - - - 1000 THEN 'High' - WHEN p.price > 500 THEN 'Medium' - ELSE 'Low' - END as price_tier, - REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags, - TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created, - COALESCE(up.preferred_category, 'None') as user_preference, - CASE - WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates' - WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates' - ELSE 'Weekly Updates' - END as notification_frequency -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN Payments p - ON u.user_id = p.user_id -LEFT JOIN ProductDetails pd - ON o.product = pd.product_id -LEFT JOIN UserPreferences up - ON u.user_id = up.user_id]]> - - - ($7, 1000), _UTF-16LE'High':VARCHAR(6) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Medium':VARCHAR(6) CHARACTER SET "UTF-16LE", _UTF-16LE'Low':VARCHAR(6) CHARACTER SET "UTF-16LE")], formatted_tags=[REGEXP_REPLACE($13, _UTF-16LE',', _UTF-16LE' | ')], product_created=[TO_TIMESTAMP_LTZ($12, 3)], user_preference=[COALESCE($15, _UTF-16LE'None')], notification_frequency=[CASE(=($16, _UTF-16LE'HIGH'), _UTF-16LE'Frequent Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", =($16, _UTF-16LE'MEDIUM'), _UTF-16LE'Daily Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", _UTF-16LE'Weekly Updates':VARCHAR(16) CHARACTER SET "UTF-16LE")]) -+- LogicalJoin(condition=[=($0, $14)], joinType=[left]) - :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) - : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) - +- LogicalTableScan(table=[[default_catalog, default_database, UserPreferences]]) -]]> - (price, 1000), 'High', >(price, 500), 'Medium', 'Low') AS price_tier, REGEXP_REPLACE(tags, ',', ' | ') AS formatted_tags, TO_TIMESTAMP_LTZ(created_date, 3) AS product_created, COALESCE(preferred_category, 'None') AS user_preference, CASE(=(notification_level, 'HIGH'), 'Frequent Updates', =(notification_level, 'MEDIUM'), 'Daily Updates', 'Weekly Updates') AS notification_frequency]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (user_id)], joinConditions=[=(user_id, user_id2)], select=[user_id,name,product,price,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($0, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,product_id,product_name,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id, name, product, price, description, created_date, tags]) - : +- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (product_id)], joinConditions=[=(product, product_id)], select=[user_id,name,product,price,product_id,description,created_date,tags], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) product_id, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags)]) - : :- Exchange(distribution=[hash[product]]) - : : +- Calc(select=[user_id, name, product, price]) - : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,user_id0,product,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) user_id1)]) - : : :- Exchange(distribution=[hash[user_id]]) - : : : +- ChangelogNormalize(key=[user_id]) - : : : +- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) - : : :- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) - : +- Exchange(distribution=[hash[product_id]]) - : +- ChangelogNormalize(key=[product_id]) - : +- Exchange(distribution=[hash[product_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, description, created_date, tags], metadata=[]]], fields=[product_id, description, created_date, tags]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + :- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, description, created_date, tags]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -1092,92 +427,30 @@ Calc(select=[user_id, UPPER(name) AS user_name_upper, LOWER(product) AS product_ - - = 4 THEN 'High Rating' - WHEN r.rating >= 3 THEN 'Medium Rating' - ELSE 'Low Rating' - END AS rating_category, - TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN ProductDetails pd - ON o.product = pd.product_id -LEFT JOIN Reviews r - ON pd.product_id = r.product_id]]> - - - =($13, 4), _UTF-16LE'High Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", >=($13, 3), _UTF-16LE'Medium Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", _UTF-16LE'Low Rating':VARCHAR(13) CHARACTER SET "UTF-16LE")], days_since_created=[CAST(/INT(Reinterpret(-(CURRENT_DATE, $10)), 86400000)):INTEGER]) -+- LogicalJoin(condition=[=($6, $12)], joinType=[left]) - :- LogicalJoin(condition=[=($5, $6)], joinType=[left]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Reviews]]) -]]> - =(rating, 4), 'High Rating', >=(rating, 3), 'Medium Rating', 'Low Rating') AS rating_category, CAST(/INT(Reinterpret(-(CURRENT_DATE(), created_date)), 86400000) AS INTEGER) AS days_since_created]) -+- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (product_id), noUniqueKey], joinConditions=[=(product, product_id), =(product_id, product_id0)], select=[user_id,name,order_id,product,product_id,product_name,price,created_date,product_id0,rating,review_text], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DATE created_date, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $6), =($6, $12)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:1;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,product_id,product_name,price,weight,created_date,review_id,product_id0,rating,review_text,review_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DOUBLE weight, DATE created_date, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text, DATE review_date)]) :- Exchange(distribution=[hash[product]]) - : +- Calc(select=[user_id, name, order_id, product]) - : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[product_id]]) : +- ChangelogNormalize(key=[product_id]) : +- Exchange(distribution=[hash[product_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, product_name, price, created_date], metadata=[]]], fields=[product_id, product_name, price, created_date]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, price, weight, created_date]) +- Exchange(distribution=[hash[product_id]]) - +- Calc(select=[product_id, rating, review_text]) - +- ChangelogNormalize(key=[review_id]) - +- Exchange(distribution=[hash[review_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Reviews, project=[product_id, rating, review_text, review_id], metadata=[]]], fields=[product_id, rating, review_text, review_id]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Reviews]], fields=[review_id, product_id, rating, review_text, review_date]) ]]> - - - - - =($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner]) - :- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]]) - +- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) - +- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]]) -]]> - - - - - - - - - 100) AS u -JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o - ON u.user_id = o.user_id -LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p - ON u.user_id = p.user_id -LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s - ON u.user_id = s.user_id]]> - - - ($2, 100)]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalProject(user_id=[$1], order_id=[$0], product=[$2]) - : : +- LogicalFilter(condition=[IS NOT NULL($2)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalProject(user_id=[$2], price=[$1]) - : +- LogicalFilter(condition=[>($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalProject(user_id=[$1], location=[$0]) - +- LogicalFilter(condition=[IS NOT NULL($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) -]]> - (cash, 100)]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, cash], metadata=[]]], fields=[user_id, cash]) + : +- ChangelogNormalize(key=[user_id], condition=[>(cash, 100)]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) : +- Calc(select=[user_id, order_id, product], where=[IS NOT NULL(product)]) : +- TableSourceScan(table=[[default_catalog, default_database, Orders, filter=[]]], fields=[order_id, user_id, product]) @@ -1319,193 +520,19 @@ Calc(select=[user_id, order_id, product, price, location]) - - 100 - ) AS p - ON o.user_id = p.user_id -) AS op -ON u.user_id = op.user_id]]> - - - ($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) -]]> - (price, 100)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) -]]> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - @@ -1516,128 +543,53 @@ Sink(table=[default_catalog.default_database.sink1], fields=[a, day, EXPR$2, EXP +- GroupAggregate(advice=[1], groupBy=[a, day], select=[a, day, SUM_RETRACT(b) AS EXPR$2, COUNT_RETRACT(DISTINCT c) AS EXPR$3]) +- Exchange(distribution=[hash[a, day]]) +- Calc(select=[a, day, b0 AS b, c]) - +- MultiJoin(commonJoinKey=[a], joinTypes=[INNER], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(a, d)], select=[a,day,b0,c,d], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) day, BIGINT b0, VARCHAR(2147483647) c, INTEGER d)]) + +- MultiJoin(joinFilter=[=($0, $6)], joinTypes=[[INNER, INNER]], joinConditions=[[true, =($0, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:3;]}], select=[a,b,day,b0,day0,c,d], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) day, BIGINT b0, VARCHAR(2147483647) day0, VARCHAR(2147483647) c, INTEGER d)]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd') AS day]) - : +- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a], metadata=[]]], fields=[a]) + : +- Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd') AS day]) + : +- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b]) +- Exchange(distribution=[hash[d]]) - +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) + +- Calc(select=[b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd')) AS day, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) Sink(table=[default_catalog.default_database.sink2], fields=[a, day, b, c]) +- Calc(select=[a, day, b0 AS b, c]) - +- MultiJoin(commonJoinKey=[a], joinTypes=[INNER], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(a, d)], select=[a,day,b0,c,d], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) day, BIGINT b0, VARCHAR(2147483647) c, INTEGER d)]) + +- MultiJoin(joinFilter=[=($0, $6)], joinTypes=[[INNER, INNER]], joinConditions=[[true, =($0, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:3;]}], select=[a,b,day,b0,day0,c,d], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) day, BIGINT b0, VARCHAR(2147483647) day0, VARCHAR(2147483647) c, INTEGER d)]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd') AS day]) - : +- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a], metadata=[]]], fields=[a]) + : +- Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd') AS day]) + : +- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b]) +- Exchange(distribution=[hash[d]]) - +- Calc(select=[b, c, d], where=[>(b, 100)]) + +- Calc(select=[b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd')) AS day, c, d], where=[>(b, 100)]) +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. related rel plan: -Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) -+- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a], metadata=[]]], fields=[a], changelogMode=[I,UB,UA,D]) - - -]]> - - - - - - - - - - - - - = 0 -JOIN AddressPK a - ON u.user_id = a.user_id - AND a.location IS NOT NULL]]> - - - =($9, 0))], joinType=[inner]) - : :- LogicalJoin(condition=[AND(=($0, $5), IS NOT NULL($6))], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, UsersPK]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, OrdersPK]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, PaymentsPK]]) - +- LogicalTableScan(table=[[default_catalog, default_database, AddressPK]]) -]]> - =(price, 0)]) - : +- Exchange(distribution=[hash[payment_id, user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) + : +- ChangelogNormalize(key=[payment_id, user_id], condition=[>=(price, 0)]) + : +- Exchange(distribution=[hash[payment_id, user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[user_id], condition=[IS NOT NULL(location)]) +- Exchange(distribution=[hash[user_id]]) @@ -1646,113 +598,53 @@ Sink(table=[default_catalog.default_database.sink_four_way], fields=[user_id, or - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - @@ -2001,322 +753,128 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ]]> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - @@ -2332,102 +890,53 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ]]> - - - - - - - - 10]]> - - - ($7, 10))]) - +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) -]]> - 10)]) + +- Calc(select=[payment_id, price, user_id], where=[(price > 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> @@ -2445,119 +954,56 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id]) - : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[payment_id, user_id], where=[>(price, 10)]) + +- Calc(select=[payment_id, price, user_id], where=[>(price, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id]) - : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[payment_id, user_id], where=[(price > 10)]) + +- Calc(select=[payment_id, price, user_id], where=[(price > 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - - 10]]> - - - ($7, 10))]) - +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) -]]> - (price, 10)]) + +- Calc(select=[payment_id, price, user_id], where=[>(price, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - - - - - - - - - - - - - - - - - - - - - - - - - - 100]]> - - - (-($1, $2), 100)]) - +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], promo_id=[$7], product_key0=[$8], promo_text=[$9]) - +- LogicalJoin(condition=[=($6, $8)], joinType=[left]) - :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], $f6=[-($1, $2)]) - : +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$4], product_key=[$5], quantity=[$6]) - : +- LogicalJoin(condition=[=($3, $5)], joinType=[left]) - : :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], $f3=[-($1, $2)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Products]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Promotions]]) -]]> - (-(price, discount), 100)]) : : +- Exchange(distribution=[hash[product_id]]) : : +- TableSourceScan(table=[[default_catalog, default_database, Products, filter=[]]], fields=[product_id, price, discount]) : +- Exchange(distribution=[hash[product_key]]) - : +- Calc(select=[product_key, quantity]) - : +- ChangelogNormalize(key=[sale_id]) - : +- Exchange(distribution=[hash[sale_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) + : +- ChangelogNormalize(key=[sale_id]) + : +- Exchange(distribution=[hash[sale_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) +- Exchange(distribution=[hash[product_key]]) - +- Calc(select=[product_key, promo_text]) - +- ChangelogNormalize(key=[promo_id]) - +- Exchange(distribution=[hash[promo_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) + +- ChangelogNormalize(key=[promo_id]) + +- Exchange(distribution=[hash[promo_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) ]]> - - - - - - diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java index 10a2a9f3e0cc9..98bf16eb85da8 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java @@ -36,13 +36,23 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.stream.Collectors; /** - * A {@link JoinKeyExtractor} that derives keys based on {@link AttributeRef} mappings provided in - * {@code joinAttributeMap}. It defines how attributes from different input streams are related - * through equi-join conditions, assuming input 0 is the base and subsequent inputs join to - * preceding ones. + * A {@link JoinKeyExtractor} that derives keys from {@link AttributeRef} mappings in {@code + * joinAttributeMap}. It describes how attributes from different inputs are equated via equi-join + * conditions. Input 0 is the base; each subsequent input joins to one of the preceding inputs. + * + *

Example used throughout the comments: t1.id1 = t2.user_id2 and t3.user_id3 = t2.user_id2. All + * three attributes (t1.id1, t2.user_id2, t3.user_id3) represent the same conceptual key. + * + *

The {@code joinAttributeMap} for this example would be structured as follows: + * + *

    + *
  • Key `1` (for t2): A list containing one element `ConditionAttributeRef(leftInputId=0, + * leftFieldIndex=id1_idx, rightInputId=1, rightFieldIndex=user_id2_idx)`. + *
  • Key `2` (for t3): A list containing one element `ConditionAttributeRef(leftInputId=1, + * leftFieldIndex=user_id2_idx, rightInputId=2, rightFieldIndex=user_id3_idx)`. + *
*/ public class AttributeBasedJoinKeyExtractor implements JoinKeyExtractor, Serializable { private static final long serialVersionUID = 1L; @@ -50,9 +60,15 @@ public class AttributeBasedJoinKeyExtractor implements JoinKeyExtractor, Seriali private final Map> joinAttributeMap; private final List inputTypes; - // Cache for pre-computed key extraction structures - private final Map> inputIdToExtractorsMap; - private final Map> inputKeyFieldIndices; + // Cache for pre-computed key extraction structures. + // leftKeyExtractorsMap: extractors that read the left side (joined row so far) + // using the same attribute order as in joinAttributeMap. + private final Map> leftKeyExtractorsMap; + // rightKeyExtractorsMap: extractors to extract the right-side key from each input. + private final Map> rightKeyExtractorsMap; + + // Data structures for the "common join key" shared by all inputs. + // Input 0 provides the canonical order and defines commonJoinKeyType. private final Map> commonJoinKeyExtractors; private RowType commonJoinKeyType; @@ -70,18 +86,19 @@ public AttributeBasedJoinKeyExtractor( final List inputTypes) { this.joinAttributeMap = joinAttributeMap; this.inputTypes = inputTypes; - this.inputIdToExtractorsMap = new HashMap<>(); - this.inputKeyFieldIndices = new HashMap<>(); + this.leftKeyExtractorsMap = new HashMap<>(); + this.rightKeyExtractorsMap = new HashMap<>(); this.commonJoinKeyExtractors = new HashMap<>(); initializeCaches(); initializeCommonJoinKeyStructures(); + validateKeyStructures(); } // ==================== Public Interface Methods ==================== @Override - public RowData getJoinKey(RowData row, int inputId) { + public RowData getJoinKey(final RowData row, final int inputId) { if (inputId == 0) { return null; } @@ -91,50 +108,51 @@ public RowData getJoinKey(RowData row, int inputId) { return null; } - final List keyFieldIndices = inputKeyFieldIndices.get(inputId); - if (keyFieldIndices == null || keyFieldIndices.isEmpty()) { + final List keyExtractors = rightKeyExtractorsMap.get(inputId); + if (keyExtractors == null || keyExtractors.isEmpty()) { return null; } - return buildKeyRow(row, inputId, keyFieldIndices); + return buildKeyRowFromSourceRow(row, keyExtractors); } @Override - public RowData getLeftSideJoinKey(int depth, RowData joinedRowData) { + public RowData getLeftSideJoinKey(final int depth, final RowData joinedRowData) { if (depth == 0) { return null; } - List keyExtractors = inputIdToExtractorsMap.get(depth); + final List keyExtractors = leftKeyExtractorsMap.get(depth); if (keyExtractors == null || keyExtractors.isEmpty()) { return null; } - return buildKeyRow(keyExtractors, joinedRowData); + return buildKeyRowFromJoinedRow(keyExtractors, joinedRowData); } @Override @Nullable - public RowType getJoinKeyType(int inputId) { + public RowType getJoinKeyType(final int inputId) { if (inputId == 0) { return null; } - final List keyFieldIndices = createJoinKeyFieldInputExtractors(inputId); - if (keyFieldIndices.isEmpty()) { + final List keyExtractors = this.rightKeyExtractorsMap.get(inputId); + + if (keyExtractors == null || keyExtractors.isEmpty()) { return null; } - return buildJoinKeyType(inputId, keyFieldIndices); + return buildJoinKeyType(inputId, keyExtractors); } @Override - public int[] getJoinKeyIndices(int inputId) { - final List keyFieldIndices = this.inputKeyFieldIndices.get(inputId); + public int[] getJoinKeyIndices(final int inputId) { + final List keyFieldIndices = this.rightKeyExtractorsMap.get(inputId); if (keyFieldIndices == null) { return new int[0]; } - return keyFieldIndices.stream().mapToInt(i -> i).toArray(); + return keyFieldIndices.stream().mapToInt(KeyExtractor::getFieldIndexInSourceRow).toArray(); } @Override @@ -144,18 +162,18 @@ public RowType getCommonJoinKeyType() { } @Override - public @Nullable RowData getCommonJoinKey(RowData row, int inputId) { - List extractors = commonJoinKeyExtractors.get(inputId); + public @Nullable RowData getCommonJoinKey(final RowData row, final int inputId) { + final List extractors = commonJoinKeyExtractors.get(inputId); if (extractors == null || extractors.isEmpty()) { return null; } - return buildCommonJoinKey(row, extractors); + return buildKeyRowFromSourceRow(row, extractors); } @Override - public int[] getCommonJoinKeyIndices(int inputId) { - List extractors = commonJoinKeyExtractors.get(inputId); + public int[] getCommonJoinKeyIndices(final int inputId) { + final List extractors = commonJoinKeyExtractors.get(inputId); if (extractors == null || extractors.isEmpty()) { return new int[0]; } @@ -168,55 +186,92 @@ public int[] getCommonJoinKeyIndices(int inputId) { private void initializeCaches() { if (this.inputTypes != null) { for (int i = 0; i < this.inputTypes.size(); i++) { - this.inputIdToExtractorsMap.put(i, createLeftJoinKeyFieldExtractors(i)); - this.inputKeyFieldIndices.put(i, createJoinKeyFieldInputExtractors(i)); + this.leftKeyExtractorsMap.put(i, createLeftJoinKeyFieldExtractors(i)); + this.rightKeyExtractorsMap.put(i, createRightJoinKeyExtractors(i)); } } } - private List createLeftJoinKeyFieldExtractors(int depth) { - if (depth == 0) { + private List createLeftJoinKeyFieldExtractors(final int inputId) { + if (inputId == 0) { return Collections.emptyList(); } - List attributeMapping = joinAttributeMap.get(depth); + final List attributeMapping = joinAttributeMap.get(inputId); if (attributeMapping == null || attributeMapping.isEmpty()) { return Collections.emptyList(); } - List keyExtractors = new ArrayList<>(); - for (ConditionAttributeRef entry : attributeMapping) { - AttributeRef leftAttrRef = getLeftAttributeRef(depth, entry); + final List keyExtractors = new ArrayList<>(); + for (final ConditionAttributeRef entry : attributeMapping) { + final AttributeRef leftAttrRef = getLeftAttributeRef(inputId, entry); keyExtractors.add(createKeyExtractor(leftAttrRef)); } - keyExtractors.sort( - Comparator.comparingInt(KeyExtractor::getInputIdToAccess) - .thenComparingInt(KeyExtractor::getFieldIndexInSourceRow)); return keyExtractors; } - private static AttributeRef getLeftAttributeRef(int depth, ConditionAttributeRef entry) { - AttributeRef leftAttrRef = new AttributeRef(entry.leftInputId, entry.leftFieldIndex); - if (leftAttrRef.inputId >= depth) { + private List createRightJoinKeyExtractors(final int inputId) { + if (inputId == 0) { + return Collections.emptyList(); + } + + final List attributeMapping = joinAttributeMap.get(inputId); + if (attributeMapping == null) { + return Collections.emptyList(); + } + + final List keyExtractors = new ArrayList<>(); + for (final ConditionAttributeRef entry : attributeMapping) { + final AttributeRef rightAttrRef = getRightAttributeRef(inputId, entry); + keyExtractors.add(createKeyExtractor(rightAttrRef)); + } + + return keyExtractors; + } + + private static AttributeRef getLeftAttributeRef( + final int inputId, final ConditionAttributeRef entry) { + final AttributeRef leftAttrRef = new AttributeRef(entry.leftInputId, entry.leftFieldIndex); + if (leftAttrRef.inputId >= inputId) { throw new IllegalStateException( - "Invalid joinAttributeMap configuration for depth " - + depth + "Invalid joinAttributeMap configuration for inputId " + + inputId + ". Left attribute " + leftAttrRef + " does not reference a previous input (< " - + depth + + inputId + ")."); } return leftAttrRef; } - private KeyExtractor createKeyExtractor(AttributeRef attrRef) { - RowType rowType = inputTypes.get(attrRef.inputId); + private static AttributeRef getRightAttributeRef( + final int inputId, final ConditionAttributeRef entry) { + final AttributeRef rightAttrRef = + new AttributeRef(entry.rightInputId, entry.rightFieldIndex); + // For a given join step (e.g., input 2 joining to a previous input), the "right" + // attribute in the join condition belongs to the current input (input 2). + if (rightAttrRef.inputId != inputId) { + throw new IllegalStateException( + "Invalid joinAttributeMap configuration for inputId " + + inputId + + ". Right attribute " + + rightAttrRef + + " must reference the current input (" + + inputId + + ")."); + } + return rightAttrRef; + } + + private KeyExtractor createKeyExtractor(final AttributeRef attrRef) { + final RowType rowType = inputTypes.get(attrRef.inputId); validateFieldIndex(attrRef.inputId, attrRef.fieldIndex, rowType); - LogicalType fieldType = rowType.getTypeAt(attrRef.fieldIndex); + final LogicalType fieldType = rowType.getTypeAt(attrRef.fieldIndex); - // Calculate absolute field index by summing up field counts of previous inputs + // Absolute field index within the concatenated joined row = + // current field index + sum(field counts of all previous inputs). int absoluteFieldIndex = attrRef.fieldIndex; for (int i = 0; i < attrRef.inputId; i++) { absoluteFieldIndex += inputTypes.get(i).getFieldCount(); @@ -225,71 +280,45 @@ private KeyExtractor createKeyExtractor(AttributeRef attrRef) { return new KeyExtractor(attrRef.inputId, attrRef.fieldIndex, absoluteFieldIndex, fieldType); } - private List createJoinKeyFieldInputExtractors(int inputId) { - final List attributeMapping = joinAttributeMap.get(inputId); - if (attributeMapping == null) { - return Collections.emptyList(); - } - - return attributeMapping.stream() - .filter(rightAttrRef -> rightAttrRef.rightInputId == inputId) - .map(rightAttrRef -> rightAttrRef.rightFieldIndex) - .distinct() - .sorted() - .collect(Collectors.toList()); - } - // ==================== Key Building Methods ==================== - private RowData buildKeyRow(List keyExtractors, RowData joinedRowData) { + private RowData buildKeyRowFromJoinedRow( + final List keyExtractors, final RowData joinedRowData) { if (keyExtractors.isEmpty()) { return null; } - GenericRowData keyRow = new GenericRowData(keyExtractors.size()); + final GenericRowData keyRow = new GenericRowData(keyExtractors.size()); for (int i = 0; i < keyExtractors.size(); i++) { keyRow.setField(i, keyExtractors.get(i).getLeftSideKey(joinedRowData)); } return keyRow; } - private GenericRowData buildKeyRow( - RowData sourceRow, int inputId, List keyFieldIndices) { - final GenericRowData keyRow = new GenericRowData(keyFieldIndices.size()); - final RowType rowType = inputTypes.get(inputId); - - for (int i = 0; i < keyFieldIndices.size(); i++) { - final int fieldIndex = keyFieldIndices.get(i); - validateFieldIndex(inputId, fieldIndex, rowType); - - final LogicalType fieldType = rowType.getTypeAt(fieldIndex); - final RowData.FieldGetter fieldGetter = - RowData.createFieldGetter(fieldType, fieldIndex); - final Object value = fieldGetter.getFieldOrNull(sourceRow); - keyRow.setField(i, value); + private GenericRowData buildKeyRowFromSourceRow( + final RowData sourceRow, final List keyExtractors) { + if (keyExtractors.isEmpty()) { + return null; } - return keyRow; - } - private RowData buildCommonJoinKey(RowData row, List extractors) { - GenericRowData commonJoinKeyRow = new GenericRowData(extractors.size()); - - for (int i = 0; i < extractors.size(); i++) { - commonJoinKeyRow.setField(i, extractors.get(i).getRightSideKey(row)); + final GenericRowData keyRow = new GenericRowData(keyExtractors.size()); + for (int i = 0; i < keyExtractors.size(); i++) { + keyRow.setField(i, keyExtractors.get(i).getRightSideKey(sourceRow)); } - return commonJoinKeyRow; + return keyRow; } - private RowType buildJoinKeyType(int inputId, List keyFieldIndices) { + private RowType buildJoinKeyType(final int inputId, final List keyExtractors) { final RowType originalRowType = inputTypes.get(inputId); - final LogicalType[] keyTypes = new LogicalType[keyFieldIndices.size()]; - final String[] keyNames = new String[keyFieldIndices.size()]; + final LogicalType[] keyTypes = new LogicalType[keyExtractors.size()]; + final String[] keyNames = new String[keyExtractors.size()]; - for (int i = 0; i < keyFieldIndices.size(); i++) { - final int fieldIndex = keyFieldIndices.get(i); + for (int i = 0; i < keyExtractors.size(); i++) { + final KeyExtractor extractor = keyExtractors.get(i); + final int fieldIndex = extractor.getFieldIndexInSourceRow(); validateFieldIndex(inputId, fieldIndex, originalRowType); - keyTypes[i] = originalRowType.getTypeAt(fieldIndex); + keyTypes[i] = extractor.fieldType; keyNames[i] = originalRowType.getFieldNames().get(fieldIndex) + "_key"; } @@ -298,6 +327,34 @@ private RowType buildJoinKeyType(int inputId, List keyFieldIndices) { // ==================== Common Key Methods ==================== + /** + * Builds the data structures that describe the common join key shared by all inputs. + * + *

Algorithm: + * + *

    + *
  1. Collect all attributes referenced by any equality condition. + *
  2. Run union-find: for each equi-join condition, union the two attributes so that directly + * or transitively equal attributes end up in the same set. + *
  3. Form equivalence sets by grouping attributes that share the same union-find root. + *
  4. Keep only those sets that touch every join step (see {@link + * #isCommonConceptualAttributeSet(Set)}). Each kept set is one conceptual key common to + * all inputs. + *
  5. For each input, pick that input's attributes from the kept sets (in a consistent order) + * and create per-input key extractors. For input 0, also build the {@link RowType} of the + * common key. + *
+ * + *

Example (same throughout): + * + *

    + *
  1. Join conditions: t1.id1 = t2.user_id2 and t3.user_id3 = t2.user_id2. + *
  2. Union-find groups attributes into { (t1,id1), (t2,user_id2), (t3,user_id3) }. + *
  3. The group touches both steps (t2↔t1 and t3↔t2), so it is the common key. + *
  4. Create extractors for t1.id1, t2.user_id2, and t3.user_id3. + *
  5. Define a one-field commonJoinKeyType based on t1.id1. + *
+ */ private void initializeCommonJoinKeyStructures() { this.commonJoinKeyType = null; @@ -312,28 +369,43 @@ private void initializeCommonJoinKeyStructures() { return; } - Map parent = new HashMap<>(); - Map rank = new HashMap<>(); - Set allAttrRefs = collectAllAttributeRefs(); + // Maps an attribute to the representative of its equivalence set. + final Map attributeToRoot = new HashMap<>(); + // Used for the union-by-rank optimization to keep the union-find tree shallow. + final Map rootRank = new HashMap<>(); + final Set allAttrRefs = collectAllAttributeRefs(); if (allAttrRefs.isEmpty()) { return; } - initializeDisjointSets(parent, rank, allAttrRefs); - processJoinConditions(parent, rank); - Map> equivalenceSets = - buildEquivalenceSets(parent, allAttrRefs); - List> commonConceptualAttributeSets = - findCommonConceptualAttributeSets(equivalenceSets); + initializeDisjointSets(attributeToRoot, rootRank, allAttrRefs); + findAttributeRoots(attributeToRoot, rootRank); + final Map> equivalenceSets = + buildEquivalenceSets(attributeToRoot, allAttrRefs); - processCommonAttributes(commonConceptualAttributeSets); + // From all equivalence sets, keep only those that touch every join step. + final List> commonConceptualAttributeSets = + findCommonConceptualAttributeSets(equivalenceSets); + processCommonAttributes(commonConceptualAttributeSets, attributeToRoot); } + /** + * Returns every attribute reference (input id + field index) present in the configured equality + * conditions. + * + *

Example: with t1.id1 = t2.user_id2 and t3.user_id3 = t2.user_id2, this returns { (t1,id1), + * (t2,user_id2), (t3,user_id3) }. + */ private Set collectAllAttributeRefs() { - Set allAttrRefs = new HashSet<>(); - for (List mapping : joinAttributeMap.values()) { - for (ConditionAttributeRef attrRef : mapping) { + final Set allAttrRefs = new HashSet<>(); + for (final Map.Entry> entry : + joinAttributeMap.entrySet()) { + // Skip joinAttributeMap for key 0 where there are no join conditions to the left + if (entry.getKey() == 0) { + continue; + } + for (final ConditionAttributeRef attrRef : entry.getValue()) { allAttrRefs.add(new AttributeRef(attrRef.leftInputId, attrRef.leftFieldIndex)); allAttrRefs.add(new AttributeRef(attrRef.rightInputId, attrRef.rightFieldIndex)); } @@ -341,43 +413,70 @@ private Set collectAllAttributeRefs() { return allAttrRefs; } + /** + * Initializes union-find: every attribute is the root of its own set with rank 0. + * + *

Example: attributes {(t1,id1), (t2,user_id2), (t3,user_id3)} start with attributeToRoot[a] + * = a. + */ private void initializeDisjointSets( - Map parent, - Map rank, - Set allAttrRefs) { - for (AttributeRef attrRef : allAttrRefs) { - parent.put(attrRef, attrRef); - rank.put(attrRef, 0); + final Map attributeToRoot, + final Map rootRank, + final Set allAttrRefs) { + for (final AttributeRef attrRef : allAttrRefs) { + attributeToRoot.put(attrRef, attrRef); + rootRank.put(attrRef, 0); } } - private void processJoinConditions( - Map parent, Map rank) { - for (List mapping : joinAttributeMap.values()) { - for (ConditionAttributeRef condition : mapping) { + /** + * Applies all equi-join conditions to union-find by uniting the corresponding attribute sets. + * + *

Example: unite (t2,user_id2) with (t1,id1), then (t3,user_id3) with (t2,user_id2), + * yielding one set { (t1,id1), (t2,user_id2), (t3,user_id3) }. + */ + private void findAttributeRoots( + final Map attributeToRoot, + final Map rootRank) { + for (final Map.Entry> entry : + joinAttributeMap.entrySet()) { + // Skip joinAttributeMap for key 0 where there are no join conditions to the left + if (entry.getKey() == 0) { + continue; + } + for (final ConditionAttributeRef condition : entry.getValue()) { unionAttributeSets( - parent, - rank, + attributeToRoot, + rootRank, new AttributeRef(condition.leftInputId, condition.leftFieldIndex), new AttributeRef(condition.rightInputId, condition.rightFieldIndex)); } } } + /** Converts the union-find forest into a map from root → full equivalence set. */ private Map> buildEquivalenceSets( - Map parent, Set allAttrRefs) { - Map> equivalenceSets = new HashMap<>(); - for (AttributeRef attrRef : allAttrRefs) { - AttributeRef root = findAttributeSet(parent, attrRef); + final Map attributeToRoot, + final Set allAttrRefs) { + final Map> equivalenceSets = new HashMap<>(); + for (final AttributeRef attrRef : allAttrRefs) { + final AttributeRef root = findAttributeSet(attributeToRoot, attrRef); equivalenceSets.computeIfAbsent(root, k -> new HashSet<>()).add(attrRef); } return equivalenceSets; } + /** + * From all equivalence sets, keep only those that intersect every join step. A set is "common" + * only if each step has at least one attribute in the set. + * + *

Example: with steps (t2.user_id2 = t1.id1) and (t3.user_id3 = t2.user_id2), { (t1,id1), + * (t2,user_id2), (t3,user_id3) } is kept; a set touching only one step is discarded. + */ private List> findCommonConceptualAttributeSets( - Map> equivalenceSets) { - List> commonConceptualAttributeSets = new ArrayList<>(); - for (Set eqSet : equivalenceSets.values()) { + final Map> equivalenceSets) { + final List> commonConceptualAttributeSets = new ArrayList<>(); + for (final Set eqSet : equivalenceSets.values()) { if (isCommonConceptualAttributeSet(eqSet)) { commonConceptualAttributeSets.add(eqSet); } @@ -385,18 +484,25 @@ private List> findCommonConceptualAttributeSets( return commonConceptualAttributeSets; } - private boolean isCommonConceptualAttributeSet(Set eqSet) { + /** + * Returns true if the given equivalence set intersects every configured join step. For each + * step, we check whether the set contains any left or right attribute of that step. + * + *

Example: with steps (t2.user_id2 = t1.id1) and (t3.user_id3 = t2.user_id2): - { (t1,id1), + * (t2,user_id2), (t3,user_id3) } → true - { (t1,other), (t2,other) } → false + */ + private boolean isCommonConceptualAttributeSet(final Set eqSet) { if (joinAttributeMap.isEmpty()) { return false; } - for (List conditionsForStep : joinAttributeMap.values()) { + for (final List conditionsForStep : joinAttributeMap.values()) { if (conditionsForStep.isEmpty()) { return false; } boolean foundInThisStep = false; - for (ConditionAttributeRef condition : conditionsForStep) { + for (final ConditionAttributeRef condition : conditionsForStep) { if (eqSet.contains( new AttributeRef(condition.leftInputId, condition.leftFieldIndex)) || eqSet.contains( @@ -413,10 +519,20 @@ private boolean isCommonConceptualAttributeSet(Set eqSet) { return true; } - private void processCommonAttributes(List> commonConceptualAttributeSets) { + /** + * For each input, select its attributes that belong to the common sets and initialize + * extractors. If any input contributes none, the multi-join is invalid (exception). + * + *

Example: from { (t1,id1), (t2,user_id2), (t3,user_id3) }, input 0 uses id1, input 1 uses + * user_id2, input 2 uses user_id3. If some input had none, we would throw an exception. + */ + private void processCommonAttributes( + final List> commonConceptualAttributeSets, + final Map attributeToRoot) { for (int currentInputId = 0; currentInputId < inputTypes.size(); currentInputId++) { final List commonAttrsForThisInput = - findCommonAttributesForInput(currentInputId, commonConceptualAttributeSets); + findCommonAttributesForInput( + currentInputId, commonConceptualAttributeSets, attributeToRoot); if (commonAttrsForThisInput.isEmpty()) { // This indicates that there is no common join key among all inputs. @@ -433,40 +549,69 @@ private void processCommonAttributes(List> commonConceptualAtt } } + /** + * For a given input, pick at most one attribute per common set (first match), then sort by + * field index. This defines the field order of the common key for that input. + * + *

Example: if input 1 participates in two common sets with fields user_id and account_id, + * this returns [account_id, user_id] if account_id's index < user_id's index. + */ private List findCommonAttributesForInput( - int currentInputId, List> commonConceptualAttributeSets) { - List commonAttrsForThisInput = new ArrayList<>(); - for (Set eqSet : commonConceptualAttributeSets) { - for (AttributeRef attrRef : eqSet) { + final int currentInputId, + final List> commonConceptualAttributeSets, + final Map attributeToRoot) { + final List commonAttrsForThisInput = new ArrayList<>(); + for (final Set eqSet : commonConceptualAttributeSets) { + for (final AttributeRef attrRef : eqSet) { if (attrRef.inputId == currentInputId) { commonAttrsForThisInput.add(attrRef); break; } } } - commonAttrsForThisInput.sort(Comparator.comparingInt(attr -> attr.fieldIndex)); + + // Important: ensure a consistent conceptual attribute ordering derived from roots. + // The common key fields must have a canonical order across all inputs. This is + // achieved by sorting based on the properties of the union-find root attribute + // for each conceptual key set. We sort first by the root's field index and then + // by the current attribute's field index as a tie-breaker for stability. This + // ensures that input 0's attribute order defines the canonical order. + commonAttrsForThisInput.sort( + Comparator.comparingInt( + a -> { + AttributeRef root = attributeToRoot.get(a); + return root != null ? root.fieldIndex : -1; + }) + // This is for stable ordering when two roots happen to have the same + // fieldIndex + .thenComparingInt(a -> a.fieldIndex)); + return commonAttrsForThisInput; } + /** + * Creates extractor objects and key type metadata for an input. Input 0 defines {@code + * commonJoinKeyType} shared across all inputs. + * + *

Example: for input 1 with common attributes [user_id], we create an extractor for that + * field and name it "user_id_common" with its logical type. + */ private void processInputCommonAttributes( - int currentInputId, List commonAttrsForThisInput) { + final int currentInputId, final List commonAttrsForThisInput) { final List extractors = new ArrayList<>(); - final LogicalType[] keyFieldTypes = new LogicalType[commonAttrsForThisInput.size()]; final String[] keyFieldNames = new String[commonAttrsForThisInput.size()]; final RowType originalRowType = inputTypes.get(currentInputId); for (int i = 0; i < commonAttrsForThisInput.size(); i++) { final AttributeRef attr = commonAttrsForThisInput.get(i); - validateFieldIndex(currentInputId, attr.fieldIndex, originalRowType); - final LogicalType fieldType = originalRowType.getTypeAt(attr.fieldIndex); - extractors.add( - new KeyExtractor(currentInputId, attr.fieldIndex, attr.fieldIndex, fieldType)); - keyFieldTypes[i] = fieldType; + extractors.add(createKeyExtractor(attr)); keyFieldNames[i] = originalRowType.getFieldNames().get(attr.fieldIndex) + "_common"; } this.commonJoinKeyExtractors.put(currentInputId, extractors); + final LogicalType[] keyFieldTypes = + extractors.stream().map(e -> e.fieldType).toArray(LogicalType[]::new); if (currentInputId == 0 && !extractors.isEmpty()) { this.commonJoinKeyType = RowType.of(keyFieldTypes, keyFieldNames); } @@ -474,7 +619,8 @@ private void processInputCommonAttributes( // ==================== Helper Methods ==================== - private void validateFieldIndex(int inputId, int fieldIndex, RowType rowType) { + private void validateFieldIndex( + final int inputId, final int fieldIndex, final RowType rowType) { if (fieldIndex >= rowType.getFieldCount() || fieldIndex < 0) { throw new IndexOutOfBoundsException( "joinAttributeMap references field index " @@ -487,39 +633,166 @@ private void validateFieldIndex(int inputId, int fieldIndex, RowType rowType) { } private static AttributeRef findAttributeSet( - Map parent, AttributeRef item) { - if (!parent.get(item).equals(item)) { - parent.put(item, findAttributeSet(parent, parent.get(item))); + final Map attributeToRoot, final AttributeRef item) { + if (!attributeToRoot.get(item).equals(item)) { + attributeToRoot.put(item, findAttributeSet(attributeToRoot, attributeToRoot.get(item))); } - return parent.get(item); + return attributeToRoot.get(item); } private static void unionAttributeSets( - Map parent, - Map rank, - AttributeRef a, - AttributeRef b) { - AttributeRef rootA = findAttributeSet(parent, a); - AttributeRef rootB = findAttributeSet(parent, b); - + final Map attributeToRoot, + final Map rootRank, + final AttributeRef a, + final AttributeRef b) { + final AttributeRef rootA = findAttributeSet(attributeToRoot, a); + final AttributeRef rootB = findAttributeSet(attributeToRoot, b); + + // Standard union-by-rank implementation to merge sets. The resulting root + // depends on the rank of the sets, not on attribute properties like inputId. + // A subsequent sorting step establishes a canonical ordering for the common key fields. + // Following this logic, the root will be the common join key attributes from input 0 if (!rootA.equals(rootB)) { - if (rank.get(rootA) < rank.get(rootB)) { - parent.put(rootA, rootB); - } else if (rank.get(rootA) > rank.get(rootB)) { - parent.put(rootB, rootA); + if (rootRank.get(rootA) < rootRank.get(rootB)) { + attributeToRoot.put(rootA, rootB); + } else if (rootRank.get(rootA) > rootRank.get(rootB)) { + attributeToRoot.put(rootB, rootA); } else { - parent.put(rootB, rootA); - rank.put(rootA, rank.get(rootA) + 1); + attributeToRoot.put(rootB, rootA); + rootRank.put(rootA, rootRank.get(rootA) + 1); + } + } + } + + // ==================== Validation Methods ==================== + + /** + * Validates internal key structures for consistency: + * + *

    + *
  1. For every input id, the number of left-side extractors equals the number of right-side + * key-field indices. + *
  2. If a common join key is defined, then for every input id the number of common key + * extractors equals the number of fields in {@code commonJoinKeyType}. + *
  3. If a common join key is defined, each extractor's logical type matches the + * corresponding field type in {@code commonJoinKeyType}. + *
+ * + *

Throws {@link IllegalStateException} if any inconsistency is found. + */ + public void validateKeyStructures() { + final int numInputs = inputTypes == null ? 0 : inputTypes.size(); + + for (int inputId = 0; inputId < numInputs; inputId++) { + final List leftExtractors = leftKeyExtractorsMap.get(inputId); + final List rightExtractors = rightKeyExtractorsMap.get(inputId); + + final int extractorsLength = + validateExtractorsLength(leftExtractors, rightExtractors, inputId); + + for (int j = 0; j < extractorsLength; j++) { + final KeyExtractor rightExtractor = rightExtractors.get(j); + final KeyExtractor leftExtractor = leftExtractors.get(j); + + if (leftExtractor == null || rightExtractor == null) { + throw new IllegalStateException( + "Null extractor found when validating key structures for field " + + j + + " on input " + + inputId + + ": left extractor " + + leftExtractor + + ", right extractor " + + rightExtractor + + "."); + } + + final LogicalType leftType = leftExtractor.fieldType; + final LogicalType rightType = rightExtractor.fieldType; + if (!Objects.equals(leftType.getTypeRoot(), rightType.getTypeRoot())) { + throw new IllegalStateException( + "Type mismatch for join key field " + + j + + " on input " + + inputId + + ": left type " + + leftType.getTypeRoot() + + " vs right type " + + rightType.getTypeRoot() + + "."); + } + } + } + + if (this.commonJoinKeyType != null) { + final int expectedCommonFields = this.commonJoinKeyType.getFieldCount(); + + for (int inputId = 0; inputId < numInputs; inputId++) { + final List commonExtractors = commonJoinKeyExtractors.get(inputId); + final int actual = commonExtractors == null ? 0 : commonExtractors.size(); + + if (actual != expectedCommonFields) { + throw new IllegalStateException( + "Mismatch in common key counts for input " + + inputId + + ": extractors (" + + actual + + ") vs commonJoinKeyType fields (" + + expectedCommonFields + + ")."); + } + + for (int i = 0; i < expectedCommonFields; i++) { + final LogicalType extractorType = commonExtractors.get(i).fieldType; + final LogicalType expectedType = this.commonJoinKeyType.getTypeAt(i); + + if (!Objects.equals(extractorType.getTypeRoot(), expectedType.getTypeRoot())) { + throw new IllegalStateException( + "Type mismatch for common key field " + + i + + " on input " + + inputId + + ": extractor type " + + extractorType.getTypeRoot() + + " vs commonJoinKeyType " + + expectedType.getTypeRoot() + + "."); + } + } } } } + private static int validateExtractorsLength( + final List leftExtractors, + final List rightExtractors, + final int inputId) { + final int leftSize = leftExtractors == null ? 0 : leftExtractors.size(); + final int rightSize = rightExtractors == null ? 0 : rightExtractors.size(); + + if (leftSize != rightSize) { + throw new IllegalStateException( + "Mismatch in key counts for input " + + inputId + + ": left extractors (" + + leftSize + + ") vs right extractors (" + + rightSize + + ")."); + } + return leftSize; + } + // ==================== Inner Classes ==================== - /** Helper class to store pre-computed information for extracting a key part. */ - // TODO we actually need int[] for the indices - // because we can have multiple common join keys as fields - // this whole file will be refactored in a next ticket + /** + * Helper class to store pre-computed information for extracting a key part. + * + *

This class uses two separate {@link RowData.FieldGetter} instances because a key extractor + * may need to extract a field from the left (already joined) side, which is a wide, + * concatenated row, or the right (current input) side, which is a single source row. The field + * indices for these two cases are different. + */ private static final class KeyExtractor implements Serializable { private static final long serialVersionUID = 1L; @@ -527,41 +800,57 @@ private static final class KeyExtractor implements Serializable { private final int fieldIndexInSourceRow; private final int absoluteFieldIndex; private final LogicalType fieldType; - private transient RowData.FieldGetter fieldGetter; + private transient RowData.FieldGetter leftFieldGetter; + private transient RowData.FieldGetter rightFieldGetter; public KeyExtractor( - int inputIdToAccess, - int fieldIndexInSourceRow, - int absoluteFieldIndex, - LogicalType fieldType) { + final int inputIdToAccess, + final int fieldIndexInSourceRow, + final int absoluteFieldIndex, + final LogicalType fieldType) { this.inputIdToAccess = inputIdToAccess; this.fieldIndexInSourceRow = fieldIndexInSourceRow; this.absoluteFieldIndex = absoluteFieldIndex; this.fieldType = fieldType; - this.fieldGetter = + this.rightFieldGetter = RowData.createFieldGetter(this.fieldType, this.fieldIndexInSourceRow); - } - - public Object getRightSideKey(RowData joinedRowData) { - if (joinedRowData == null) { + this.leftFieldGetter = + RowData.createFieldGetter(this.fieldType, this.absoluteFieldIndex); + } + + /** + * Returns the key value from a single input row (right/current side of the current join + * step). + * + *

Example: for a step joining t1 (input 0) and t2 (input 1) t1.id1 = t2.user_id2, the + * extractor configured for t2 (input 1) returns t2.user_id2 value from the provided t2 row. + */ + public Object getRightSideKey(final RowData row) { + if (row == null) { return null; } - if (this.fieldGetter == null) { - this.fieldGetter = + if (this.rightFieldGetter == null) { + this.rightFieldGetter = RowData.createFieldGetter(this.fieldType, this.fieldIndexInSourceRow); } - return this.fieldGetter.getFieldOrNull(joinedRowData); + return this.rightFieldGetter.getFieldOrNull(row); } - public Object getLeftSideKey(RowData joinedRowData) { + /** + * Returns the key value from the already-joined row (left side of the current join step). + * + *

Example: for a step joining t1 (input 0) and t2 (input 1) t1.id1 = t2.user_id2, the + * extractor configured for t2 (input 1) returns t1.id1 value from the provided t1 row. + */ + public Object getLeftSideKey(final RowData joinedRowData) { if (joinedRowData == null) { return null; } - if (this.fieldGetter == null) { - this.fieldGetter = + if (this.leftFieldGetter == null) { + this.leftFieldGetter = RowData.createFieldGetter(this.fieldType, this.absoluteFieldIndex); } - return this.fieldGetter.getFieldOrNull(joinedRowData); + return this.leftFieldGetter.getFieldOrNull(joinedRowData); } public int getInputIdToAccess() { @@ -572,12 +861,14 @@ public int getFieldIndexInSourceRow() { return fieldIndexInSourceRow; } - private void readObject(java.io.ObjectInputStream in) + private void readObject(final java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { in.defaultReadObject(); if (this.fieldType != null) { - this.fieldGetter = + this.rightFieldGetter = RowData.createFieldGetter(this.fieldType, this.fieldIndexInSourceRow); + this.leftFieldGetter = + RowData.createFieldGetter(this.fieldType, this.absoluteFieldIndex); } } } @@ -591,20 +882,20 @@ public AttributeRef() { // Default constructor for deserialization } - public AttributeRef(int inputId, int fieldIndex) { + public AttributeRef(final int inputId, final int fieldIndex) { this.inputId = inputId; this.fieldIndex = fieldIndex; } @Override - public boolean equals(Object o) { + public boolean equals(final Object o) { if (this == o) { return true; } if (o == null || getClass() != o.getClass()) { return false; } - AttributeRef that = (AttributeRef) o; + final AttributeRef that = (AttributeRef) o; return inputId == that.inputId && fieldIndex == that.fieldIndex; } @@ -631,7 +922,10 @@ public ConditionAttributeRef() { } public ConditionAttributeRef( - int leftInputId, int leftFieldIndex, int rightInputId, int rightFieldIndex) { + final int leftInputId, + final int leftFieldIndex, + final int rightInputId, + final int rightFieldIndex) { this.leftInputId = leftInputId; this.leftFieldIndex = leftFieldIndex; this.rightInputId = rightInputId; @@ -639,14 +933,14 @@ public ConditionAttributeRef( } @Override - public boolean equals(Object o) { + public boolean equals(final Object o) { if (this == o) { return true; } if (o == null || getClass() != o.getClass()) { return false; } - ConditionAttributeRef that = (ConditionAttributeRef) o; + final ConditionAttributeRef that = (ConditionAttributeRef) o; return leftInputId == that.leftInputId && leftFieldIndex == that.leftFieldIndex && rightInputId == that.rightInputId diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/multijoin/StreamingFourWayMixedOuterJoinOperatorTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/multijoin/StreamingFourWayMixedOuterJoinOperatorTest.java index 9b5faad697e76..cfa5b8ebf9ce6 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/multijoin/StreamingFourWayMixedOuterJoinOperatorTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/multijoin/StreamingFourWayMixedOuterJoinOperatorTest.java @@ -25,6 +25,7 @@ import org.apache.flink.table.runtime.operators.join.stream.utils.JoinInputSideSpec; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; @@ -1181,24 +1182,25 @@ protected RowType createInputTypeInfo(int inputIndex) { String userIdFieldName = String.format("user_id_%d", inputIndex); String pkFieldName = String.format("pk_%d", inputIndex); // Generic PK name - if (inputIndex == 0) { // Users: user_id (VARCHAR), pk (VARCHAR), details (BIGINT) + if (inputIndex == 0) { // Users: user_id (CHAR NOT NULL), pk (VARCHAR), details (BIGINT) return RowType.of( new LogicalType[] { - VarCharType.STRING_TYPE, VarCharType.STRING_TYPE, new BigIntType() + new CharType(false, 20), VarCharType.STRING_TYPE, new BigIntType() }, new String[] { userIdFieldName, pkFieldName, String.format("details_%d", inputIndex) }); } else if (inputIndex - == 3) { // Shipments: user_id (VARCHAR), pk (VARCHAR), details (BIGINT) + == 3) { // Shipments: user_id (CHAR NOT NULL), pk (VARCHAR), details (BIGINT) return RowType.of( new LogicalType[] { - VarCharType.STRING_TYPE, VarCharType.STRING_TYPE, new BigIntType() + new CharType(false, 20), VarCharType.STRING_TYPE, new BigIntType() }, new String[] { userIdFieldName, pkFieldName, String.format("details_%d", inputIndex) }); - } else { // Orders (1), Payments (2): user_id (VARCHAR), pk (VARCHAR), details (VARCHAR) + } else { // Orders (1), Payments (2): user_id (CHAR NOT NULL), pk (CHAR NOT NULL), details + // (VARCHAR) return super.createInputTypeInfo(inputIndex); } } From 5c439ce7e47ebc6b6dfa479169f4f88b6a0a508d Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Fri, 26 Sep 2025 16:42:34 +0200 Subject: [PATCH 3/4] [FLINK-38445][table] Upsert keys test plans without sinkMaterialize --- .../plan/stream/sql/MultiJoinTest.java | 160 ++ .../planner/plan/stream/sql/MultiJoinTest.xml | 1489 +++++++---------- .../AttributeBasedJoinKeyExtractor.java | 17 +- 3 files changed, 816 insertions(+), 850 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java index 81cc1a10ef9e3..d71142c199fe1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java @@ -1625,4 +1625,164 @@ void testCrossJoinUnnestWithMultiJoinInsert() { + "CROSS JOIN UNNEST(split(REGEXP_REPLACE(d.data, '^\\[\"|\"\\]$', '') , '\", \"')) AS T(edata)\n" + "WHERE NOT (s.location IS NOT NULL)"); } + + @Test + void testPreservesUpsertKeyTwoWayLeftJoinOrders() { + util.tableEnv() + .executeSql( + "CREATE TABLE sink_two_way (" + + " `user_id` STRING NOT NULL," + + " `order_id` STRING NOT NULL," + + " product STRING," + + " user_region_id INT," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "INSERT INTO sink_two_way " + + "SELECT" + + " o.user_id," + + " o.order_id," + + " o.product," + + " u.region_id " + + "FROM OrdersPK o " + + "LEFT JOIN UsersPK u" + + " ON u.user_id = o.user_id"); + } + + @Test + void testPreservesUpsertKeyTwoWayInnerJoinOrders() { + util.tableEnv() + .executeSql( + "CREATE TABLE sink_two_way (" + + " `user_id` STRING NOT NULL," + + " `order_id` STRING NOT NULL," + + " product STRING," + + " user_region_id INT," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "INSERT INTO sink_two_way " + + "SELECT" + + " o.user_id," + + " o.order_id," + + " o.product," + + " u.region_id " + + "FROM UsersPK u " + + "INNER JOIN OrdersPK o " + + " ON u.user_id = o.user_id"); + } + + @Test + void testPreservesUpsertKeyTwoWayInnerJoinOrdersDoesNot() { + util.tableEnv() + .executeSql( + "CREATE TABLE OrdersSimplePK (" + + " order_id STRING NOT NULL," + + " user_id STRING NOT NULL," + + " product STRING," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (order_id) NOT ENFORCED" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE sink_two_way (" + + " `user_id` STRING NOT NULL," + + " `order_id` STRING NOT NULL," + + " product STRING," + + " user_region_id INT," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`order_id`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "INSERT INTO sink_two_way " + + "SELECT" + + " o.user_id," + + " o.order_id," + + " o.product," + + " u.region_id " + + "FROM UsersPK u " + + "INNER JOIN OrdersSimplePK o " + + " ON u.user_id = o.user_id"); + } + + @Test + void testPreservesUpsertKeyThreeWayJoin() { + util.tableEnv() + .executeSql( + "CREATE TABLE sink_three_way (" + + " `user_id` STRING NOT NULL," + + " `order_id` STRING NOT NULL," + + " `user_id2` STRING NOT NULL," + + " `payment_id` STRING NOT NULL," + + " `user_id3` STRING NOT NULL," + + " `description` STRING," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`, `user_id2`, `payment_id`, `user_id3`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "INSERT INTO sink_three_way " + + "SELECT" + + " o.user_id," + + " o.order_id," + + " p.user_id," + + " p.payment_id," + + " u.user_id," + + " u.description " + + "FROM UsersPK u " + + "JOIN OrdersPK o" + + " ON o.user_id = u.user_id " + + "JOIN PaymentsPK p" + + " ON o.user_id = p.user_id"); + } + + @Test + void testPreservesUpsertKeyFourWayComplex() { + util.tableEnv() + .executeSql( + "CREATE TABLE sink_four_way (" + + " user_id_0 STRING NOT NULL," + + " order_id STRING NOT NULL," + + " user_id_1 STRING NOT NULL," + + " payment_id STRING NOT NULL," + + " user_id_2 STRING NOT NULL," + + " name STRING," + + " location STRING," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id_0`, `order_id`, `user_id_1`, `payment_id`, `user_id_2`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "INSERT INTO sink_four_way " + + "SELECT" + + " u.user_id," + + " o.order_id," + + " o.user_id," + + " p.payment_id," + + " p.user_id," + + " u.name," + + " a.location " + + "FROM UsersPK u " + + "JOIN OrdersPK o" + + " ON u.user_id = o.user_id AND o.product IS NOT NULL " + + "JOIN PaymentsPK p" + + " ON u.user_id = p.user_id AND p.price >= 0 " + + "JOIN AddressPK a" + + " ON u.user_id = a.user_id AND a.location IS NOT NULL"); + } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index eb29e71a3b7b7..8196aa986d271 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -16,253 +16,43 @@ See the License for the specific language governing permissions and limitations under the License. --> - - - (order_count, 0)]) -+- GroupAggregate(groupBy=[user_id, name, category_name], select=[user_id, name, category_name, COUNT_RETRACT(DISTINCT order_id) AS order_count, SUM_RETRACT(quantity) AS total_items, SUM_RETRACT($f5) AS total_value, AVG_RETRACT(unit_price) AS avg_item_price, MAX_RETRACT(price) AS max_payment, MIN_RETRACT(price) AS min_payment, COUNT_RETRACT($f8) AS bulk_orders]) - +- Exchange(distribution=[hash[user_id, name, category_name]]) - +- Calc(select=[user_id, name, category_name, order_id, quantity, *(quantity, unit_price) AS $f5, unit_price, price, CASE(>(quantity, 5), 1, null:INTEGER) AS $f8]) - +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($8, $11), =($0, $16)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:8;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:8;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price,category_id,category_name,parent_category,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - :- Exchange(distribution=[hash[user_id]]) - : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($3, $7)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;], 1=[LeftInputId:0;LeftFieldIndex:3;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) - : :- Exchange(distribution=[hash[order_id]]) - : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) - : : :- Exchange(distribution=[hash[user_id]]) - : : : +- ChangelogNormalize(key=[user_id]) - : : : +- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[order_id]]) - : +- ChangelogNormalize(key=[item_id]) - : +- Exchange(distribution=[hash[item_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) - :- Exchange(distribution=[hash[category_id]]) - : +- ChangelogNormalize(key=[category_id]) - : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, parent_category]) - +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) -]]> - - - - - (total_sales, 0)]) -+- GroupAggregate(groupBy=[category_name], select=[category_name, COUNT_RETRACT(DISTINCT user_id) AS unique_users, COUNT_RETRACT(sale_id) AS total_sales, SUM_RETRACT(amount) AS total_revenue, AVG_RETRACT(amount) AS avg_sale_amount, MAX_RETRACT(amount) AS max_sale_amount]) - +- Exchange(distribution=[hash[category_name]]) - +- Calc(select=[category_name, user_id, sale_id, amount]) - +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $9), =($5, $6)), =($0, $11)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:3;, LeftInputId:1;LeftFieldIndex:2;RightInputId:2;RightFieldIndex:0;], 3=[LeftInputId:0;LeftFieldIndex:0;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,category_id,category_name,parent_category,user_id1,sale_id,user_id2,product_id,amount,sale_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) product_id, DOUBLE amount, DATE sale_date)]) - :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[category_id]) - : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Categories]], fields=[category_id, category_name, parent_category, user_id]) - +- Exchange(distribution=[hash[user_id]]) - +- ChangelogNormalize(key=[sale_id]) - +- Exchange(distribution=[hash[sale_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, user_id, product_id, amount, sale_date]) -]]> - - - - - (budget, 600000)]) - : +- Exchange(distribution=[hash[dept_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Departments, filter=[]]], fields=[dept_id, dept_name, budget]) - +- Exchange(distribution=[hash[dept_id]]) - +- Calc(select=[project_id, project_name, dept_id]) - +- ChangelogNormalize(key=[project_id], condition=[=(status, 'ACTIVE')]) - +- Exchange(distribution=[hash[project_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Projects, filter=[]]], fields=[project_id, project_name, dept_id, status]) -]]> - - - - - - - - - - - - - - - - - - - - (price, 1000)), 'High-Value Premium', is_premium, 'Premium', >(price, 500), 'Standard High-Value', 'Standard') AS product_tier, CASE(AND(>=(rating, 4), is_verified), 'Highly Recommended', >=(rating, 3), 'Recommended', >=(rating, 2), 'Average', 'Not Recommended') AS recommendation_status, CASE(>(discount_rate, 0.2), *(price, -(1, discount_rate)), CAST(price AS DOUBLE)) AS final_price]) -+- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($5, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:5;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,category_id,category_name,is_premium,discount_rate,review_id,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) - :- Exchange(distribution=[hash[product]]) - : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - :- Exchange(distribution=[hash[category_id]]) - : +- ChangelogNormalize(key=[category_id]) - : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, is_premium, discount_rate]) - +- Exchange(distribution=[hash[product_id]]) - +- ChangelogNormalize(key=[review_id]) - +- Exchange(distribution=[hash[review_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) -]]> + + + = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]> - - - + (total_orders, 10), 'Frequent Customer', >(total_orders, 5), 'Regular Customer', 'Occasional Customer') AS customer_type], where=[>(total_spent, 0)]) -+- GroupAggregate(groupBy=[user_id, name], select=[user_id, name, COUNT_RETRACT(DISTINCT order_id) AS total_orders, SUM_RETRACT(price) AS total_spent, AVG_RETRACT(price) AS avg_order_value]) - +- Exchange(distribution=[hash[user_id, name]]) - +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($2, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:2;], 1=[LeftInputId:0;LeftFieldIndex:2;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,order_id,payment_id,price,metric_id,order_id0,metric_type,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) metric_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) metric_type, DOUBLE metric_value)]) - :- Exchange(distribution=[hash[order_id]]) - : +- Calc(select=[user_id, name, order_id, payment_id, price]) - : +- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - +- Exchange(distribution=[hash[order_id]]) - +- ChangelogNormalize(key=[metric_id]) - +- Exchange(distribution=[hash[metric_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics]], fields=[metric_id, order_id, metric_type, metric_value]) -]]> - - - - - =($2, $7), <($7, 0)))], joinType=[inner]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) ]]> - - =($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) - :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) +Calc(select=[user_id_0, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[AND(=($0, $7), OR(>=($2, $6), <($6, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $7), OR(>=($2, $6), <($6, 0))), =($7, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) + :- Exchange(distribution=[hash[user_id_0]]) + : +- ChangelogNormalize(key=[user_id_0]) + : +- Exchange(distribution=[hash[user_id_0]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) + :- Exchange(distribution=[hash[user_id_1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + :- Exchange(distribution=[hash[user_id_2]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) + +- Exchange(distribution=[hash[user_id_3]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) ]]> =($2, $7), <($7, 0)))], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -272,324 +62,229 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], location +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) == Optimized Physical Plan == -Calc(select=[user_id, name, order_id, payment_id, location]) -+- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) - :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) +Calc(select=[user_id_0, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[AND(=($0, $7), OR(>=($2, $6), <($6, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $7), OR(>=($2, $6), <($6, 0))), =($7, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) + :- Exchange(distribution=[hash[user_id_0]]) + : +- ChangelogNormalize(key=[user_id_0]) + : +- Exchange(distribution=[hash[user_id_0]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) + :- Exchange(distribution=[hash[user_id_1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + :- Exchange(distribution=[hash[user_id_2]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) + +- Exchange(distribution=[hash[user_id_3]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) == Optimized Execution Plan == -Calc(select=[user_id, name, order_id, payment_id, location]) -+- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) - :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) +Calc(select=[user_id_0, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[AND(=($0, $7), OR(>=($2, $6), <($6, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $7), OR(>=($2, $6), <($6, 0))), =($7, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) + :- Exchange(distribution=[hash[user_id_0]]) + : +- ChangelogNormalize(key=[user_id_0]) + : +- Exchange(distribution=[hash[user_id_0]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) + :- Exchange(distribution=[hash[user_id_1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + :- Exchange(distribution=[hash[user_id_2]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) + +- Exchange(distribution=[hash[user_id_3]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) ]]> - + + = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]> + + =($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) - :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) +LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], location=[$9]) ++- LogicalJoin(condition=[=($8, $10)], joinType=[left]) + :- LogicalJoin(condition=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinType=[inner]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) ]]> - - =($2, $6), <($6, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $7), OR(>=($2, $6), <($6, 0))), =($7, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) + :- Exchange(distribution=[hash[user_id_0]]) + : +- ChangelogNormalize(key=[user_id_0]) + : +- Exchange(distribution=[hash[user_id_0]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) + :- Exchange(distribution=[hash[user_id_1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + :- Exchange(distribution=[hash[user_id_2]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) + +- Exchange(distribution=[hash[user_id_3]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) ]]> - - + + + + + - - =(FLOOR($2), FLOOR($8)), <($8, 0)))]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:2;, LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:3;]}], select=[user_id,name,cash,order_id,user_id0,product,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- Calc(select=[user_id, name, cash, order_id, user_id0, product, UPPER(name) AS $f6]) - : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($4, $0)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) - : : :- Exchange(distribution=[hash[user_id]]) - : : : +- ChangelogNormalize(key=[user_id]) - : : : +- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - +- Exchange(distribution=[hash[location]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) + : +- Calc(select=[user_id_0, name, order_id, payment_id]) + : +- MultiJoin(joinFilter=[=($0, $5)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $3), =($0, $5)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id_0,name,order_id,user_id_1,payment_id,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2)]) + : :- Exchange(distribution=[hash[user_id_0]]) + : : +- ChangelogNormalize(key=[user_id_0]) + : : +- Exchange(distribution=[hash[user_id_0]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) + : :- Exchange(distribution=[hash[user_id_1]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + : +- Exchange(distribution=[hash[user_id_2]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id_2], metadata=[]]], fields=[payment_id, user_id_2]) + +- Exchange(distribution=[hash[user_id_3]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) ]]> - - - + + + - - - + (price, 1000), 'High', >(price, 500), 'Medium', 'Low') AS price_tier, REGEXP_REPLACE(tags, ',', ' | ') AS formatted_tags, TO_TIMESTAMP_LTZ(created_date, 3) AS product_created, COALESCE(preferred_category, 'None') AS user_preference, CASE(=(notification_level, 'HIGH'), 'Frequent Updates', =(notification_level, 'MEDIUM'), 'Daily Updates', 'Weekly Updates') AS notification_frequency]) -+- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($0, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,product_id,product_name,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) - :- Exchange(distribution=[hash[user_id]]) - : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) - :- Exchange(distribution=[hash[product_id]]) - : +- ChangelogNormalize(key=[product_id]) - : +- Exchange(distribution=[hash[product_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, description, created_date, tags]) - +- Exchange(distribution=[hash[user_id]]) - +- ChangelogNormalize(key=[user_id]) - +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, UserPreferences]], fields=[user_id, preferred_category, notification_level]) +LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], location=[$9]) ++- LogicalJoin(condition=[=($8, $10)], joinType=[left]) + :- LogicalJoin(condition=[=($4, $8)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) ]]> - - =(rating, 4), 'High Rating', >=(rating, 3), 'Medium Rating', 'Low Rating') AS rating_category, CAST(/INT(Reinterpret(-(CURRENT_DATE(), created_date)), 86400000) AS INTEGER) AS days_since_created]) -+- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $6), =($6, $12)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:1;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,product_id,product_name,price,weight,created_date,review_id,product_id0,rating,review_text,review_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DOUBLE weight, DATE created_date, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text, DATE review_date)]) - :- Exchange(distribution=[hash[product]]) - : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - :- Exchange(distribution=[hash[product_id]]) - : +- ChangelogNormalize(key=[product_id]) - : +- Exchange(distribution=[hash[product_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, price, weight, created_date]) - +- Exchange(distribution=[hash[product_id]]) - +- ChangelogNormalize(key=[review_id]) - +- Exchange(distribution=[hash[review_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Reviews]], fields=[review_id, product_id, rating, review_text, review_date]) +Calc(select=[user_id_0, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT, LEFT]], joinConditions=[[true, =($0, $3), =($3, $5), =($5, $7)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:1;LeftFieldIndex:1;RightInputId:2;RightFieldIndex:1;], 3=[LeftInputId:2;LeftFieldIndex:1;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,order_id,user_id_1,payment_id,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) + :- Exchange(distribution=[hash[user_id_0]]) + : +- ChangelogNormalize(key=[user_id_0]) + : +- Exchange(distribution=[hash[user_id_0]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) + :- Exchange(distribution=[hash[user_id_1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + :- Exchange(distribution=[hash[user_id_2]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id_2], metadata=[]]], fields=[payment_id, user_id_2]) + +- Exchange(distribution=[hash[user_id_3]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) ]]> - - + + + + + =($rowtime, -($rowtime0, 60000:INTERVAL MINUTE)), <=($rowtime, +($rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, $rowtime, id0, price, $rowtime0]) - :- Exchange(distribution=[hash[id]]) - : +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) - : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, $rowtime]) - +- Exchange(distribution=[hash[id]]) - +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) - +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, $rowtime]) +LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6]) ++- LogicalJoin(condition=[=($4, $8)], joinType=[full]) + :- LogicalJoin(condition=[=($0, $4)], joinType=[full]) + : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) ]]> - - - - + + + + + (cash, 100)]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) - :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id, order_id, product], where=[IS NOT NULL(product)]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, filter=[]]], fields=[order_id, user_id, product]) - :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id, price], where=[>(price, 50)]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[], project=[price, user_id], metadata=[]]], fields=[price, user_id]) - +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[user_id, location], where=[IS NOT NULL(location)]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments, filter=[]]], fields=[location, user_id]) +LogicalProject(id=[$0], val=[$1], price=[$4]) ++- LogicalJoin(condition=[AND(=($0, $3), >=($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner]) + :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]]) ]]> - - (price, 100)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) +Calc(select=[id, val, price]) ++- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-60000, leftUpperBound=60000, leftTimeIndex=2, rightTimeIndex=2], where=[AND(=(id, id0), >=(rowtime, -(rowtime0, 60000:INTERVAL MINUTE)), <=(rowtime, +(rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, rowtime, id0, price, rowtime0]) + :- Exchange(distribution=[hash[id]]) + : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)]) + : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, rowtime]) + +- Exchange(distribution=[hash[id]]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, rowtime]) ]]> - - + + + = 0 JOIN AddressPK a ON u.user_id = a.user_id AND a.location IS NOT NULL]]> + + (b, 100)]) - +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) - -advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). -advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. - -related rel plan: -Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) -+- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b], changelogMode=[I,UB,UA,D]) - - +LogicalSink(table=[default_catalog.default_database.sink_four_way], fields=[user_id, order_id, user_id0, payment_id, user_id1, name, location]) ++- LogicalProject(user_id=[$0], order_id=[$4], user_id0=[$5], payment_id=[$7], user_id1=[$8], name=[$1], location=[$11]) + +- LogicalJoin(condition=[AND(=($0, $10), IS NOT NULL($11))], joinType=[inner]) + :- LogicalJoin(condition=[AND(=($0, $8), >=($9, 0))], joinType=[inner]) + : :- LogicalJoin(condition=[AND(=($0, $5), IS NOT NULL($6))], joinType=[inner]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, UsersPK]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, OrdersPK]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, PaymentsPK]]) + +- LogicalTableScan(table=[[default_catalog, default_database, AddressPK]]) ]]> - - =(price, 0)]) - : +- Exchange(distribution=[hash[payment_id, user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) + : +- Calc(select=[payment_id, user_id]) + : +- ChangelogNormalize(key=[payment_id, user_id], condition=[>=(price, 0)]) + : +- Exchange(distribution=[hash[payment_id, user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[user_id], condition=[IS NOT NULL(location)]) +- Exchange(distribution=[hash[user_id]]) @@ -598,53 +293,81 @@ Sink(table=[default_catalog.default_database.sink_four_way], fields=[user_id, or + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - + + + + - - + + + + + + + + + + + + - - - + + + - - - + - - - - - + + + - - - + - - - - - - - - + + + + + + + - + + + + + + + + + + + + + + + + + + 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) +MultiJoin(joinFilter=[true], joinTypes=[[INNER, INNER]], joinConditions=[[true, true]], joinAttributeMap=[{}], select=[user_id_0,name,order_id,payment_id], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) payment_id)]) +:- Exchange(distribution=[single]) +: +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, true]], joinAttributeMap=[{}], select=[user_id_0,name,order_id], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id)]) +: :- Exchange(distribution=[single]) +: : +- ChangelogNormalize(key=[user_id_0]) +: : +- Exchange(distribution=[hash[user_id_0]]) +: : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) +: +- Exchange(distribution=[single]) +: +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id], metadata=[]]], fields=[order_id]) ++- Exchange(distribution=[single]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id], metadata=[]]], fields=[payment_id]) ]]> - - - + + + + + + + + + + + + + 10]]> + + + ($7, 10))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - -== Optimized Physical Plan == -Calc(select=[user_id, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) -+- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[payment_id, price, user_id], where=[>(price, 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) - -== Optimized Execution Plan == -Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) -+- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - :- Exchange(distribution=[hash[user_id]]) - : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) - :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[payment_id, price, user_id], where=[(price > 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - - (price, 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) +Calc(select=[user_id_0, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) ++- MultiJoin(joinFilter=[=($0, $4)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $2), =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id_0,order_id,user_id_1,payment_id,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2)]) + :- Exchange(distribution=[hash[user_id_0]]) + : +- Calc(select=[user_id_0]) + : +- ChangelogNormalize(key=[user_id_0], condition=[=(name, 'Gus')]) + : +- Exchange(distribution=[hash[user_id_0]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) + :- Exchange(distribution=[hash[user_id_1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + +- Exchange(distribution=[hash[user_id_2]]) + +- Calc(select=[payment_id, user_id_2], where=[>(price, 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) ]]> - - + + + + + - - - + - - + + + + + - - - - - (-(price, discount), 100)]) - : : +- Exchange(distribution=[hash[product_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Products, filter=[]]], fields=[product_id, price, discount]) - : +- Exchange(distribution=[hash[product_key]]) - : +- ChangelogNormalize(key=[sale_id]) - : +- Exchange(distribution=[hash[sale_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) - +- Exchange(distribution=[hash[product_key]]) - +- ChangelogNormalize(key=[promo_id]) - +- Exchange(distribution=[hash[promo_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) + + + ($7, 10))]) + +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +== Optimized Physical Plan == +Calc(select=[user_id_0, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) ++- MultiJoin(joinFilter=[=($0, $4)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $2), =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id_0,order_id,user_id_1,payment_id,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2)]) + :- Exchange(distribution=[hash[user_id_0]]) + : +- Calc(select=[user_id_0]) + : +- ChangelogNormalize(key=[user_id_0], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) + : +- Exchange(distribution=[hash[user_id_0]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) + :- Exchange(distribution=[hash[user_id_1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + +- Exchange(distribution=[hash[user_id_2]]) + +- Calc(select=[payment_id, user_id_2], where=[>(price, 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) + +== Optimized Execution Plan == +Calc(select=[user_id_0, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) ++- MultiJoin(joinFilter=[=($0, $4)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $2), =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id_0,order_id,user_id_1,payment_id,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2)]) + :- Exchange(distribution=[hash[user_id_0]]) + : +- Calc(select=[user_id_0]) + : +- ChangelogNormalize(key=[user_id_0], condition=[(name = 'Gus')]) + : +- Exchange(distribution=[hash[user_id_0]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) + :- Exchange(distribution=[hash[user_id_1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) + +- Exchange(distribution=[hash[user_id_2]]) + +- Calc(select=[payment_id, user_id_2], where=[(price > 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) ]]> - - + + + 10]]> + + ($7, 10))]) + +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + + + 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) ]]> diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java index 98bf16eb85da8..d5e073c8b9b67 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java @@ -212,10 +212,6 @@ private List createLeftJoinKeyFieldExtractors(final int inputId) { } private List createRightJoinKeyExtractors(final int inputId) { - if (inputId == 0) { - return Collections.emptyList(); - } - final List attributeMapping = joinAttributeMap.get(inputId); if (attributeMapping == null) { return Collections.emptyList(); @@ -684,6 +680,19 @@ public void validateKeyStructures() { final int numInputs = inputTypes == null ? 0 : inputTypes.size(); for (int inputId = 0; inputId < numInputs; inputId++) { + if (inputId == 0) { + if (!leftKeyExtractorsMap.get(inputId).isEmpty()) { + throw new IllegalStateException( + "Input 0 should not have left key extractors, but found left extractors " + + leftKeyExtractorsMap.get(inputId) + + "."); + } + + // We skip validating the extracted keys type equality for input 0 + // because it has no left-side extractors. + continue; + } + final List leftExtractors = leftKeyExtractorsMap.get(inputId); final List rightExtractors = rightKeyExtractorsMap.get(inputId); From a67c9896b1d3b4d0cfaac8d2211b4d2541b59c1d Mon Sep 17 00:00:00 2001 From: Dmitriy Linevich <48554952+ldadima@users.noreply.github.com> Date: Thu, 12 Mar 2026 22:55:10 +0700 Subject: [PATCH 4/4] [FLINK-39015][table] Fix key extractor for multi join by changing GenericRowData to BinaryRowData This closes #27508. --- .../exec/stream/MultiJoinSemanticTests.java | 3 +- .../exec/stream/MultiJoinTestPrograms.java | 87 +- .../plan/stream/sql/MultiJoinTest.java | 160 -- .../planner/plan/stream/sql/MultiJoinTest.xml | 1504 ++++++++++------- .../runtime/stream/sql/JoinITCase.scala | 17 + .../stream/StreamingMultiJoinOperator.java | 10 + .../AttributeBasedJoinKeyExtractor.java | 63 +- .../stream/keyselector/JoinKeyExtractor.java | 3 + .../runtime/typeutils/RowDataSerializer.java | 6 +- .../StreamingMultiJoinOperatorTestBase.java | 1 + 10 files changed, 995 insertions(+), 859 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java index 83eb11649ff22..b3b51cc4f79c2 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java @@ -51,8 +51,7 @@ public List programs() { MultiJoinTestPrograms.MULTI_JOIN_LEFT_OUTER_WITH_NULL_KEYS, MultiJoinTestPrograms.MULTI_JOIN_NULL_SAFE_JOIN_WITH_NULL_KEYS, MultiJoinTestPrograms.MULTI_JOIN_MIXED_CHANGELOG_MODES, - MultiJoinTestPrograms - .MULTI_JOIN_WITH_TIME_ATTRIBUTES_IN_CONDITIONS_MATERIALIZATION, + MultiJoinTestPrograms.MULTI_JOIN_WITH_TIME_ATTRIBUTES_IN_CONDITIONS_MATERIALIZATION, MultiJoinTestPrograms.MULTI_JOIN_TWO_WAY_INNER_JOIN_WITH_WHERE_IN, MultiJoinTestPrograms.MULTI_JOIN_THREE_WAY_INNER_JOIN_MULTI_KEY_TYPES, MultiJoinTestPrograms.MULTI_JOIN_FOUR_WAY_MIXED_JOIN_MULTI_KEY_TYPES_SHUFFLED); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java index c19a5a280929a..31442c08972d0 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.table.api.config.OptimizerConfigOptions; -import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.test.program.SinkTestStep; import org.apache.flink.table.test.program.SourceTestStep; import org.apache.flink.table.test.program.TableTestProgram; @@ -171,7 +170,7 @@ public class MultiJoinTestPrograms { "name STRING", "cash INT", "user_id_0 STRING PRIMARY KEY NOT ENFORCED") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .producedValues( Row.ofKind(RowKind.INSERT, "Gus", 100, "1"), Row.ofKind(RowKind.INSERT, "Joe no order", 10, "8"), @@ -195,7 +194,7 @@ public class MultiJoinTestPrograms { "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING", "user_id_1 STRING") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedValues( Row.ofKind(RowKind.INSERT, "order0", "ProdB", "1"), Row.ofKind(RowKind.INSERT, "order6", "ProdF", "6"), @@ -212,7 +211,7 @@ public class MultiJoinTestPrograms { "payment_id STRING PRIMARY KEY NOT ENFORCED", "price INT", "user_id_2 STRING") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedValues( Row.ofKind(RowKind.INSERT, "payment1", 50, "1"), Row.ofKind(RowKind.INSERT, "payment5", -1, "5"), @@ -225,7 +224,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("Shipments") .addSchema("location STRING", "user_id_3 STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .producedValues( Row.ofKind(RowKind.INSERT, "London", "1"), Row.ofKind(RowKind.INSERT, "Paris", "2"), @@ -242,7 +241,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .consumedValues( "+I[3, Nomad, order3, payment3, New York]", "+I[1, Gus, order0, payment1, London]", @@ -518,7 +517,7 @@ public class MultiJoinTestPrograms { "name STRING", "user_id_0 STRING PRIMARY KEY NOT ENFORCED", "cash INT") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "Gus", "1", 100), Row.ofKind(RowKind.INSERT, "Nomad", "3", 50), @@ -543,7 +542,7 @@ public class MultiJoinTestPrograms { "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING", "user_id_1 STRING") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "order2", "ProdB", "2"), Row.ofKind(RowKind.INSERT, "order0", "ProdB", "1"), @@ -561,7 +560,7 @@ public class MultiJoinTestPrograms { "user_id_2 STRING", "payment_id STRING PRIMARY KEY NOT ENFORCED", "price INT") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "3", "3", 30), Row.ofKind(RowKind.INSERT, "1", "1", 50), @@ -575,7 +574,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("Shipments") .addSchema("location STRING", "user_id_3 STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "Paris", "2"), Row.ofKind(RowKind.INSERT, "London", "1"), @@ -593,7 +592,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .consumedBeforeRestore( "+I[1, Gus, order0, 1, London]", "+I[1, Gus, order1, 1, London]", @@ -626,7 +625,7 @@ public class MultiJoinTestPrograms { "name STRING", "cash INT", "user_id_0 STRING PRIMARY KEY NOT ENFORCED") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .producedValues( Row.ofKind(RowKind.INSERT, "Gus", 100, "1"), Row.ofKind(RowKind.INSERT, "Joe no order", 10, "8"), @@ -650,7 +649,7 @@ public class MultiJoinTestPrograms { "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING", "user_id_1 STRING") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedValues( Row.ofKind(RowKind.INSERT, "order0", "ProdB", "1"), Row.ofKind(RowKind.INSERT, "order6", "ProdF", "6"), @@ -667,7 +666,7 @@ public class MultiJoinTestPrograms { "payment_id STRING PRIMARY KEY NOT ENFORCED", "price INT", "user_id_2 STRING") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedValues( Row.ofKind(RowKind.INSERT, "1", 50, "1"), Row.ofKind(RowKind.INSERT, "5", -1, "5"), @@ -680,7 +679,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("Shipments") .addSchema("location STRING", "user_id_3 STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .producedValues( Row.ofKind(RowKind.INSERT, "London", "1"), Row.ofKind(RowKind.INSERT, "Paris", "2"), @@ -697,7 +696,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .consumedValues( "+I[1, Gus, order0, 1, London]", "+I[1, Gus, order1, 1, London]", @@ -729,7 +728,7 @@ public class MultiJoinTestPrograms { "name STRING", "user_id_0 STRING PRIMARY KEY NOT ENFORCED", "cash INT") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "Gus", "1", 100), Row.ofKind(RowKind.INSERT, "Nomad", "3", 50), @@ -754,7 +753,7 @@ public class MultiJoinTestPrograms { "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING", "user_id_1 STRING") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "order2", "ProdB", "2"), Row.ofKind(RowKind.INSERT, "order0", "ProdB", "1"), @@ -772,7 +771,7 @@ public class MultiJoinTestPrograms { "user_id_2 STRING", "payment_id STRING PRIMARY KEY NOT ENFORCED", "price INT") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "3", "payment3", 30), Row.ofKind(RowKind.INSERT, "1", "payment1", 50), @@ -786,7 +785,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("Shipments") .addSchema("location STRING", "user_id_3 STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "Paris", "2"), Row.ofKind(RowKind.INSERT, "London", "1"), @@ -804,7 +803,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .consumedBeforeRestore( "+I[1, Gus, order0, payment1, London]", "+I[1, Gus, order1, payment1, London]", @@ -851,7 +850,7 @@ public class MultiJoinTestPrograms { "user_id_1 STRING NOT NULL", "product STRING", "CONSTRAINT `PRIMARY` PRIMARY KEY (`order_id`, `user_id_1`) NOT ENFORCED") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedBeforeRestore( Row.ofKind( RowKind.INSERT, "order1", "1", "ProdA")) @@ -866,7 +865,7 @@ public class MultiJoinTestPrograms { "user_id_2 STRING NOT NULL", "price INT", "CONSTRAINT `PRIMARY` PRIMARY KEY (`payment_id`, `user_id_2`) NOT ENFORCED") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedBeforeRestore( Row.ofKind( RowKind.INSERT, "payment1", "1", 100), @@ -882,7 +881,7 @@ public class MultiJoinTestPrograms { "user_id_3 STRING NOT NULL", "location STRING", "CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id_3`) NOT ENFORCED") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedBeforeRestore( Row.ofKind(RowKind.INSERT, "1", "London"), Row.ofKind(RowKind.INSERT, "3", "Berlin")) @@ -891,7 +890,7 @@ public class MultiJoinTestPrograms { .build()) .setupTableSink( SinkTestStep.newBuilder("sink") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .addSchema( "user_id_0 STRING NOT NULL", "order_id STRING NOT NULL", @@ -1060,7 +1059,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("AppendTable") .addSchema("id STRING PRIMARY KEY NOT ENFORCED, val STRING") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedValues( Row.ofKind(RowKind.INSERT, "1", "append1"), Row.ofKind(RowKind.INSERT, "2", "append2"), @@ -1069,7 +1068,7 @@ public class MultiJoinTestPrograms { .setupTableSource( SourceTestStep.newBuilder("RetractTable") .addSchema("ref_id STRING, data STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .producedValues( Row.ofKind(RowKind.INSERT, "1", "retract1"), Row.ofKind(RowKind.INSERT, "2", "retract2"), @@ -1081,7 +1080,7 @@ public class MultiJoinTestPrograms { SourceTestStep.newBuilder("UpsertTable") .addSchema( "key_id STRING PRIMARY KEY NOT ENFORCED, status STRING") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .producedValues( Row.ofKind(RowKind.INSERT, "1", "active"), Row.ofKind(RowKind.INSERT, "2", "pending"), @@ -1096,7 +1095,7 @@ public class MultiJoinTestPrograms { "val STRING", "data STRING", "status STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .consumedValues( "+I[1, append1, retract1, active]", "+I[2, append2, retract2, active]", @@ -1123,7 +1122,7 @@ public class MultiJoinTestPrograms { "user_id STRING", "order_id STRING PRIMARY KEY NOT ENFORCED", "product STRING") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .producedValues( Row.ofKind(RowKind.INSERT, "2", "order2", "Product B"), Row.ofKind( @@ -1251,7 +1250,7 @@ public class MultiJoinTestPrograms { .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .addSchema( "user_id INT NOT NULL", "shard_id INT NOT NULL", @@ -1264,7 +1263,7 @@ public class MultiJoinTestPrograms { .build()) .setupTableSource( SourceTestStep.newBuilder("Orders") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .addSchema( "user_id INT NOT NULL", "order_id BIGINT NOT NULL", @@ -1281,7 +1280,7 @@ public class MultiJoinTestPrograms { .build()) .setupTableSink( SinkTestStep.newBuilder("sink") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .addSchema( "`user_id` INT NOT NULL", "`order_id` BIGINT NOT NULL", @@ -1311,7 +1310,7 @@ public class MultiJoinTestPrograms { .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .addSchema( "user_id INT NOT NULL", "shard_id INT NOT NULL", @@ -1324,7 +1323,7 @@ public class MultiJoinTestPrograms { .build()) .setupTableSource( SourceTestStep.newBuilder("Orders") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .addSchema( "user_id INT NOT NULL", "order_id BIGINT NOT NULL", @@ -1339,7 +1338,7 @@ public class MultiJoinTestPrograms { .build()) .setupTableSource( SourceTestStep.newBuilder("Payments") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .addSchema( "user_id INT NOT NULL", "payment_id BIGINT NOT NULL", @@ -1354,7 +1353,7 @@ public class MultiJoinTestPrograms { .build()) .setupTableSink( SinkTestStep.newBuilder("sink") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .addSchema( "`user_id` INT NOT NULL", "`order_id` BIGINT NOT NULL", @@ -1407,7 +1406,7 @@ public class MultiJoinTestPrograms { "auctionDateTime AS TO_TIMESTAMP(auctionTimestamp)", "expires AS TO_TIMESTAMP(expiresTimestamp)", "WATERMARK FOR auctionDateTime AS auctionDateTime - INTERVAL '1' SECOND") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedValues( Row.ofKind( RowKind.INSERT, @@ -1436,7 +1435,7 @@ public class MultiJoinTestPrograms { "bidTimestamp STRING", "bidDateTime AS TO_TIMESTAMP(bidTimestamp)", "WATERMARK FOR bidDateTime AS bidDateTime - INTERVAL '1' SECOND") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .producedValues( Row.ofKind( RowKind.INSERT, @@ -1497,7 +1496,7 @@ public class MultiJoinTestPrograms { .addSchema( "`record_id` STRING PRIMARY KEY NOT ENFORCED", "`user_id` INT") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .producedValues( Row.ofKind(RowKind.INSERT, "record_1", 1), Row.ofKind(RowKind.INSERT, "record_2", 2), @@ -1507,7 +1506,7 @@ public class MultiJoinTestPrograms { SourceTestStep.newBuilder("Users") .addSchema( "`user_id` INT PRIMARY KEY NOT ENFORCED", "`id` STRING") - .addMode(ChangelogMode.upsert()) + .addOption("changelog-mode", "I,UA,D") .producedValues( Row.ofKind(RowKind.INSERT, 1, "record_1"), Row.ofKind(RowKind.INSERT, 2, "record_2"), @@ -1516,7 +1515,7 @@ public class MultiJoinTestPrograms { .setupTableSink( SinkTestStep.newBuilder("sink") .addSchema("record_id STRING", "`user_id` INT", "id STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .consumedValues( // Only records with user_id 1 and 2 should be // included due to WHERE IN clause @@ -1636,7 +1635,7 @@ public class MultiJoinTestPrograms { "k1 STRING", "k2 INT", "k3 BOOLEAN") - .addMode(ChangelogMode.insertOnly()) + .addOption("changelog-mode", "I") .consumedValues( "+I[Gus, order1, payment1, K1, 100, true]", "+I[Bob, order2, payment2, A1, 200, false]") @@ -1809,7 +1808,7 @@ public class MultiJoinTestPrograms { "k2 INT", "k3 BOOLEAN", "k4 STRING") - .addMode(ChangelogMode.all()) + .addOption("changelog-mode", "I,UA,UB,D") .consumedValues( "+I[Bob, order2, payment2, null, A1, 200, false, k4_val2]") .testMaterializedData() diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java index d71142c199fe1..81cc1a10ef9e3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java @@ -1625,164 +1625,4 @@ void testCrossJoinUnnestWithMultiJoinInsert() { + "CROSS JOIN UNNEST(split(REGEXP_REPLACE(d.data, '^\\[\"|\"\\]$', '') , '\", \"')) AS T(edata)\n" + "WHERE NOT (s.location IS NOT NULL)"); } - - @Test - void testPreservesUpsertKeyTwoWayLeftJoinOrders() { - util.tableEnv() - .executeSql( - "CREATE TABLE sink_two_way (" - + " `user_id` STRING NOT NULL," - + " `order_id` STRING NOT NULL," - + " product STRING," - + " user_region_id INT," - + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'" - + ")"); - - util.verifyRelPlanInsert( - "INSERT INTO sink_two_way " - + "SELECT" - + " o.user_id," - + " o.order_id," - + " o.product," - + " u.region_id " - + "FROM OrdersPK o " - + "LEFT JOIN UsersPK u" - + " ON u.user_id = o.user_id"); - } - - @Test - void testPreservesUpsertKeyTwoWayInnerJoinOrders() { - util.tableEnv() - .executeSql( - "CREATE TABLE sink_two_way (" - + " `user_id` STRING NOT NULL," - + " `order_id` STRING NOT NULL," - + " product STRING," - + " user_region_id INT," - + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'" - + ")"); - - util.verifyRelPlanInsert( - "INSERT INTO sink_two_way " - + "SELECT" - + " o.user_id," - + " o.order_id," - + " o.product," - + " u.region_id " - + "FROM UsersPK u " - + "INNER JOIN OrdersPK o " - + " ON u.user_id = o.user_id"); - } - - @Test - void testPreservesUpsertKeyTwoWayInnerJoinOrdersDoesNot() { - util.tableEnv() - .executeSql( - "CREATE TABLE OrdersSimplePK (" - + " order_id STRING NOT NULL," - + " user_id STRING NOT NULL," - + " product STRING," - + " CONSTRAINT `PRIMARY` PRIMARY KEY (order_id) NOT ENFORCED" - + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); - - util.tableEnv() - .executeSql( - "CREATE TABLE sink_two_way (" - + " `user_id` STRING NOT NULL," - + " `order_id` STRING NOT NULL," - + " product STRING," - + " user_region_id INT," - + " CONSTRAINT `PRIMARY` PRIMARY KEY (`order_id`) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'" - + ")"); - - util.verifyRelPlanInsert( - "INSERT INTO sink_two_way " - + "SELECT" - + " o.user_id," - + " o.order_id," - + " o.product," - + " u.region_id " - + "FROM UsersPK u " - + "INNER JOIN OrdersSimplePK o " - + " ON u.user_id = o.user_id"); - } - - @Test - void testPreservesUpsertKeyThreeWayJoin() { - util.tableEnv() - .executeSql( - "CREATE TABLE sink_three_way (" - + " `user_id` STRING NOT NULL," - + " `order_id` STRING NOT NULL," - + " `user_id2` STRING NOT NULL," - + " `payment_id` STRING NOT NULL," - + " `user_id3` STRING NOT NULL," - + " `description` STRING," - + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`, `user_id2`, `payment_id`, `user_id3`) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'" - + ")"); - - util.verifyRelPlanInsert( - "INSERT INTO sink_three_way " - + "SELECT" - + " o.user_id," - + " o.order_id," - + " p.user_id," - + " p.payment_id," - + " u.user_id," - + " u.description " - + "FROM UsersPK u " - + "JOIN OrdersPK o" - + " ON o.user_id = u.user_id " - + "JOIN PaymentsPK p" - + " ON o.user_id = p.user_id"); - } - - @Test - void testPreservesUpsertKeyFourWayComplex() { - util.tableEnv() - .executeSql( - "CREATE TABLE sink_four_way (" - + " user_id_0 STRING NOT NULL," - + " order_id STRING NOT NULL," - + " user_id_1 STRING NOT NULL," - + " payment_id STRING NOT NULL," - + " user_id_2 STRING NOT NULL," - + " name STRING," - + " location STRING," - + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id_0`, `order_id`, `user_id_1`, `payment_id`, `user_id_2`) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'" - + ")"); - - util.verifyRelPlanInsert( - "INSERT INTO sink_four_way " - + "SELECT" - + " u.user_id," - + " o.order_id," - + " o.user_id," - + " p.payment_id," - + " p.user_id," - + " u.name," - + " a.location " - + "FROM UsersPK u " - + "JOIN OrdersPK o" - + " ON u.user_id = o.user_id AND o.product IS NOT NULL " - + "JOIN PaymentsPK p" - + " ON u.user_id = p.user_id AND p.price >= 0 " - + "JOIN AddressPK a" - + " ON u.user_id = a.user_id AND a.location IS NOT NULL"); - } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index 8196aa986d271..d641e2836dbb0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -16,43 +16,258 @@ See the License for the specific language governing permissions and limitations under the License. --> - - - = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]> + + + (order_count, 0)]) ++- GroupAggregate(groupBy=[user_id, name, category_name], select=[user_id, name, category_name, COUNT_RETRACT(DISTINCT order_id) AS order_count, SUM_RETRACT(quantity) AS total_items, SUM_RETRACT($f5) AS total_value, AVG_RETRACT(unit_price) AS avg_item_price, MAX_RETRACT(price) AS max_payment, MIN_RETRACT(price) AS min_payment, COUNT_RETRACT($f8) AS bulk_orders]) + +- Exchange(distribution=[hash[user_id, name, category_name]]) + +- Calc(select=[user_id, name, category_name, order_id, quantity, *(quantity, unit_price) AS $f5, unit_price, price, CASE(>(quantity, 5), 1, null:INTEGER) AS $f8]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $16)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price,category_id,category_name,parent_category,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($8, $11)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:8;], 1=[LeftInputId:0;LeftFieldIndex:8;RightInputId:1;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price,category_id,category_name,parent_category], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category)]) + : :- Exchange(distribution=[hash[product_name]]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($3, $7)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;], 1=[LeftInputId:0;LeftFieldIndex:3;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) + : : :- Exchange(distribution=[hash[order_id]]) + : : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : : : :- Exchange(distribution=[hash[user_id]]) + : : : : +- ChangelogNormalize(key=[user_id]) + : : : : +- Exchange(distribution=[hash[user_id]]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : : +- Exchange(distribution=[hash[order_id]]) + : : +- ChangelogNormalize(key=[item_id]) + : : +- Exchange(distribution=[hash[item_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) + : +- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, parent_category]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +]]> - + + + =($2, $7), <($7, 0)))], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +Calc(select=[category_name, unique_users, total_sales, total_revenue, avg_sale_amount, max_sale_amount], where=[>(total_sales, 0)]) ++- GroupAggregate(groupBy=[category_name], select=[category_name, COUNT_RETRACT(DISTINCT user_id) AS unique_users, COUNT_RETRACT(sale_id) AS total_sales, SUM_RETRACT(amount) AS total_revenue, AVG_RETRACT(amount) AS avg_sale_amount, MAX_RETRACT(amount) AS max_sale_amount]) + +- Exchange(distribution=[hash[category_name]]) + +- Calc(select=[category_name, user_id, sale_id, amount]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $9), =($5, $6)), =($0, $11)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:3;, LeftInputId:1;LeftFieldIndex:2;RightInputId:2;RightFieldIndex:0;], 3=[LeftInputId:0;LeftFieldIndex:0;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,category_id,category_name,parent_category,user_id1,sale_id,user_id2,product_id,amount,sale_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) product_id, DOUBLE amount, DATE sale_date)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Categories]], fields=[category_id, category_name, parent_category, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- ChangelogNormalize(key=[sale_id]) + +- Exchange(distribution=[hash[sale_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, user_id, product_id, amount, sale_date]) +]]> + + + + + (budget, 600000)]) + : +- Exchange(distribution=[hash[dept_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Departments, filter=[]]], fields=[dept_id, dept_name, budget]) + +- Exchange(distribution=[hash[dept_id]]) + +- Calc(select=[project_id, project_name, dept_id]) + +- ChangelogNormalize(key=[project_id], condition=[=(status, 'ACTIVE')]) + +- Exchange(distribution=[hash[project_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Projects, filter=[]]], fields=[project_id, project_name, dept_id, status]) ]]> + + + + + + + =($2, $6), <($6, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $7), OR(>=($2, $6), <($6, 0))), =($7, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT, LEFT, LEFT, LEFT]], joinConditions=[[true, AND(=($1, $4), =($3, $7)), AND(=($2, $8), =($3, $10)), =($10, $12), AND(=($6, $13), =($7, $15)), AND(=($9, $18), =($10, $19))]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:1;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:2;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;], 1=[LeftInputId:0;LeftFieldIndex:1;RightInputId:1;RightFieldIndex:0;, LeftInputId:0;LeftFieldIndex:3;RightInputId:1;RightFieldIndex:3;], 2=[LeftInputId:0;LeftFieldIndex:2;RightInputId:2;RightFieldIndex:0;, LeftInputId:0;LeftFieldIndex:3;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;], 4=[LeftInputId:1;LeftFieldIndex:2;RightInputId:4;RightFieldIndex:0;, LeftInputId:1;LeftFieldIndex:3;RightInputId:4;RightFieldIndex:2;], 5=[LeftInputId:2;LeftFieldIndex:1;RightInputId:5;RightFieldIndex:2;, LeftInputId:2;LeftFieldIndex:2;RightInputId:5;RightFieldIndex:3;]}], select=[assignment_id,user_id,detail_id,common_id,user_id0,name,depart_num,common_id0,detail_id0,creator_nm,common_id1,phase_id,common_id2,org_id,org_name,common_id3,user_id1,name0,depart_num0,common_id4], rowType=[RecordType(VARCHAR(2147483647) assignment_id, VARCHAR(2147483647) user_id, VARCHAR(2147483647) detail_id, VARCHAR(2147483647) common_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) name, VARCHAR(2147483647) depart_num, VARCHAR(2147483647) common_id0, VARCHAR(2147483647) detail_id0, VARCHAR(2147483647) creator_nm, VARCHAR(2147483647) common_id1, VARCHAR(2147483647) phase_id, VARCHAR(2147483647) common_id2, VARCHAR(2147483647) org_id, VARCHAR(2147483647) org_name, VARCHAR(2147483647) common_id3, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) name0, VARCHAR(2147483647) depart_num0, VARCHAR(2147483647) common_id4)]) +:- Exchange(distribution=[hash[common_id]]) +: +- ChangelogNormalize(key=[assignment_id]) +: +- Exchange(distribution=[hash[assignment_id]]) +: +- TableSourceScan(table=[[default_catalog, default_database, Assignments]], fields=[assignment_id, user_id, detail_id, common_id]) +:- Exchange(distribution=[hash[common_id]])(reuse_id=[1]) +: +- ChangelogNormalize(key=[user_id]) +: +- Exchange(distribution=[hash[user_id]]) +: +- TableSourceScan(table=[[default_catalog, default_database, Customers]], fields=[user_id, name, depart_num, common_id]) +:- Exchange(distribution=[hash[common_id]]) +: +- ChangelogNormalize(key=[detail_id]) +: +- Exchange(distribution=[hash[detail_id]]) +: +- TableSourceScan(table=[[default_catalog, default_database, Documents]], fields=[detail_id, creator_nm, common_id]) +:- Exchange(distribution=[hash[common_id]]) +: +- ChangelogNormalize(key=[phase_id]) +: +- Exchange(distribution=[hash[phase_id]]) +: +- TableSourceScan(table=[[default_catalog, default_database, PhaseDetails]], fields=[phase_id, common_id]) +:- Exchange(distribution=[hash[common_id]]) +: +- ChangelogNormalize(key=[org_id]) +: +- Exchange(distribution=[hash[org_id]]) +: +- TableSourceScan(table=[[default_catalog, default_database, Organizations]], fields=[org_id, org_name, common_id]) ++- Reused(reference_id=[1]) +]]> + + + + + + + + + + (price, 1000)), 'High-Value Premium', is_premium, 'Premium', >(price, 500), 'Standard High-Value', 'Standard') AS product_tier, CASE(AND(>=(rating, 4), is_verified), 'Highly Recommended', >=(rating, 3), 'Recommended', >=(rating, 2), 'Average', 'Not Recommended') AS recommendation_status, CASE(>(discount_rate, 0.2), *(price, -(1, discount_rate)), CAST(price AS DOUBLE)) AS final_price]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($5, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:5;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,category_id,category_name,is_premium,discount_rate,review_id,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) + :- Exchange(distribution=[hash[product]]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + :- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, is_premium, discount_rate]) + +- Exchange(distribution=[hash[product_id]]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) +]]> + + + + + (total_orders, 10), 'Frequent Customer', >(total_orders, 5), 'Regular Customer', 'Occasional Customer') AS customer_type], where=[>(total_spent, 0)]) ++- GroupAggregate(groupBy=[user_id, name], select=[user_id, name, COUNT_RETRACT(DISTINCT order_id) AS total_orders, SUM_RETRACT(price) AS total_spent, AVG_RETRACT(price) AS avg_order_value]) + +- Exchange(distribution=[hash[user_id, name]]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($2, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:2;], 1=[LeftInputId:0;LeftFieldIndex:2;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,order_id,payment_id,price,metric_id,order_id0,metric_type,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) metric_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) metric_type, DOUBLE metric_value)]) + :- Exchange(distribution=[hash[order_id]]) + : +- Calc(select=[user_id, name, order_id, payment_id, price]) + : +- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[order_id]]) + +- ChangelogNormalize(key=[metric_id]) + +- Exchange(distribution=[hash[metric_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics]], fields=[metric_id, order_id, metric_type, metric_value]) +]]> + + + + + + + + + + =($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> =($2, $7), <($7, 0)))], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -62,229 +277,332 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) == Optimized Physical Plan == -Calc(select=[user_id_0, name, order_id, payment_id, location]) -+- MultiJoin(joinFilter=[AND(=($0, $7), OR(>=($2, $6), <($6, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $7), OR(>=($2, $6), <($6, 0))), =($7, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) == Optimized Execution Plan == -Calc(select=[user_id_0, name, order_id, payment_id, location]) -+- MultiJoin(joinFilter=[AND(=($0, $7), OR(>=($2, $6), <($6, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $7), OR(>=($2, $6), <($6, 0))), =($7, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> - - = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]> - - + =($2, $7), <($7, 0)))], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> + + =($2, $6), <($6, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $7), OR(>=($2, $6), <($6, 0))), =($7, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($4, $13)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:4;], 1=[LeftInputId:0;LeftFieldIndex:4;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,id,name0,age,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) id, VARCHAR(2147483647) name0, INTEGER age, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id0]]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $11)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,id,name0,age,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) id, VARCHAR(2147483647) name0, INTEGER age, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($1, $7)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:1;], 1=[LeftInputId:0;LeftFieldIndex:1;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,id,name0,age], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) id, VARCHAR(2147483647) name0, INTEGER age)]) + : : :- Exchange(distribution=[hash[name]]) + : : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : : : :- Exchange(distribution=[hash[user_id]]) + : : : : +- ChangelogNormalize(key=[user_id]) + : : : : +- Exchange(distribution=[hash[user_id]]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : : +- Exchange(distribution=[hash[name]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, LookupTable]], fields=[id, name, age]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> - - - - - + + + + =(FLOOR($2), FLOOR($8)), <($8, 0)))]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:2;, LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:3;]}], select=[user_id,name,cash,order_id,user_id0,product,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) + : :- Exchange(distribution=[hash[user_id, $f6]]) + : : +- Calc(select=[user_id, name, cash, order_id, user_id0, product, UPPER(name) AS $f6]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($4, $0)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id, $f3]]) + : +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[location]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> - - - - - + + + + (price, 1000), 'High', >(price, 500), 'Medium', 'Low') AS price_tier, REGEXP_REPLACE(tags, ',', ' | ') AS formatted_tags, TO_TIMESTAMP_LTZ(created_date, 3) AS product_created, COALESCE(preferred_category, 'None') AS user_preference, CASE(=(notification_level, 'HIGH'), 'Frequent Updates', =(notification_level, 'MEDIUM'), 'Daily Updates', 'Weekly Updates') AS notification_frequency]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,product_id,product_name,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) + :- Exchange(distribution=[hash[user_id]]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($5, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,product_id,product_name,description,created_date,tags], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags)]) + : :- Exchange(distribution=[hash[product]]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, description, created_date, tags]) + +- Exchange(distribution=[hash[user_id]]) + +- ChangelogNormalize(key=[user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, UserPreferences]], fields=[user_id, preferred_category, notification_level]) ]]> - - - - - + + =(rating, 4), 'High Rating', >=(rating, 3), 'Medium Rating', 'Low Rating') AS rating_category, CAST(/INT(Reinterpret(-(CURRENT_DATE(), created_date)), 86400000) AS INTEGER) AS days_since_created]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $6), =($6, $12)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:1;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,product_id,product_name,price,weight,created_date,review_id,product_id0,rating,review_text,review_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DOUBLE weight, DATE created_date, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text, DATE review_date)]) + :- Exchange(distribution=[hash[product]]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, price, weight, created_date]) + +- Exchange(distribution=[hash[product_id]]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Reviews]], fields=[review_id, product_id, rating, review_text, review_date]) ]]> + + =($rowtime, -($rowtime0, 60000:INTERVAL MINUTE)), <=($rowtime, +($rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, $rowtime, id0, price, $rowtime0]) + :- Exchange(distribution=[hash[id]]) + : +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) + : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, $rowtime]) + +- Exchange(distribution=[hash[id]]) + +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, $rowtime]) ]]> - - - - - + + =($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner]) - :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]]) - +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) - +- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]]) +Calc(select=[developer_id, person.info.name AS developer_name, ticket_id, reporter.info.priority AS ticket_priority, feedback_id, author.info.rating AS feedback_rating, sub_id, subscriber.info.plan AS subscription_plan]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($9, $13)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:9;], 1=[LeftInputId:0;LeftFieldIndex:9;RightInputId:1;RightFieldIndex:3;]}], select=[developer_id,person,experience_years,ticket_id,reporter,issue,feedback_id,author,message,$f9,sub_id,subscriber,active,$f3], rowType=[RecordType(VARCHAR(2147483647) developer_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, VARCHAR(2147483647) name, VARCHAR(2147483647) region) info) person, INTEGER experience_years, VARCHAR(2147483647) ticket_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, VARCHAR(2147483647) priority) info) reporter, VARCHAR(2147483647) issue, VARCHAR(2147483647) feedback_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, INTEGER rating) info) author, VARCHAR(2147483647) message, VARCHAR(2147483647) $f9, VARCHAR(2147483647) sub_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, VARCHAR(2147483647) plan) info) subscriber, BOOLEAN active, VARCHAR(2147483647) $f3)]) + :- Exchange(distribution=[hash[$f9]]) + : +- Calc(select=[developer_id, person, experience_years, ticket_id, reporter, issue, feedback_id, author, message, person.info.id AS $f9]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($6, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:3;]}], select=[developer_id,person,experience_years,ticket_id,reporter,issue,$f6,feedback_id,author,message,$f3], rowType=[RecordType(VARCHAR(2147483647) developer_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, VARCHAR(2147483647) name, VARCHAR(2147483647) region) info) person, INTEGER experience_years, VARCHAR(2147483647) ticket_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, VARCHAR(2147483647) priority) info) reporter, VARCHAR(2147483647) issue, VARCHAR(2147483647) $f6, VARCHAR(2147483647) feedback_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, INTEGER rating) info) author, VARCHAR(2147483647) message, VARCHAR(2147483647) $f3)]) + : :- Exchange(distribution=[hash[$f6]]) + : : +- Calc(select=[developer_id, person, experience_years, ticket_id, reporter, issue, person.info.id AS $f6]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($3, $7)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;], 1=[LeftInputId:0;LeftFieldIndex:3;RightInputId:1;RightFieldIndex:3;]}], select=[developer_id,person,experience_years,$f3,ticket_id,reporter,issue,$f30], rowType=[RecordType(VARCHAR(2147483647) developer_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, VARCHAR(2147483647) name, VARCHAR(2147483647) region) info) person, INTEGER experience_years, VARCHAR(2147483647) $f3, VARCHAR(2147483647) ticket_id, RecordType:peek_no_expand(RecordType:peek_no_expand(VARCHAR(2147483647) id, VARCHAR(2147483647) priority) info) reporter, VARCHAR(2147483647) issue, VARCHAR(2147483647) $f30)]) + : : :- Exchange(distribution=[hash[$f3]]) + : : : +- Calc(select=[developer_id, person, experience_years, person.info.id AS $f3]) + : : : +- ChangelogNormalize(key=[developer_id]) + : : : +- Exchange(distribution=[hash[developer_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Developers]], fields=[developer_id, person, experience_years]) + : : +- Exchange(distribution=[hash[$f3]]) + : : +- Calc(select=[ticket_id, reporter, issue, reporter.info.id AS $f3]) + : : +- ChangelogNormalize(key=[ticket_id]) + : : +- Exchange(distribution=[hash[ticket_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, SupportTickets]], fields=[ticket_id, reporter, issue]) + : +- Exchange(distribution=[hash[$f3]]) + : +- Calc(select=[feedback_id, author, message, author.info.id AS $f3]) + : +- ChangelogNormalize(key=[feedback_id]) + : +- Exchange(distribution=[hash[feedback_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Feedback]], fields=[feedback_id, author, message]) + +- Exchange(distribution=[hash[$f3]]) + +- Calc(select=[sub_id, subscriber, active, subscriber.info.id AS $f3]) + +- ChangelogNormalize(key=[sub_id]) + +- Exchange(distribution=[hash[sub_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Subscriptions]], fields=[sub_id, subscriber, active]) ]]> + + =(rowtime, -(rowtime0, 60000:INTERVAL MINUTE)), <=(rowtime, +(rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, rowtime, id0, price, rowtime0]) - :- Exchange(distribution=[hash[id]]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)]) - : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, rowtime]) - +- Exchange(distribution=[hash[id]]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)]) - +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, rowtime]) +Calc(select=[user_id, order_id, product, price, location]) ++- MultiJoin(joinFilter=[=($0, $3)], joinTypes=[[INNER, INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $3), =($0, $6), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:0;], 3=[LeftInputId:0;LeftFieldIndex:0;RightInputId:3;RightFieldIndex:0;]}], select=[user_id,name,cash,user_id0,order_id,product,user_id1,price,user_id2,location], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) user_id1, INTEGER price, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) location)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id], condition=[>(cash, 100)]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, order_id, product], where=[IS NOT NULL(product)]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, filter=[]]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, price], where=[>(price, 50)]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[], project=[price, user_id], metadata=[]]], fields=[price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[user_id, location], where=[IS NOT NULL(location)]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments, filter=[]]], fields=[location, user_id]) ]]> - - - = 0 JOIN AddressPK a ON u.user_id = a.user_id AND a.location IS NOT NULL]]> + + + (price, 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) +]]> - + + + =($9, 0))], joinType=[inner]) - : :- LogicalJoin(condition=[AND(=($0, $5), IS NOT NULL($6))], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, UsersPK]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, OrdersPK]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, PaymentsPK]]) - +- LogicalTableScan(table=[[default_catalog, default_database, AddressPK]]) +Sink(table=[default_catalog.default_database.sink1], fields=[a, day, EXPR$2, EXPR$3]) ++- GroupAggregate(advice=[1], groupBy=[a, day], select=[a, day, SUM_RETRACT(b) AS EXPR$2, COUNT_RETRACT(DISTINCT c) AS EXPR$3]) + +- Exchange(distribution=[hash[a, day]]) + +- Calc(select=[a, day, b0 AS b, c]) + +- MultiJoin(joinFilter=[=($0, $6)], joinTypes=[[INNER, INNER]], joinConditions=[[true, =($0, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:3;]}], select=[a,b,day,b0,day0,c,d], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) day, BIGINT b0, VARCHAR(2147483647) day0, VARCHAR(2147483647) c, INTEGER d)]) + :- Exchange(distribution=[hash[a]]) + : +- Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd') AS day]) + : +- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b]) + +- Exchange(distribution=[hash[d]]) + +- Calc(select=[b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd')) AS day, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) + +Sink(table=[default_catalog.default_database.sink2], fields=[a, day, b, c]) ++- Calc(select=[a, day, b0 AS b, c]) + +- MultiJoin(joinFilter=[=($0, $6)], joinTypes=[[INNER, INNER]], joinConditions=[[true, =($0, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:3;]}], select=[a,b,day,b0,day0,c,d], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) day, BIGINT b0, VARCHAR(2147483647) day0, VARCHAR(2147483647) c, INTEGER d)]) + :- Exchange(distribution=[hash[a]]) + : +- Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd') AS day]) + : +- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b]) + +- Exchange(distribution=[hash[d]]) + +- Calc(select=[b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP(), 'yyMMdd')) AS day, c, d], where=[>(b, 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) + +advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). +advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. + +related rel plan: +Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) ++- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b], changelogMode=[I,UB,UA,D]) + + ]]> + + =(price, 0)]) - : +- Exchange(distribution=[hash[payment_id, user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) + : +- ChangelogNormalize(key=[payment_id, user_id], condition=[>=(price, 0)]) + : +- Exchange(distribution=[hash[payment_id, user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[user_id], condition=[IS NOT NULL(location)]) +- Exchange(distribution=[hash[user_id]]) @@ -293,81 +611,53 @@ Sink(table=[default_catalog.default_database.sink_four_way], fields=[user_id, or - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + - - - - - - - - - - - - - - - - - + + + + - - - + + + - + + + + + + + + + + + + + + + - - - - - - - - - - - - - - - - - - + - - - - - - - + - - - - - - - - - - - 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - - - 10]]> - - - + + ($7, 10))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +== Optimized Physical Plan == +Calc(select=[user_id, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) ++- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, price, user_id], where=[>(price, 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) + +== Optimized Execution Plan == +Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) ++- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, price, user_id], where=[(price > 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> + + (price, 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) ++- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id], condition=[=(name, 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, price, user_id], where=[>(price, 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - - - - - - - - + + - - - - - + + + + - - - ($7, 10))]) - +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - -== Optimized Physical Plan == -Calc(select=[user_id_0, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) -+- MultiJoin(joinFilter=[=($0, $4)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $2), =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id_0,order_id,user_id_1,payment_id,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- Calc(select=[user_id_0]) - : +- ChangelogNormalize(key=[user_id_0], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - +- Exchange(distribution=[hash[user_id_2]]) - +- Calc(select=[payment_id, user_id_2], where=[>(price, 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) - -== Optimized Execution Plan == -Calc(select=[user_id_0, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) -+- MultiJoin(joinFilter=[=($0, $4)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $2), =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id_0,order_id,user_id_1,payment_id,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- Calc(select=[user_id_0]) - : +- ChangelogNormalize(key=[user_id_0], condition=[(name = 'Gus')]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - +- Exchange(distribution=[hash[user_id_2]]) - +- Calc(select=[payment_id, user_id_2], where=[(price > 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) + + + - - - 10]]> - - + + ($7, 10))]) - +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +Calc(select=[product_id, -(price, discount) AS net_price, quantity, promo_text]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($6, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:1;]}], select=[product_id,price,discount,sale_id,product_key,quantity,$f6,promo_id,product_key0,promo_text], rowType=[RecordType(VARCHAR(2147483647) product_id, DOUBLE price, DOUBLE discount, VARCHAR(2147483647) sale_id, DOUBLE product_key, INTEGER quantity, DOUBLE $f6, VARCHAR(2147483647) promo_id, DOUBLE product_key0, VARCHAR(2147483647) promo_text)]) + :- Exchange(distribution=[hash[$f6]]) + : +- Calc(select=[product_id, price, discount, sale_id, product_key, quantity, -(price, discount) AS $f6]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($3, $5)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;], 1=[LeftInputId:0;LeftFieldIndex:3;RightInputId:1;RightFieldIndex:1;]}], select=[product_id,price,discount,$f3,sale_id,product_key,quantity], rowType=[RecordType(VARCHAR(2147483647) product_id, DOUBLE price, DOUBLE discount, DOUBLE $f3, VARCHAR(2147483647) sale_id, DOUBLE product_key, INTEGER quantity)]) + : :- Exchange(distribution=[hash[$f3]]) + : : +- Calc(select=[product_id, price, discount, -(price, discount) AS $f3]) + : : +- ChangelogNormalize(key=[product_id], condition=[>(-(price, discount), 100)]) + : : +- Exchange(distribution=[hash[product_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Products, filter=[]]], fields=[product_id, price, discount]) + : +- Exchange(distribution=[hash[product_key]]) + : +- ChangelogNormalize(key=[sale_id]) + : +- Exchange(distribution=[hash[sale_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) + +- Exchange(distribution=[hash[product_key]]) + +- ChangelogNormalize(key=[promo_id]) + +- Exchange(distribution=[hash[promo_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) ]]> - + + + 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:8;], 1=[LeftInputId:0;LeftFieldIndex:8;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id1]]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, OR(=($0, $8), =($1, $6))]], joinAttributeMap=[{}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[single]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($4, $0)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[single]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala index e55fa33070b56..fa5ef6a2752ca 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.runtime.stream.sql import org.apache.flink.table.api._ import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.api.config.ExecutionConfigOptions +import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.planner.expressions.utils.FuncWithOpen import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.runtime.utils._ @@ -623,6 +624,22 @@ class JoinITCase(miniBatch: MiniBatchMode, state: StateBackendMode, enableAsyncS assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) } + @TestTemplate + def testInnerMultiJoinWithEqualPk(): Unit = { + tEnv.getConfig.getConfiguration + .setString(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED.key(), "true") + val query1 = "SELECT SUM(a2) AS a2, a1 FROM A group by a1" + val query2 = "SELECT SUM(b2) AS b2, b1 FROM B group by b1" + val query = s"SELECT a1, b1 FROM ($query1) JOIN ($query2) ON a1 = b1" + + val sink = new TestingRetractSink + tEnv.sqlQuery(query).toRetractStream[Row].addSink(sink).setParallelism(1) + env.execute() + + val expected = Seq("1,1", "2,2", "3,3") + assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) + } + @TestTemplate def testInnerJoinWithPk(): Unit = { val query1 = "SELECT SUM(a2) AS a2, a1 FROM A group by a1" diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMultiJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMultiJoinOperator.java index 3764b886d00d5..f92c43684d874 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMultiJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMultiJoinOperator.java @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.operators.join.stream; import org.apache.flink.api.common.functions.DefaultOpenContext; +import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; import org.apache.flink.streaming.api.operators.AbstractInput; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; import org.apache.flink.streaming.api.operators.Input; @@ -609,6 +610,10 @@ private Integer processRecords( } } + private boolean isHeapBackend() { + return getKeyedStateBackend() instanceof HeapKeyedStateBackend; + } + private static RowData newJoinedRowData(int depth, RowData joinedRowData, RowData record) { RowData newJoinedRowData; if (depth == 0) { @@ -808,6 +813,11 @@ private void initializeStateHandlers() { "Keyed state store not found when initializing keyed state store handlers."); } + boolean prohibitReuseRow = isHeapBackend(); + if (prohibitReuseRow) { + this.keyExtractor.requiresKeyDeepCopy(); + } + this.stateHandlers = new ArrayList<>(inputSpecs.size()); for (int i = 0; i < inputSpecs.size(); i++) { MultiJoinStateView stateView; diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java index d5e073c8b9b67..6de9e570adc6f 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java @@ -20,6 +20,7 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.utils.NoCommonJoinKeyException; @@ -64,13 +65,22 @@ public class AttributeBasedJoinKeyExtractor implements JoinKeyExtractor, Seriali // leftKeyExtractorsMap: extractors that read the left side (joined row so far) // using the same attribute order as in joinAttributeMap. private final Map> leftKeyExtractorsMap; + // RowData serializers for left key. + private final Map leftKeySerializersMap; // rightKeyExtractorsMap: extractors to extract the right-side key from each input. private final Map> rightKeyExtractorsMap; + // RowData serializers for right key. + private final Map rightKeySerializersMap; // Data structures for the "common join key" shared by all inputs. // Input 0 provides the canonical order and defines commonJoinKeyType. private final Map> commonJoinKeyExtractors; + // RowData serializers for common join key. + private final Map commonJoinKeySerializersMap; private RowType commonJoinKeyType; + // Controls whether key rows built are serialized and copied. This is required for the + // Heap State Backend to prevent object reuse issues, which can lead to data corruption + private boolean requiresKeyDeepCopy; /** * Creates an AttributeBasedJoinKeyExtractor. @@ -87,8 +97,12 @@ public AttributeBasedJoinKeyExtractor( this.joinAttributeMap = joinAttributeMap; this.inputTypes = inputTypes; this.leftKeyExtractorsMap = new HashMap<>(); + this.leftKeySerializersMap = new HashMap<>(); this.rightKeyExtractorsMap = new HashMap<>(); + this.rightKeySerializersMap = new HashMap<>(); this.commonJoinKeyExtractors = new HashMap<>(); + this.commonJoinKeySerializersMap = new HashMap<>(); + this.requiresKeyDeepCopy = false; initializeCaches(); initializeCommonJoinKeyStructures(); @@ -113,7 +127,7 @@ public RowData getJoinKey(final RowData row, final int inputId) { return null; } - return buildKeyRowFromSourceRow(row, keyExtractors); + return buildKeyRowFromSourceRow(row, keyExtractors, rightKeySerializersMap.get(inputId)); } @Override @@ -127,7 +141,8 @@ public RowData getLeftSideJoinKey(final int depth, final RowData joinedRowData) return null; } - return buildKeyRowFromJoinedRow(keyExtractors, joinedRowData); + return buildKeyRowFromJoinedRow( + keyExtractors, joinedRowData, leftKeySerializersMap.get(depth)); } @Override @@ -168,7 +183,7 @@ public RowType getCommonJoinKeyType() { return null; } - return buildKeyRowFromSourceRow(row, extractors); + return buildKeyRowFromSourceRow(row, extractors, commonJoinKeySerializersMap.get(inputId)); } @Override @@ -181,13 +196,23 @@ public int[] getCommonJoinKeyIndices(final int inputId) { return extractors.stream().mapToInt(KeyExtractor::getFieldIndexInSourceRow).toArray(); } + @Override + public void requiresKeyDeepCopy() { + this.requiresKeyDeepCopy = true; + } + // ==================== Initialization Methods ==================== private void initializeCaches() { if (this.inputTypes != null) { for (int i = 0; i < this.inputTypes.size(); i++) { - this.leftKeyExtractorsMap.put(i, createLeftJoinKeyFieldExtractors(i)); - this.rightKeyExtractorsMap.put(i, createRightJoinKeyExtractors(i)); + List leftJoinKeyExtractors = createLeftJoinKeyFieldExtractors(i); + this.leftKeyExtractorsMap.put(i, leftJoinKeyExtractors); + this.leftKeySerializersMap.put(i, createJoinKeySerializer(leftJoinKeyExtractors)); + + List rightJoinKeyExtractors = createRightJoinKeyExtractors(i); + this.rightKeyExtractorsMap.put(i, rightJoinKeyExtractors); + this.rightKeySerializersMap.put(i, createJoinKeySerializer(rightJoinKeyExtractors)); } } } @@ -226,6 +251,11 @@ private List createRightJoinKeyExtractors(final int inputId) { return keyExtractors; } + private RowDataSerializer createJoinKeySerializer(List keyExtractors) { + return new RowDataSerializer( + keyExtractors.stream().map(e -> e.fieldType).toArray(LogicalType[]::new)); + } + private static AttributeRef getLeftAttributeRef( final int inputId, final ConditionAttributeRef entry) { final AttributeRef leftAttrRef = new AttributeRef(entry.leftInputId, entry.leftFieldIndex); @@ -279,7 +309,9 @@ private KeyExtractor createKeyExtractor(final AttributeRef attrRef) { // ==================== Key Building Methods ==================== private RowData buildKeyRowFromJoinedRow( - final List keyExtractors, final RowData joinedRowData) { + final List keyExtractors, + final RowData joinedRowData, + RowDataSerializer keySerializer) { if (keyExtractors.isEmpty()) { return null; } @@ -288,11 +320,17 @@ private RowData buildKeyRowFromJoinedRow( for (int i = 0; i < keyExtractors.size(); i++) { keyRow.setField(i, keyExtractors.get(i).getLeftSideKey(joinedRowData)); } - return keyRow; + if (requiresKeyDeepCopy) { + return keySerializer.toBinaryRow(keyRow, true); + } else { + return keyRow; + } } - private GenericRowData buildKeyRowFromSourceRow( - final RowData sourceRow, final List keyExtractors) { + private RowData buildKeyRowFromSourceRow( + final RowData sourceRow, + final List keyExtractors, + RowDataSerializer keySerializer) { if (keyExtractors.isEmpty()) { return null; } @@ -301,7 +339,11 @@ private GenericRowData buildKeyRowFromSourceRow( for (int i = 0; i < keyExtractors.size(); i++) { keyRow.setField(i, keyExtractors.get(i).getRightSideKey(sourceRow)); } - return keyRow; + if (requiresKeyDeepCopy) { + return keySerializer.toBinaryRow(keyRow, true); + } else { + return keyRow; + } } private RowType buildJoinKeyType(final int inputId, final List keyExtractors) { @@ -608,6 +650,7 @@ private void processInputCommonAttributes( final LogicalType[] keyFieldTypes = extractors.stream().map(e -> e.fieldType).toArray(LogicalType[]::new); + this.commonJoinKeySerializersMap.put(currentInputId, new RowDataSerializer(keyFieldTypes)); if (currentInputId == 0 && !extractors.isEmpty()) { this.commonJoinKeyType = RowType.of(keyFieldTypes, keyFieldNames); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/JoinKeyExtractor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/JoinKeyExtractor.java index 7e6f91d6ff728..4fb96d28ed182 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/JoinKeyExtractor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/JoinKeyExtractor.java @@ -109,4 +109,7 @@ public interface JoinKeyExtractor extends Serializable { * common join key. */ int[] getCommonJoinKeyIndices(int inputId); + + /** Enables copying of row data. */ + void requiresKeyDeepCopy(); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/RowDataSerializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/RowDataSerializer.java index 3106d9e65d5c3..a83ea7a5e5546 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/RowDataSerializer.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/RowDataSerializer.java @@ -188,10 +188,14 @@ public int getArity() { /** Convert {@link RowData} into {@link BinaryRowData}. TODO modify it to code gen. */ @Override public BinaryRowData toBinaryRow(RowData row) { + return toBinaryRow(row, false); + } + + public BinaryRowData toBinaryRow(RowData row, boolean requiresDeepCopy) { if (row instanceof BinaryRowData) { return (BinaryRowData) row; } - if (reuseRow == null) { + if (reuseRow == null || requiresDeepCopy) { reuseRow = new BinaryRowData(types.length); reuseWriter = new BinaryRowWriter(reuseRow); } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/multijoin/StreamingMultiJoinOperatorTestBase.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/multijoin/StreamingMultiJoinOperatorTestBase.java index b09c7f13a9554..27861fafb11c2 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/multijoin/StreamingMultiJoinOperatorTestBase.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/multijoin/StreamingMultiJoinOperatorTestBase.java @@ -362,6 +362,7 @@ private static class SerializableKeySelector public SerializableKeySelector(JoinKeyExtractor keyExtractor, int inputIndex) { this.keyExtractor = keyExtractor; + this.keyExtractor.requiresKeyDeepCopy(); this.inputIndex = inputIndex; }