-
Notifications
You must be signed in to change notification settings - Fork 13k
/
HashJoinBuildSecondProperties.java
100 lines (87 loc) · 3.61 KB
/
HashJoinBuildSecondProperties.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.optimizer.operators;
import org.apache.flink.api.common.operators.util.FieldList;
import org.apache.flink.optimizer.CompilerException;
import org.apache.flink.optimizer.dag.TwoInputNode;
import org.apache.flink.optimizer.dataproperties.LocalProperties;
import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties;
import org.apache.flink.optimizer.plan.Channel;
import org.apache.flink.optimizer.plan.DualInputPlanNode;
import org.apache.flink.runtime.operators.DriverStrategy;
import java.util.Collections;
import java.util.List;
public final class HashJoinBuildSecondProperties extends AbstractJoinDescriptor {
public HashJoinBuildSecondProperties(FieldList keys1, FieldList keys2) {
super(keys1, keys2);
}
public HashJoinBuildSecondProperties(
FieldList keys1,
FieldList keys2,
boolean broadcastFirstAllowed,
boolean broadcastSecondAllowed,
boolean repartitionAllowed) {
super(keys1, keys2, broadcastFirstAllowed, broadcastSecondAllowed, repartitionAllowed);
}
@Override
public DriverStrategy getStrategy() {
return DriverStrategy.HYBRIDHASH_BUILD_SECOND;
}
@Override
protected List<LocalPropertiesPair> createPossibleLocalProperties() {
// all properties are possible
return Collections.singletonList(
new LocalPropertiesPair(
new RequestedLocalProperties(), new RequestedLocalProperties()));
}
@Override
public boolean areCoFulfilled(
RequestedLocalProperties requested1,
RequestedLocalProperties requested2,
LocalProperties produced1,
LocalProperties produced2) {
return true;
}
@Override
public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) {
DriverStrategy strategy;
if (!in2.isOnDynamicPath() && in1.isOnDynamicPath()) {
// sanity check that the first input is cached and remove that cache
if (!in2.getTempMode().isCached()) {
throw new CompilerException(
"No cache at point where static and dynamic parts meet.");
}
in2.setTempMode(in2.getTempMode().makeNonCached());
strategy = DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED;
} else {
strategy = DriverStrategy.HYBRIDHASH_BUILD_SECOND;
}
return new DualInputPlanNode(
node,
"Join (" + node.getOperator().getName() + ")",
in1,
in2,
strategy,
this.keys1,
this.keys2);
}
@Override
public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) {
return new LocalProperties();
}
}