Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,12 @@
import io.trino.sql.planner.plan.PlanNode;
import io.trino.sql.planner.plan.PlanVisitor;
import io.trino.sql.planner.plan.ProjectNode;
import io.trino.sql.planner.plan.UnionNode;

import java.util.List;
import java.util.Optional;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.SystemSessionProperties.isOptimizeDuplicateInsensitiveJoins;
import static io.trino.sql.planner.DeterminismEvaluator.isDeterministic;
import static io.trino.sql.planner.plan.Patterns.aggregation;
Expand Down Expand Up @@ -121,6 +124,20 @@ public Optional<PlanNode> visitProject(ProjectNode node, Void context)
.map(source -> node.replaceChildren(ImmutableList.of(source)));
}

@Override
public Optional<PlanNode> visitUnion(UnionNode node, Void context)
{
List<PlanNode> rewrittenSources = node.getSources().stream()
.map(source -> source.accept(this, null).orElse(source))
.collect(toImmutableList());

if (rewrittenSources.equals(node.getSources())) {
return Optional.empty();
}

return Optional.of(node.replaceChildren(rewrittenSources));
}

@Override
public Optional<PlanNode> visitJoin(JoinNode node, Void context)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package io.trino.sql.planner.iterative.rule;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableListMultimap;
import com.google.common.collect.ImmutableMap;
import io.trino.sql.planner.Symbol;
import io.trino.sql.planner.iterative.rule.test.BaseRuleTest;
Expand All @@ -29,6 +30,7 @@
import static io.trino.sql.planner.assertions.PlanMatchPattern.filter;
import static io.trino.sql.planner.assertions.PlanMatchPattern.join;
import static io.trino.sql.planner.assertions.PlanMatchPattern.project;
import static io.trino.sql.planner.assertions.PlanMatchPattern.union;
import static io.trino.sql.planner.assertions.PlanMatchPattern.values;
import static io.trino.sql.planner.iterative.rule.test.PlanBuilder.expression;
import static io.trino.sql.planner.plan.Assignments.identity;
Expand Down Expand Up @@ -201,4 +203,39 @@ public void testNondeterministicProjection()
})
.doesNotFire();
}

