Skip to content

Commit f82bde7

Browse files
committed
[FLINK-35652][table] Shuffle input stream of lookup join based on LOOKUP hint
1 parent 82bb81b commit f82bde7

File tree

26 files changed

+2084
-418
lines changed

26 files changed

+2084
-418
lines changed

flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/LookupJoinHintOptions.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -100,6 +100,12 @@ public class LookupJoinHintOptions {
100100
.noDefaultValue()
101101
.withDescription("Max attempt number of the 'fixed-delay' retry strategy.");
102102

103+
public static final ConfigOption<Boolean> SHUFFLE =
104+
key("shuffle")
105+
.booleanType()
106+
.defaultValue(false)
107+
.withDescription("Enable shuffle before lookup join.");
108+
103109
public static final String LOOKUP_MISS_PREDICATE = "lookup_miss";
104110

105111
private static final Set<ConfigOption<?>> requiredKeys = new HashSet<>();
@@ -117,6 +123,7 @@ public class LookupJoinHintOptions {
117123
supportedKeys.add(RETRY_STRATEGY);
118124
supportedKeys.add(FIXED_DELAY);
119125
supportedKeys.add(MAX_ATTEMPTS);
126+
supportedKeys.add(SHUFFLE);
120127
}
121128

122129
public static ImmutableSet<ConfigOption> getRequiredOptions() {

flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,8 @@ public BatchExecLookupJoin(
7676
@Nullable LookupJoinUtil.AsyncLookupOptions asyncLookupOptions,
7777
InputProperty inputProperty,
7878
RowType outputType,
79-
String description) {
79+
String description,
80+
boolean preferCustomShuffle) {
8081
super(
8182
ExecNodeContext.newNodeId(),
8283
ExecNodeContext.newContext(BatchExecLookupJoin.class),
@@ -94,7 +95,8 @@ public BatchExecLookupJoin(
9495
ChangelogMode.insertOnly(),
9596
Collections.singletonList(inputProperty),
9697
outputType,
97-
description);
98+
description,
99+
preferCustomShuffle);
98100
}
99101

100102
@JsonCreator
@@ -117,7 +119,8 @@ public BatchExecLookupJoin(
117119
LookupJoinUtil.AsyncLookupOptions asyncLookupOptions,
118120
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
119121
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
120-
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
122+
@JsonProperty(FIELD_NAME_DESCRIPTION) String description,
123+
@JsonProperty(FIELD_NAME_PREFER_CUSTOM_SHUFFLE) boolean preferCustomShuffle) {
121124
super(
122125
id,
123126
context,
@@ -135,7 +138,8 @@ public BatchExecLookupJoin(
135138
ChangelogMode.insertOnly(),
136139
inputProperties,
137140
outputType,
138-
description);
141+
description,
142+
preferCustomShuffle);
139143
}
140144

141145
@Override

flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java

Lines changed: 25 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -157,6 +157,8 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase<RowData> {
157157

158158
public static final String FIELD_NAME_ASYNC_OPTIONS = "asyncOptions";
159159
public static final String FIELD_NAME_RETRY_OPTIONS = "retryOptions";
160+
public static final String FIELD_NAME_PREFER_CUSTOM_SHUFFLE = "preferCustomShuffle";
161+
public static final String CUSTOM_SHUFFLE_TRANSFORMATION = "custom-shuffle";
160162

161163
@JsonProperty(FIELD_NAME_JOIN_TYPE)
162164
private final FlinkJoinType joinType;
@@ -197,6 +199,9 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase<RowData> {
197199
@JsonInclude(JsonInclude.Include.NON_NULL)
198200
private final @Nullable LookupJoinUtil.RetryLookupOptions retryOptions;
199201

202+
@JsonProperty(FIELD_NAME_PREFER_CUSTOM_SHUFFLE)
203+
private final boolean preferCustomShuffle;
204+
200205
protected CommonExecLookupJoin(
201206
int id,
202207
ExecNodeContext context,
@@ -214,7 +219,8 @@ protected CommonExecLookupJoin(
214219
ChangelogMode inputChangelogMode,
215220
List<InputProperty> inputProperties,
216221
RowType outputType,
217-
String description) {
222+
String description,
223+
boolean preferCustomShuffle) {
218224
super(id, context, persistedConfig, inputProperties, outputType, description);
219225
checkArgument(inputProperties.size() == 1);
220226
this.joinType = checkNotNull(joinType);
@@ -227,6 +233,7 @@ protected CommonExecLookupJoin(
227233
this.inputChangelogMode = inputChangelogMode;
228234
this.asyncLookupOptions = asyncLookupOptions;
229235
this.retryOptions = retryOptions;
236+
this.preferCustomShuffle = preferCustomShuffle;
230237
}
231238

232239
public TemporalTableSourceSpec getTemporalTableSourceSpec() {
@@ -252,23 +259,36 @@ protected Transformation<RowData> createJoinTransformation(
252259
ResultRetryStrategy retryStrategy =
253260
retryOptions != null ? retryOptions.toRetryStrategy() : null;
254261

262+
boolean tryApplyCustomShuffle = preferCustomShuffle && !upsertMaterialize;
255263
UserDefinedFunction lookupFunction =
256264
LookupJoinUtil.getLookupFunction(
257265
temporalTable,
258266
lookupKeys.keySet(),
259267
planner.getFlinkContext().getClassLoader(),
260268
isAsyncEnabled,
261-
retryStrategy);
269+
retryStrategy,
270+
tryApplyCustomShuffle);
271+
Transformation<RowData> inputTransformation =
272+
(Transformation<RowData>) inputEdge.translateToPlan(planner);
273+
if (tryApplyCustomShuffle) {
274+
inputTransformation =
275+
LookupJoinUtil.tryApplyCustomShufflePartitioner(
276+
planner,
277+
temporalTable,
278+
inputRowType,
279+
lookupKeys,
280+
inputTransformation,
281+
inputChangelogMode,
282+
createTransformationMeta(CUSTOM_SHUFFLE_TRANSFORMATION, config));
283+
}
284+
262285
UserDefinedFunctionHelper.prepareInstance(config, lookupFunction);
263286

264287
boolean isLeftOuterJoin = joinType == FlinkJoinType.LEFT;
265288
if (isAsyncEnabled) {
266289
assert lookupFunction instanceof AsyncTableFunction;
267290
}
268291

269-
Transformation<RowData> inputTransformation =
270-
(Transformation<RowData>) inputEdge.translateToPlan(planner);
271-
272292
if (upsertMaterialize) {
273293
// upsertMaterialize only works on sync lookup mode, async lookup is unsupported.
274294
assert !isAsyncEnabled && !inputChangelogMode.containsOnly(RowKind.INSERT);

flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -120,7 +120,8 @@ public StreamExecLookupJoin(
120120
@Nullable int[] inputUpsertKey,
121121
InputProperty inputProperty,
122122
RowType outputType,
123-
String description) {
123+
String description,
124+
boolean preferCustomShuffle) {
124125
this(
125126
ExecNodeContext.newNodeId(),
126127
ExecNodeContext.newContext(StreamExecLookupJoin.class),
@@ -144,7 +145,8 @@ public StreamExecLookupJoin(
144145
: null,
145146
Collections.singletonList(inputProperty),
146147
outputType,
147-
description);
148+
description,
149+
preferCustomShuffle);
148150
}
149151

150152
@JsonCreator
@@ -176,7 +178,8 @@ public StreamExecLookupJoin(
176178
@JsonProperty(FIELD_NAME_STATE) @Nullable List<StateMetadata> stateMetadataList,
177179
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
178180
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
179-
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
181+
@JsonProperty(FIELD_NAME_DESCRIPTION) String description,
182+
@JsonProperty(FIELD_NAME_PREFER_CUSTOM_SHUFFLE) boolean preferCustomShuffle) {
180183
super(
181184
id,
182185
context,
@@ -193,7 +196,8 @@ public StreamExecLookupJoin(
193196
inputChangelogMode,
194197
inputProperties,
195198
outputType,
196-
description);
199+
description,
200+
preferCustomShuffle);
197201
this.lookupKeyContainsPrimaryKey = lookupKeyContainsPrimaryKey;
198202
this.upsertMaterialize = upsertMaterialize;
199203
this.inputUpsertKey = inputUpsertKey;

flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/ExecNodeUtil.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,9 @@
2828
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
2929
import org.apache.flink.streaming.api.transformations.LegacySourceTransformation;
3030
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
31+
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
3132
import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
33+
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
3234
import org.apache.flink.table.api.TableException;
3335
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
3436
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
@@ -390,4 +392,15 @@ public static void makeLegacySourceTransformationsBounded(Transformation<?> tran
390392
}
391393
transformation.getInputs().forEach(ExecNodeUtil::makeLegacySourceTransformationsBounded);
392394
}
395+
396+
/** Create a {@link PartitionTransformation}. */
397+
public static <I> Transformation<I> createPartitionTransformation(
398+
Transformation<I> input,
399+
TransformationMetadata transformationMeta,
400+
StreamPartitioner<I> streamPartitioner) {
401+
PartitionTransformation<I> transformation =
402+
new PartitionTransformation<>(input, streamPartitioner);
403+
transformationMeta.fill(transformation);
404+
return transformation;
405+
}
393406
}

flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/KeySelectorUtil.java

Lines changed: 59 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,9 @@
3535
import org.apache.flink.table.types.logical.LogicalType;
3636
import org.apache.flink.table.types.logical.RowType;
3737

38+
import java.util.Arrays;
39+
import java.util.Map;
40+
3841
/** Utility for KeySelector. */
3942
public class KeySelectorUtil {
4043

@@ -91,4 +94,60 @@ public static RowDataKeySelector getRowDataSelector(
9194
return EmptyRowDataKeySelector.INSTANCE;
9295
}
9396
}
97+
98+
/**
99+
* Create a {@link RowDataKeySelector} which select the columns of lookup keys from the row of
100+
* left table in lookup join.
101+
*
102+
* @param classLoader the user classloader
103+
* @param lookupKeysOfRightTable the lookup keys
104+
* @param leftTableRowType the row type of left table
105+
* @return the RowDataKeySelector
106+
*/
107+
public static RowDataKeySelector getLookupKeysSelectorFromLeftTable(
108+
ClassLoader classLoader,
109+
Map<Integer, LookupJoinUtil.LookupKey> lookupKeysOfRightTable,
110+
InternalTypeInfo<RowData> leftTableRowType) {
111+
LogicalType[] inputFieldTypes = leftTableRowType.toRowFieldTypes();
112+
int[] lookupKeyIndicesInOrder =
113+
LookupJoinUtil.getOrderedLookupKeys(lookupKeysOfRightTable.keySet());
114+
// 1. Generate the map from left table to lookup keys.
115+
int[] inputMapping = new int[lookupKeysOfRightTable.size()];
116+
Arrays.fill(inputMapping, ProjectionCodeGenerator.EMPTY_INPUT_MAPPING_VALUE());
117+
// 2. Generate all lookup keys in order.
118+
LookupJoinUtil.LookupKey[] orderedLookupKeys =
119+
new LookupJoinUtil.LookupKey[lookupKeyIndicesInOrder.length];
120+
// 3. Generate the logical types of all lookup keys.
121+
LogicalType[] orderedLookupKeyLogicalTypes = new LogicalType[lookupKeysOfRightTable.size()];
122+
int cnt = 0;
123+
for (int idx : lookupKeyIndicesInOrder) {
124+
LookupJoinUtil.LookupKey key = lookupKeysOfRightTable.get(idx);
125+
if (key instanceof LookupJoinUtil.ConstantLookupKey) {
126+
LogicalType keyType = ((LookupJoinUtil.ConstantLookupKey) key).sourceType;
127+
orderedLookupKeyLogicalTypes[cnt] = keyType;
128+
} else if (key instanceof LookupJoinUtil.FieldRefLookupKey) {
129+
int leftIdx = ((LookupJoinUtil.FieldRefLookupKey) key).index;
130+
inputMapping[cnt] = leftIdx;
131+
orderedLookupKeyLogicalTypes[cnt] = inputFieldTypes[leftIdx];
132+
} else {
133+
throw new UnsupportedOperationException("The lookup key " + key + " is invalid.");
134+
}
135+
orderedLookupKeys[cnt] = key;
136+
cnt++;
137+
}
138+
RowType orderedLookupKeyRowType = RowType.of(orderedLookupKeyLogicalTypes);
139+
GeneratedProjection generatedProjection =
140+
ProjectionCodeGenerator.generateProjectionForLookupKeysFromLeftTable(
141+
orderedLookupKeys,
142+
new CodeGeneratorContext(new Configuration(), classLoader),
143+
"LookupKeyProjection",
144+
leftTableRowType.toRowType(),
145+
orderedLookupKeyRowType,
146+
inputMapping,
147+
GenericRowData.class);
148+
return new GenericRowDataKeySelector(
149+
InternalTypeInfo.of(orderedLookupKeyRowType),
150+
InternalSerializers.create(orderedLookupKeyRowType),
151+
generatedProjection);
152+
}
94153
}

0 commit comments

Comments
 (0)