Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 3e28732

Browse files
skrzypo987sopel39
skrzypo987
authored andcommitted
Simplify creation of join operator
Since the split into spilling and non-spilling, creation of join operators had some unnecessary casting. This commit cleans it and makes clear spilling/non-spilling code paths in LocalExecutionPlanner. Changes in this class are strictly mechanical. No logical changes are made.
1 parent ebecf59 commit 3e28732

14 files changed

+268
-458
lines changed

core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java

+73-38
Original file line numberDiff line numberDiff line change
@@ -14,40 +14,45 @@
1414
package io.trino.operator;
1515

1616
import io.trino.operator.join.JoinBridgeManager;
17+
import io.trino.operator.join.LookupJoinOperatorFactory.JoinType;
18+
import io.trino.operator.join.LookupSourceFactory;
19+
import io.trino.operator.join.unspilled.PartitionedLookupSourceFactory;
1720
import io.trino.spi.type.Type;
1821
import io.trino.spiller.PartitioningSpillerFactory;
22+
import io.trino.sql.planner.plan.JoinNode;
1923
import io.trino.sql.planner.plan.PlanNodeId;
2024
import io.trino.type.BlockTypeOperators;
2125

2226
import java.util.List;
2327
import java.util.Optional;
2428
import java.util.OptionalInt;
2529