@Test
public void testUnion()
{
tester().assertThat(new OptimizeDuplicateInsensitiveJoins(tester().getMetadata()))
.on(p -> {
Symbol symbolA = p.symbol("a");
Symbol symbolB = p.symbol("b");
Symbol symbolC = p.symbol("c");
Symbol symbolD = p.symbol("d");
Symbol symbolE = p.symbol("e");
return p.aggregation(a -> a
.singleGroupingSet(symbolE)
.source(p.union(
ImmutableListMultimap.<Symbol, Symbol>builder()
.put(symbolE, symbolA)
.put(symbolE, symbolC)
.build(),
ImmutableList.of(
p.join(
INNER,
p.values(symbolA),
p.values(symbolB)),
p.join(
INNER,
p.values(symbolC),
p.values(symbolD))))));
})
.matches(
aggregation(ImmutableMap.of(), union(
join(INNER, ImmutableList.of(), values("A"), values("B"))
.with(JoinNode.class, JoinNode::isMaySkipOutputDuplicates),
join(INNER, ImmutableList.of(), values("C"), values("D"))
.with(JoinNode.class, JoinNode::isMaySkipOutputDuplicates))));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -6,13 +6,13 @@ local exchange (GATHER, SINGLE, [])
local exchange (REPARTITION, HASH, ["d_year", "i_brand_id", "i_category_id", "i_class_id", "i_manufact_id"])
remote exchange (REPARTITION, HASH, ["i_brand_id_7", "i_category_id_9", "i_class_id_8", "i_manufact_id_10"])
partial aggregation over (d_year_14, expr, expr_22, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10)
join (RIGHT, PARTITIONED):
join (RIGHT, PARTITIONED, can skip output duplicates):
remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"])
scan catalog_returns
local exchange (GATHER, SINGLE, [])
remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"])
join (INNER, REPLICATED):
join (INNER, REPLICATED):
join (INNER, REPLICATED, can skip output duplicates):
join (INNER, REPLICATED, can skip output duplicates):
scan catalog_sales
local exchange (GATHER, SINGLE, [])
remote exchange (REPLICATE, BROADCAST, [])
Expand All @@ -22,13 +22,13 @@ local exchange (GATHER, SINGLE, [])
scan item
remote exchange (REPARTITION, HASH, ["i_brand_id_34", "i_category_id_38", "i_class_id_36", "i_manufact_id_40"])
partial aggregation over (d_year_58, expr_87, expr_88, i_brand_id_34, i_category_id_38, i_class_id_36, i_manufact_id_40)
join (RIGHT, PARTITIONED):
join (RIGHT, PARTITIONED, can skip output duplicates):
remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"])
scan store_returns
local exchange (GATHER, SINGLE, [])
remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"])
join (INNER, REPLICATED):
join (INNER, REPLICATED):
join (INNER, REPLICATED, can skip output duplicates):
join (INNER, REPLICATED, can skip output duplicates):
scan store_sales
local exchange (GATHER, SINGLE, [])
remote exchange (REPLICATE, BROADCAST, [])
Expand All @@ -38,13 +38,13 @@ local exchange (GATHER, SINGLE, [])
scan item
remote exchange (REPARTITION, HASH, ["i_brand_id_100", "i_category_id_104", "i_class_id_102", "i_manufact_id_106"])
partial aggregation over (d_year_124, expr_153, expr_154, i_brand_id_100, i_category_id_104, i_class_id_102, i_manufact_id_106)
join (RIGHT, PARTITIONED):
join (RIGHT, PARTITIONED, can skip output duplicates):
remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"])
scan web_returns
local exchange (GATHER, SINGLE, [])
remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"])
join (INNER, REPLICATED):
join (INNER, REPLICATED):
join (INNER, REPLICATED, can skip output duplicates):
join (INNER, REPLICATED, can skip output duplicates):
scan web_sales
local exchange (GATHER, SINGLE, [])
remote exchange (REPLICATE, BROADCAST, [])
Expand All @@ -57,13 +57,13 @@ local exchange (GATHER, SINGLE, [])
local exchange (GATHER, SINGLE, [])
remote exchange (REPARTITION, HASH, ["i_brand_id_216", "i_category_id_220", "i_class_id_218", "i_manufact_id_222"])
partial aggregation over (d_year_240, expr_296, expr_297, i_brand_id_216, i_category_id_220, i_class_id_218, i_manufact_id_222)
join (RIGHT, PARTITIONED):
join (RIGHT, PARTITIONED, can skip output duplicates):
remote exchange (REPARTITION, HASH, ["cr_item_sk_266", "cr_order_number_280"])
scan catalog_returns
local exchange (GATHER, SINGLE, [])
remote exchange (REPARTITION, HASH, ["cs_item_sk_185", "cs_order_number_187"])
join (INNER, REPLICATED):
join (INNER, REPLICATED):
join (INNER, REPLICATED, can skip output duplicates):
join (INNER, REPLICATED, can skip output duplicates):
scan catalog_sales
local exchange (GATHER, SINGLE, [])
remote exchange (REPLICATE, BROADCAST, [])
Expand All @@ -73,13 +73,13 @@ local exchange (GATHER, SINGLE, [])
scan item
remote exchange (REPARTITION, HASH, ["i_brand_id_332", "i_category_id_336", "i_class_id_334", "i_manufact_id_338"])
partial aggregation over (d_year_356, expr_405, expr_406, i_brand_id_332, i_category_id_336, i_class_id_334, i_manufact_id_338)
join (RIGHT, PARTITIONED):
join (RIGHT, PARTITIONED, can skip output duplicates):
remote exchange (REPARTITION, HASH, ["sr_item_sk_382", "sr_ticket_number_389"])
scan store_returns
local exchange (GATHER, SINGLE, [])
remote exchange (REPARTITION, HASH, ["ss_item_sk_299", "ss_ticket_number_306"])
join (INNER, REPLICATED):
join (INNER, REPLICATED):
join (INNER, REPLICATED, can skip output duplicates):
join (INNER, REPLICATED, can skip output duplicates):
scan store_sales
local exchange (GATHER, SINGLE, [])
remote exchange (REPLICATE, BROADCAST, [])
Expand All @@ -89,13 +89,13 @@ local exchange (GATHER, SINGLE, [])
scan item
remote exchange (REPARTITION, HASH, ["i_brand_id_452", "i_category_id_456", "i_class_id_454", "i_manufact_id_458"])
partial aggregation over (d_year_476, expr_529, expr_530, i_brand_id_452, i_category_id_456, i_class_id_454, i_manufact_id_458)
join (RIGHT, PARTITIONED):
join (RIGHT, PARTITIONED, can skip output duplicates):
remote exchange (REPARTITION, HASH, ["wr_item_sk_502", "wr_order_number_513"])
scan web_returns
local exchange (GATHER, SINGLE, [])
remote exchange (REPARTITION, HASH, ["ws_item_sk_409", "ws_order_number_423"])
join (INNER, REPLICATED):
join (INNER, REPLICATED):
join (INNER, REPLICATED, can skip output duplicates):
join (INNER, REPLICATED, can skip output duplicates):
scan web_sales
local exchange (GATHER, SINGLE, [])
remote exchange (REPLICATE, BROADCAST, [])
Expand Down
Loading