You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Copy file name to clipboardExpand all lines: docs/content/docs/dev/table/sql/queries/hints.md
+43Lines changed: 43 additions & 0 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -279,6 +279,49 @@ SELECT /*+ NEST_LOOP(t1) */ * FROM t1 JOIN t2 ON t1.id = t2.id;
279
279
SELECT/*+ NEST_LOOP(t1, t3) */*FROM t1 JOIN t2 ONt1.id=t2.idJOIN t3 ONt1.id=t3.id;
280
280
```
281
281
282
+
#### MULTI_JOIN
283
+
284
+
{{< label Streaming >}}
285
+
286
+
`MULTI_JOIN` suggests that Flink uses the `MultiJoin operator` to process multiple regular joins simultaneously. This type of join hint is recommended when you have multiple joins that share at least one common join key and experience large intermediate state or record amplification. The MultiJoin operator eliminates intermediate state by processing joins across various input streams simultaneously, which can significantly reduce state size and improve performance in some cases.
287
+
288
+
For more details on the MultiJoin operator, including when to use it and configuration options, see [Multiple Regular Joins]({{< ref "docs/dev/table/tuning" >}}#multiple-regular-joins).
289
+
290
+
{{< hint info >}}
291
+
Note:
292
+
- The MULTI_JOIN hint can specify table names or table aliases. If a table has an alias, the hint must use the alias name.
293
+
- At least one key must be shared between the join conditions for the MultiJoin operator to be applied.
294
+
- When specified, the MULTI_JOIN hint applies to the tables listed in the hint within the current query block.
295
+
{{< /hint >}}
296
+
297
+
##### Examples
298
+
299
+
```sql
300
+
CREATETABLEt1 (id BIGINT, name STRING, age INT) WITH (...);
301
+
CREATETABLEt2 (id BIGINT, name STRING, age INT) WITH (...);
302
+
CREATETABLEt3 (id BIGINT, name STRING, age INT) WITH (...);
303
+
304
+
-- Flink will use the MultiJoin operator for the three-way join.
Copy file name to clipboardExpand all lines: docs/content/docs/dev/table/tuning.md
+26-8Lines changed: 26 additions & 8 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -307,9 +307,9 @@ MiniBatch optimization is disabled by default for regular join. In order to enab
307
307
308
308
{{< label Streaming >}}
309
309
310
-
Streaming Flink jobs with multiple non-temporal regular joins often experience operational instability and performance degradation due to large state sizes. This is often because the intermediate state created by a chain of joins is much larger than the input state itself. In Flink 2.1, we introduce a new multi-join operator, an optimization designed to significantly reduce state size and improve performance for join pipelines that involve record amplification and large intermediate state. This new operator eliminates the need to store intermediate state for joins across multiple tables by processing joins across various input streams simultaneously. This "zero intermediate state" approach primarily targets state reduction, offering substantial benefits in resource consumption and operational stability.
310
+
Streaming Flink jobs with multiple non-temporal regular joins often experience operational instability and performance degradation due to large state sizes. This is often because the intermediate state created by a chain of joins is much larger than the input state itself. In Flink 2.1, we introduce a new multi-join operator, an optimization designed to significantly reduce state size and improve performance for join pipelines that involve record amplification and large intermediate state. This new operator eliminates the need to store intermediate state for joins across multiple tables by processing joins across various input streams simultaneously. This "zero intermediate state" approach primarily targets state reduction, offering substantial benefits in resource consumption and operational stability in some cases. This technique exchanges a reduction in storage requirements for a corresponding increase in computational effort, as intermediate states are re-evaluated upon necessity.
311
311
312
-
In most joins, a significant portion of processing time is spent fetching records from the state. The efficiency of the MultiJoin operator largely depends on the size of this intermediate state. In a common scenario where a pipeline experiences record amplification—meaning each join produces more data and records than the previous one, the MultiJoin operator is more efficient. This is because it keeps the state on which the operator interacts much smaller, leading to a more stable operator. If a chain of joins actually produces less state than the original records, the MultiJoin operator will still use less state overall. However, in this specific case, binary joins might perform better because the state that the final joins need to operate on is smaller.
312
+
In most joins, a significant portion of processing time is spent fetching records from the state. The efficiency of the MultiJoin operator largely depends on the size of this intermediate state and the selectivity of the common join key(s). In a common scenario where a pipeline experiences record amplification—meaning each join produces more data and records than the previous one, the MultiJoin operator is more efficient. This is because it keeps the state on which the operator interacts much smaller, leading to a more stable operator. If a chain of joins actually produces less state than the original records, the MultiJoin operator will still use less state overall. However, in this specific case, binary joins might perform better because the state that the final joins need to operate on is smaller.
313
313
314
314
### The MultiJoin Operator
315
315
The main benefits of the MultiJoin operator are:
@@ -318,21 +318,39 @@ The main benefits of the MultiJoin operator are:
318
318
2) Improved performance for chained joins with record amplification.
319
319
3) Improved stability: linear state growth with amount of records processed, instead of polynomial growth with binary joins.
320
320
321
-
Also, pipelines with MultiJoin instead of binary joins usually have faster initialization and recovery times due to smaller state and fewer amount of nodes.
321
+
Also, pipelines with MultiJoin instead of binary joins usually have faster initialization and recovery times due to smaller state and fewer nodes.
322
322
323
323
### When to enable the MultiJoin?
324
324
325
-
If your job has multiple joins that share at least one common join key, and you observe that the intermediate state in the intermediate joins is larger than the inputs sources, consider enabling the MultiJoin operator.
325
+
If your job has multiple joins that share at least one common join key, and you observe that the intermediate state in the intermediate joins is larger than the input sources, consider enabling the MultiJoin operator.
326
+
327
+
Recommended use cases:
328
+
- The common join key(s) have a high selectivity (the number of records per key is small)
329
+
- Statement with several chained joins and considerable intermediate state
330
+
- No considerable data skew on the common join key(s)
331
+
- Joins are generating large state (state 50+ GB)
332
+
333
+
If your common join key(s) exhibit low selectivity (i.e., a high number of rows sharing the same key value), the MultiJoin operator's required recomputation of the intermediate state can severely impact performance. In such scenarios, binary joins are recommended, as these will partition the data using all join keys.
326
334
327
335
### How to enable the MultiJoin?
328
336
329
-
To enable this optimization, set the following configuration
337
+
To enable this optimization globally for all eligible joins, set the following configuration:
330
338
331
339
```sql
332
340
SET'table.optimizer.multi-join.enabled'='true';
333
341
```
334
342
335
-
Important: This is currently in an experimental state - there are open optimizations and breaking changes might be implemented in this version. We currently support only streaming INNER/LEFT joins. Support for RIGHT joins will be added soon. Due to records partitioning, you need at least one key that is shared between the join conditions, see:
343
+
Alternatively, you can enable the MultiJoin operator for specific tables using the `MULTI_JOIN` hint:
344
+
345
+
```sql
346
+
SELECT/*+ MULTI_JOIN(t1, t2, t3) */*FROM t1
347
+
JOIN t2 ONt1.id=t2.id
348
+
JOIN t3 ONt1.id=t3.id;
349
+
```
350
+
351
+
The hint approach allows you to selectively apply the MultiJoin optimization to specific query blocks without enabling it globally. You can specify either table names or table aliases in the hint. For more details on the MULTI_JOIN hint, see [Join Hints]({{< ref "docs/dev/table/sql/queries/hints" >}}#multi_join).
352
+
353
+
Important: This is currently in an experimental state - optimizations and breaking changes might be implemented. We currently support only streaming INNER/LEFT joins. Due to records partitioning, you need at least one key that is shared between the join conditions, see:
336
354
337
355
- Supported: A JOIN B ON A.key = B.key JOIN C ON A.key = C.key (Partition by key)
338
356
- Supported: A JOIN B ON A.key = B.key JOIN C ON B.key = C.key (Partition by key via transitivity)
@@ -349,9 +367,9 @@ For this 10-way join above, involving record amplification, we've observed signi
349
367
- Performance: 2x to over 100x+ increase in processed records when both at 100% busyness.
350
368
- State Size: 3x to over 1000x+ smaller as intermediate state grows.
351
369
352
-
The total state is always smaller with the MultiJoin operator. In this case, the performance is initially the same, but as the intermediate state grows, the performance of binary joins degrade and the multi join remains stable and outperforms.
370
+
The total state is always smaller with the MultiJoin operator. In this case, the performance is initially the same, but as the intermediate state grows, the performance of binary joins degrades and the multi join remains stable and outperforms.
353
371
354
-
This general benchmark for the 10-way join was run with the following configuration: 10 upsert kafka topics, 10 parallelism, 1 record per second per topic. We used rocksdb with unaligned checkpoints and with incremental checkpoints. Each job ran in one TaskManager containing 8GB process memory, 1GB off-heap memory and 20% network memory. The JobManager had 4GB process memory. The host machine contained a M1 processor chip, 32GB RAM and 1TB SSD. The sink uses a blackhole connector so we only benchmark the joins. The SQL used to generate the benchmark data had this structure:
372
+
This general benchmark for the 10-way join was run with the following configuration: 1 record per tenant_id (high selectivity), 10 upsert kafka topics, 10 parallelism, 1 record per second per topic. We used rocksdb with unaligned checkpoints and with incremental checkpoints. Each job ran in one TaskManager containing 8GB process memory, 1GB off-heap memory and 20% network memory. The JobManager had 4GB process memory. The host machine contained a M1 processor chip, 32GB RAM and 1TB SSD. The sink uses a blackhole connector so we only benchmark the joins. The SQL used to generate the benchmark data had this structure:
Copy file name to clipboardExpand all lines: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java
+5Lines changed: 5 additions & 0 deletions
Original file line number
Diff line number
Diff line change
@@ -121,6 +121,11 @@ public static HintStrategyTable createHintStrategyTable() {
Copy file name to clipboardExpand all lines: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java
Copy file name to clipboardExpand all lines: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala
+14-16Lines changed: 14 additions & 16 deletions
Original file line number
Diff line number
Diff line change
@@ -233,22 +233,20 @@ object FlinkStreamProgram {
233
233
}
234
234
235
235
// multi-join
236
-
if (tableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED)) {
Copy file name to clipboardExpand all lines: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java
+2-1Lines changed: 2 additions & 1 deletion
Original file line number
Diff line number
Diff line change
@@ -54,6 +54,7 @@ public List<TableTestProgram> programs() {
Copy file name to clipboardExpand all lines: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java
+30Lines changed: 30 additions & 0 deletions
Original file line number
Diff line number
Diff line change
@@ -1823,4 +1823,34 @@ public class MultiJoinTestPrograms {
1823
1823
+ "LEFT JOIN Shipments4K AS S ON U.k3 = S.k3 AND U.k2 > 150 AND U.k4 = S.k4 "
0 commit comments