30+
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.FULL_OUTER;
31+
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.INNER;
32+
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.LOOKUP_OUTER;
33+
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.PROBE_OUTER;
34+
import static java.util.Objects.requireNonNull;
35+
2636
public interface OperatorFactories
2737
{
28-
OperatorFactory innerJoin(
38+
public OperatorFactory join(
39+
JoinOperatorType joinType,
2940
int operatorId,
3041
PlanNodeId planNodeId,
31-
JoinBridgeManager<?> lookupSourceFactory,
32-
boolean outputSingleMatch,
33-
boolean waitForBuild,
42+
JoinBridgeManager<? extends PartitionedLookupSourceFactory> lookupSourceFactory,
3443
boolean hasFilter,
35-
boolean spillingEnabled,
3644
List<Type> probeTypes,
3745
List<Integer> probeJoinChannel,
3846
OptionalInt probeHashChannel,
3947
Optional<List<Integer>> probeOutputChannels,
40-
OptionalInt totalOperatorsCount,
41-
PartitioningSpillerFactory partitioningSpillerFactory,
4248
BlockTypeOperators blockTypeOperators);
4349

44-
OperatorFactory probeOuterJoin(
50+
public OperatorFactory spillingJoin(
51+
JoinOperatorType joinType,
4552
int operatorId,
4653
PlanNodeId planNodeId,
47-
JoinBridgeManager<?> lookupSourceFactory,
48-
boolean outputSingleMatch,
54+
JoinBridgeManager<? extends LookupSourceFactory> lookupSourceFactory,
4955
boolean hasFilter,
50-
boolean spillingEnabled,
5156
List<Type> probeTypes,
5257
List<Integer> probeJoinChannel,
5358
OptionalInt probeHashChannel,
@@ -56,32 +61,62 @@ OperatorFactory probeOuterJoin(
5661
PartitioningSpillerFactory partitioningSpillerFactory,
5762
BlockTypeOperators blockTypeOperators);
5863

59-
OperatorFactory lookupOuterJoin(
60-
int operatorId,
61-
PlanNodeId planNodeId,
62-
JoinBridgeManager<?> lookupSourceFactory,
63-
boolean waitForBuild,
64-
boolean hasFilter,
65-
boolean spillingEnabled,
66-
List<Type> probeTypes,
67-
List<Integer> probeJoinChannel,
68-
OptionalInt probeHashChannel,
69-
Optional<List<Integer>> probeOutputChannels,
70-
OptionalInt totalOperatorsCount,
71-
PartitioningSpillerFactory partitioningSpillerFactory,
72-
BlockTypeOperators blockTypeOperators);
64+
public static class JoinOperatorType
65+
{
66+
private final JoinType type;
67+
private final boolean outputSingleMatch;
68+
private final boolean waitForBuild;
7369

74-
OperatorFactory fullOuterJoin(
75-
int operatorId,
76-
PlanNodeId planNodeId,
77-
JoinBridgeManager<?> lookupSourceFactory,
78-
boolean hasFilter,
79-
boolean spillingEnabled,
80-
List<Type> probeTypes,
81-
List<Integer> probeJoinChannel,
82-
OptionalInt probeHashChannel,
83-
Optional<List<Integer>> probeOutputChannels,
84-
OptionalInt totalOperatorsCount,
85-
PartitioningSpillerFactory partitioningSpillerFactory,
86-
BlockTypeOperators blockTypeOperators);
70+
public static JoinOperatorType ofJoinNodeType(JoinNode.Type joinNodeType, boolean outputSingleMatch, boolean waitForBuild)
71+
{
72+
return switch (joinNodeType) {
73+
case INNER -> innerJoin(outputSingleMatch, waitForBuild);
74+
case LEFT -> probeOuterJoin(outputSingleMatch);
75+
case RIGHT -> lookupOuterJoin(waitForBuild);
76+
case FULL -> fullOuterJoin();
77+
};
78+
}
79+
80+
public static JoinOperatorType innerJoin(boolean outputSingleMatch, boolean waitForBuild)
81+
{
82+
return new JoinOperatorType(INNER, outputSingleMatch, waitForBuild);
83+
}
84+
85+
public static JoinOperatorType probeOuterJoin(boolean outputSingleMatch)
86+
{
87+
return new JoinOperatorType(PROBE_OUTER, outputSingleMatch, false);
88+
}
89+
90+
public static JoinOperatorType lookupOuterJoin(boolean waitForBuild)
91+
{
92+
return new JoinOperatorType(LOOKUP_OUTER, false, waitForBuild);
93+
}
94+
95+
public static JoinOperatorType fullOuterJoin()
96+
{
97+
return new JoinOperatorType(FULL_OUTER, false, false);
98+
}
99+
100+
private JoinOperatorType(JoinType type, boolean outputSingleMatch, boolean waitForBuild)
101+
{
102+
this.type = requireNonNull(type, "type is null");
103+
this.outputSingleMatch = outputSingleMatch;
104+
this.waitForBuild = waitForBuild;
105+
}
106+
107+
public boolean isOutputSingleMatch()
108+
{
109+
return outputSingleMatch;
110+
}
111+
112+
public boolean isWaitForBuild()
113+
{
114+
return waitForBuild;
115+
}
116+
117+
public JoinType getType()
118+
{
119+
return type;
120+
}
121+
}
87122
}

core/trino-main/src/main/java/io/trino/operator/TrinoOperatorFactories.java

+32-155
Original file line numberDiff line numberDiff line change
@@ -16,8 +16,8 @@
1616
import io.trino.operator.join.JoinBridgeManager;
1717
import io.trino.operator.join.JoinProbe.JoinProbeFactory;
1818
import io.trino.operator.join.LookupJoinOperatorFactory;
19-
import io.trino.operator.join.LookupJoinOperatorFactory.JoinType;
2019
import io.trino.operator.join.LookupSourceFactory;
20+
import io.trino.operator.join.unspilled.PartitionedLookupSourceFactory;
2121
import io.trino.spi.type.Type;
2222
import io.trino.spiller.PartitioningSpillerFactory;
2323
import io.trino.sql.planner.plan.PlanNodeId;
@@ -29,144 +29,76 @@
2929
import java.util.stream.IntStream;
3030

3131
import static com.google.common.collect.ImmutableList.toImmutableList;
32-
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.FULL_OUTER;
33-
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.INNER;
34-
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.LOOKUP_OUTER;
35-
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.PROBE_OUTER;
3632

3733
public class TrinoOperatorFactories
3834
implements OperatorFactories
3935
{
4036
@Override
41-
public OperatorFactory innerJoin(
37+
public OperatorFactory join(
38+
JoinOperatorType joinType,
4239
int operatorId,
4340
PlanNodeId planNodeId,
44-
JoinBridgeManager<?> lookupSourceFactory,
45-
boolean outputSingleMatch,
46-
boolean waitForBuild,
41+
JoinBridgeManager<? extends PartitionedLookupSourceFactory> lookupSourceFactory,
4742
boolean hasFilter,
48-
boolean spillingEnabled,
4943
List<Type> probeTypes,
5044
List<Integer> probeJoinChannel,
5145
OptionalInt probeHashChannel,
52-
Optional<List<Integer>> probeOutputChannels,
53-
OptionalInt totalOperatorsCount,
54-
PartitioningSpillerFactory partitioningSpillerFactory,
46+
Optional<List<Integer>> probeOutputChannelsOptional,
5547
BlockTypeOperators blockTypeOperators)
5648
{
57-
return createJoinOperatorFactory(
58-
operatorId,
59-
planNodeId,
60-
lookupSourceFactory,
61-
probeTypes,
62-
probeJoinChannel,
63-
probeHashChannel,
64-
probeOutputChannels.orElse(rangeList(probeTypes.size())),
65-
INNER,
66-
outputSingleMatch,
67-
waitForBuild,
68-
spillingEnabled,
69-
totalOperatorsCount,
70-
partitioningSpillerFactory,
71-
blockTypeOperators);
72-
}
49+
List<Integer> probeOutputChannels = probeOutputChannelsOptional.orElse(rangeList(probeTypes.size()));
50+
List<Type> probeOutputChannelTypes = probeOutputChannels.stream()
51+
.map(probeTypes::get)
52+
.collect(toImmutableList());
7353

74-
@Override
75-
public OperatorFactory probeOuterJoin(
76-
int operatorId,
77-
PlanNodeId planNodeId,
78-
JoinBridgeManager<?> lookupSourceFactory,
79-
boolean outputSingleMatch,
80-
boolean hasFilter,
81-
boolean spillingEnabled,
82-
List<Type> probeTypes,
83-
List<Integer> probeJoinChannel,
84-
OptionalInt probeHashChannel,
85-
Optional<List<Integer>> probeOutputChannels,
86-
OptionalInt totalOperatorsCount,
87-
PartitioningSpillerFactory partitioningSpillerFactory,
88-
BlockTypeOperators blockTypeOperators)
89-
{
90-
return createJoinOperatorFactory(
54+
return new io.trino.operator.join.unspilled.LookupJoinOperatorFactory(
9155
operatorId,
9256
planNodeId,
9357
lookupSourceFactory,
9458
probeTypes,
59+
probeOutputChannelTypes,
60+
lookupSourceFactory.getBuildOutputTypes(),
61+
joinType,
62+
new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel),
63+
blockTypeOperators,
9564
probeJoinChannel,
96-
probeHashChannel,
97-
probeOutputChannels.orElse(rangeList(probeTypes.size())),
98-
PROBE_OUTER,
99-
outputSingleMatch,
100-
false,
101-
spillingEnabled,
102-
totalOperatorsCount,
103-
partitioningSpillerFactory,
104-
blockTypeOperators);
65+
probeHashChannel);
10566
}
10667

10768
@Override
108-
public OperatorFactory lookupOuterJoin(
69+
public OperatorFactory spillingJoin(
70+
JoinOperatorType joinType,
10971
int operatorId,
11072
PlanNodeId planNodeId,
111-
JoinBridgeManager<?> lookupSourceFactory,
112-
boolean waitForBuild,
73+
JoinBridgeManager<? extends LookupSourceFactory> lookupSourceFactory,
11374
boolean hasFilter,
114-
boolean spillingEnabled,
11575
List<Type> probeTypes,
11676
List<Integer> probeJoinChannel,
11777
OptionalInt probeHashChannel,
118-
Optional<List<Integer>> probeOutputChannels,
78+
Optional<List<Integer>> probeOutputChannelsOptional,
11979
OptionalInt totalOperatorsCount,
12080
PartitioningSpillerFactory partitioningSpillerFactory,
12181
BlockTypeOperators blockTypeOperators)
12282
{
123-
return createJoinOperatorFactory(
124-
operatorId,
125-
planNodeId,
126-
lookupSourceFactory,
127-
probeTypes,
128-
probeJoinChannel,
129-
probeHashChannel,
130-
probeOutputChannels.orElse(rangeList(probeTypes.size())),
131-
LOOKUP_OUTER,
132-
false,
133-
waitForBuild,
134-
spillingEnabled,
135-
totalOperatorsCount,
136-
partitioningSpillerFactory,
137-
blockTypeOperators);
138-
}
83+
List<Integer> probeOutputChannels = probeOutputChannelsOptional.orElse(rangeList(probeTypes.size()));
84+
List<Type> probeOutputChannelTypes = probeOutputChannels.stream()
85+
.map(probeTypes::get)
86+
.collect(toImmutableList());
13987

140-
@Override
141-
public OperatorFactory fullOuterJoin(
142-
int operatorId,
143-
PlanNodeId planNodeId,
144-
JoinBridgeManager<?> lookupSourceFactory,
145-
boolean hasFilter,
146-
boolean spillingEnabled,
147-
List<Type> probeTypes,
148-
List<Integer> probeJoinChannel,
149-
OptionalInt probeHashChannel,
150-
Optional<List<Integer>> probeOutputChannels,
151-
OptionalInt totalOperatorsCount,
152-
PartitioningSpillerFactory partitioningSpillerFactory,
153-
BlockTypeOperators blockTypeOperators)
154-
{
155-
return createJoinOperatorFactory(
88+
return new LookupJoinOperatorFactory(
15689
operatorId,
15790
planNodeId,
15891
lookupSourceFactory,
15992
probeTypes,
93+
probeOutputChannelTypes,
94+
lookupSourceFactory.getBuildOutputTypes(),
95+
joinType,
96+
new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel),
97+
blockTypeOperators,
98+
totalOperatorsCount,
16099
probeJoinChannel,
161100
probeHashChannel,
162-
probeOutputChannels.orElse(rangeList(probeTypes.size())),
163-
FULL_OUTER,
164-
false,
165-
false,
166-
spillingEnabled,
167-
totalOperatorsCount,
168-
partitioningSpillerFactory,
169-
blockTypeOperators);
101+
partitioningSpillerFactory);
170102
}
171103

172104
private static List<Integer> rangeList(int endExclusive)
@@ -175,59 +107,4 @@ private static List<Integer> rangeList(int endExclusive)
175107
.boxed()
176108
.collect(toImmutableList());
177109
}
178-
179-
private OperatorFactory createJoinOperatorFactory(
180-
int operatorId,
181-
PlanNodeId planNodeId,
182-
JoinBridgeManager<?> lookupSourceFactoryManager,
183-
List<Type> probeTypes,
184-
List<Integer> probeJoinChannel,
185-
OptionalInt probeHashChannel,
186-
List<Integer> probeOutputChannels,
187-
JoinType joinType,
188-
boolean outputSingleMatch,
189-
boolean waitForBuild,
190-
boolean spillingEnabled,
191-
OptionalInt totalOperatorsCount,
192-
PartitioningSpillerFactory partitioningSpillerFactory,
193-
BlockTypeOperators blockTypeOperators)
194-
{
195-
List<Type> probeOutputChannelTypes = probeOutputChannels.stream()
196-
.map(probeTypes::get)
197-
.collect(toImmutableList());
198-
199-
if (spillingEnabled) {
200-
return new LookupJoinOperatorFactory(
201-
operatorId,
202-
planNodeId,
203-
(JoinBridgeManager<? extends LookupSourceFactory>) lookupSourceFactoryManager,
204-
probeTypes,
205-
probeOutputChannelTypes,
206-
lookupSourceFactoryManager.getBuildOutputTypes(),
207-
joinType,
208-
outputSingleMatch,
209-
waitForBuild,
210-
new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel),
211-
blockTypeOperators,
212-
totalOperatorsCount,
213-
probeJoinChannel,
214-
probeHashChannel,
215-
partitioningSpillerFactory);
216-
}
217-
218-
return new io.trino.operator.join.unspilled.LookupJoinOperatorFactory(
219-
operatorId,
220-
planNodeId,
221-
(JoinBridgeManager<? extends io.trino.operator.join.unspilled.PartitionedLookupSourceFactory>) lookupSourceFactoryManager,
222-
probeTypes,
223-
probeOutputChannelTypes,
224-
lookupSourceFactoryManager.getBuildOutputTypes(),
225-
joinType,
226-
outputSingleMatch,
227-
waitForBuild,
228-
new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel),
229-
blockTypeOperators,
230-
probeJoinChannel,
231-
probeHashChannel);
232-
}
233110
}

0 commit comments

Comments
 (0)