Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

In addition, after FLIP-327 is accepted, this operator should also have isInternalSorterSupported = true. This operator will sort the input before aggregate, and avoid invoking window actions, which is similar to '5)'.

Benchmark results

Generally, when operator(coGroup or aggregate) is invoked optimized with EndOfStreamWindows as the window assigner, the optimized operator EOFCoGroupOperator /EOFAggregationOperator will be applied, in which case the input(s) will be sorted first and then output the results after the input end of input. This makes the performance with our work can be mutiple times faster than before and slightly faster than batch mode because of avoiding window related work.Using hybrid shuffle in the outputEOF part can make part of operator pipeline consume records from upstream to further improve the performance.

To demonstrate our optimization improvements,we run each benchmark in different execution modes and configurations 5 times. Here are benchmark results which

...

include average throughput (records/ms), throughput range,

...

the magnification factor for each streaming job before our work and duration time(ms).

Environment Configuration

...

Code Block
languageyml
jobmanager.memory.process.size: 6400m
taskmanager.memory.process.size: 6912m

Benchmark: CoGroup

Use DataStream#coGroup to process records from two bounded streams inputs both with Data Count number of records and emit results after both inputs have ended. With this PR, when DataStream#coGroup is invoked with EndOfStreamWindows as the window assigner, EOFCoGroupOperator will be applied, in which two inputs will be sorted first and then output the coGroup resultsdistributed in Data Count number of keys. CoGroup results will be emitted after input have ended.

Code Block
languagejava
data1.coGroup(data2)
    .where(tuple -> tuple.f0)
    .equalTo(tuple -> tuple.f0)
    .window(EndOfStreamWindows.get())
    .apply(new CustomCoGroupFunction())
    .addSink(...);


In this benchmark, we run each job in CoGroupDataStream 5 times in 4 kinds of configuration: streaming mode, batch mode, optimized streaming mode after this PR and optimized streaming mode with hybrid shuffle after this PR. Here are the throughput benchmark results:

Data CountSTREAMINGBATCHOptimized STREAMINGWith hybrid shuffle
2e61e761 63 ± 1 (100%, 158245 ms)502 532 ± 75 (823%12 (844%, 18768 ms)868 1359 ± 74 (1423%41 (2157%, 7357 ms)899 ± 82 (1474%)
5e660 ± 1 (100%)528 ± 28 (880%)1176 ± 39 (1960%)1262 ± 50 (2103%)
1545 ± 35 (2452%, 6472 ms)
3e760 2e756 ± 0 (100%, 497587 ms)564 554 ± 34 (1007%23 (923%, 54112 ms)1492 1202 ± 28 (2664%9 (2003%, 24958 ms)1692 1334 ± 14 (3021%17 (2223%, 22485 ms)
5e753 56 ± 1 (100%, 878518 ms)602 572 ± 16 (1135%5 (1021%, 87296 ms)1508 1163 ± 18 (2845%2 (2076%, 42961 ms)1712 1283 ± 77 (3230%8 (2291%, 38953 ms)

This shows with the changes proposed above, DataStream#coGroup in Optimized STREAMING can be

...

20X faster than

...

STREAMING throughput and 2X faster than BATCH throughput in batch mode. Using hybrid shuffle in the outputEOF part can further improve the performance about 10%.

Benchmark: Aggregate

Use DataStream#aggregate to process records from a bounded streams which has Data Count number of records, and every 100 records have the same key. Emit results after input have ended. With this PR, when DataStream#coGroup is invoked with EndOfStreamWindows as the window assigner, EOFAggregationOperator will be applied, in which the input will be sorted first, then aggregate in each key and output resultsof records, and every 100 records have the same key. Aggregate results will be emitted after input have ended.

Code Block
languagejava
data
      .keyBy(value -> value.f0)
      .window(EndOfStreamWindows.get())
      .aggregate(new Aggregator())
      .addSink(new CountingAndDiscardingSink());


In this benchmark, we run each job in AggregationBenchmark 5 times in 4 kinds of configuration: streaming mode, batch mode, optimized streaming mode after this PR and optimized streaming mode with hybrid shuffle after this PR. Here are the throughput benchmark results:

Data CountSTREAMINGBATCHOptimized STREAMINGWith hybrid shuffleSTREAMINGBATCHOptimized STREAMINGWith hybrid shuffle
2e7188 ± 1 (100%, 106190 ms)1565 ± 15 (832%, 12775 ms)1671 ± 31 (888%, 11964 ms)1950 ± 8 (1037%, 10255 ms)
5e7171 ± 1 (100%, 290733 ms)1534 ± 13 (897%, 32588 ms)1712 ± 14 (1001%, 29201 ms)1988 ± 16 (1162%, 25149 ms)
8e7163 ± 0 (100%, 490478 ms)1561 ± 16 (957%, 51237 ms)1733 ± 9 (1063%, 46143 ms)1992 ± 15 (1222%, 40148 ms
5e6201 ± 3 (100%)1454 ± 153 (723%)1514 ± 43 (753%)1670 ± 29 (831%)
1e7207 ± 2 (100%)1522 ± 71 (735%)1620 ± 44 (782%)1866 ± 187 (901%)
2e7201 ± 3 (100%)1607 ± 9 (800%)1746 ± 31 (867%)2038 ± 155 (1014%)
5e7176 ± 1 (100%)1559 ± 6 (885%)1723 ± 22 (978%)1956 ± 56 (1111%)

This shows with the changes proposed above, DataStream#aggregate in Optimized STREAMING can be

...

8~10X faster than

...

STREAMING throughput and slightly faster to the

...

BATCH. Using hybrid shuffle in the outputEOF part can further improve the performance about 15%.

Benchmark: OperatorDelayedDeploy

...

Code Block
languageyml
taskmanager.numberOfTaskSlots: 2

When we set the Source1Process1, Source2 and Process2 in 4 different SlotSharingGroups as above and then run this job, we could see that, before this PR, all of the tasks couldn't deploy and blocked in CREATED state because only 2 slots provided in TaskManager. With our work, Source2 and Process2 can be deplyed after Source1Process1 finished and released their slots.

OperatorDelayedDeployDataStream

Efficiency Improvement

Without setting the slot sharing group

...

, we run OperatorDelayedDeployDataStreamInOneSlot each benchmark 5 times. Process1 will aggregate Data Count number of records from Source1, and every 100 records have the same key. Benchmark terminates after Process2 process Data Count number of records from Source2.

...

Here are the throughput benchmark results:

Data CountSTREAMINGOptimized STREAMINGWith hybrid shuffle
5e61e7205 183 ± 4 19 (100%, 54370 ms)1201 1371 ± 59 (586%43 (749%, 7289 ms)1381 1568 ± 14 (674%49 (856%, 6376 ms)
1e73e7186 40 ± 19 10 (100%747765 ms)1270 1461 ± 64 19 (683%20531 ms)1423 1674 ± 41 (765%17912 ms)
5e7OOM1463 ± 71666 ± 35

...

1522 ± 11 (32843 ms)1750 ± 14 (28561 ms)

This shows that, aggregation optimization above can also be detected in this benchmark. In line 1e7, the magnificaiton is smaller than Benchmark: Aggregate because the stage of Process2 processing records from Source2 in STEAMING is more faster than Optimized STREAMING as records from Source2 already buffered.

As the records number grows,

...

throughput in STREAMING gets decreased because of the buffer presure in Process2, and even causes OOM exception while it works well

...

in Optimized STREAMING which makes the magnification meaningless,. Using hybrid shuffle can make Process1 to pipeline consume records from Source1 to further improve the performance.

Compatibility, Deprecation, and Migration Plan

...