Versions Compared

Key

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

...

As shown in the figure below, we might have a job that pre-processes records from a bounded source (i.e. inputA) using an operator (i.e. operatorA) which only emits results after its input has ended, which is also known as dam operator. Then operatorB needs to join records emitted by operatorA with records from an unbounded source, and emit results with low processing latency in real-time.

...

In addition, this FLIP also adds EndOfStreamTrigger in GlobalWindows. GlobalWindows with EndOfStreamTrigger can be used with the DataStream API to specify whether the window will end only at the end of its inputs. DataStream program programs (e.g. coGroup and aggregate) can take advantage of this information to significantly increase throughput and reduce resource utilization.

...

The APIs provided in this FLIP achieves the following objectives / benefits:


1) Improve throughputReduce resource usage.  For use-case that involves a mixture of bounded and unbounded workloads (e.g. the use-case specified in the example), the user can start a single Flink job to address such use-case where the performance of the bounded workload can be optimal (e.g. similar/higher than the corresponding performance in batch mode).

2) Reduce resource usage.  For use-case that involves an operatorB (with unbounded input) depending on the output of another operatorA, where operatorA only emits results at the end of its input, Flink will deploy operatorB after operatorA is finished. This approach reduces the unnecessary resource usage when operatorA is still processing its inputs.

3) Improve usability. For use-case that needs to invoke DataStream APIs (e.g. KeyedStream#window) with a window assigner that covers all input data, users can use the off-the-shelf GlobalWindows with EndOfStreamTrigger provided in this FLIP, instead of writing tens of lines of code to define this WindowAssigner subclass.  Existing operators can also redefine operator attributes to achieve this objective.

Public Interfaces

an operatorB (with unbounded input) depending on the output of another operatorA, where operatorA only emits results at the end of its input, Flink will deploy operatorB after operatorA is finished. This approach reduces unnecessary resource usage when operatorA is still processing its inputs.


2) Improve usability. For use-case that needs to invoke DataStream APIs (e.g. KeyedStream#window) with a window assigner that covers all input data, users can use the off-the-shelf GlobalWindows with EndOfStreamTrigger provided in this FLIP, instead of writing tens of lines of code to define this WindowAssigner subclass. 


3) Introduce attributes to the operator to let Flink know if the operator only outputs results when the input has ended. All the operators defined in Flink should set the attributes accordingly to achieve the benefit above. For user-defined operators, it should set the attributes accordingly so that it can achieve the benefit.


Public Interfaces

1) Add the EndOfStreamTrigger in GlobalWindows which allows users of the DataStream API to create GlobalWindows with EndOfStreamTrigger to specify whether the computation (e.g. co-group, aggregate) should 1) Add the EndOfStreamTrigger in GlobalWindows which allows users of the DataStream API to create GlobalWindows with EndOfStreamTrigger to specify whether the computation (e.g. co-group, aggregate) should emit data only after end-of-input.

Code Block
languagejava
@PublicEvolving
public class GlobalWindows extends WindowAssigner<Object, GlobalWindow> {
   ···   

    
	/**
     * Creates a new {@link WindowAssigner} with {@link NeverTrigger} that assigns all elements to
     * the same {@link GlobalWindow}.
 The window is only useful  *if you also specify a custom
     * @returntrigger. TheOtherwise, globalthe window policy with {@link NeverTrigger} as default trigger will never be triggered and no computation will be performed.
     */
      public static* GlobalWindows create() {
@return The global window policy with {@link NeverTrigger} as default trigger.
     */
    returnpublic newstatic GlobalWindows(new NeverTriggercreate()); {
        ...
    }
	
    /**
     * Creates a new {@link WindowAssigner} with {@link EndOfStreamTrigger} that assigns all
     * elements to the same {@link GlobalWindow} and the default trigger fires if and only if the
     * input stream reachesis EOFended.
     *
     * @return The global window policy with {@link EndOfStreamTrigger} as default trigger.
     */
      public static GlobalWindows createWithEndOfStreamTrigger() {
        return new GlobalWindows(new EndOfStreamTrigger());...
    }

 }


2) Add OperatorAttributesBuilder and OperatorAttributes for operator developers to specify operator attributes that Flink runtime can use to optimize the job performance.

...

Code Block
languagejava
package org.apache.flink.streaming.api.operators;
 
/**
 * OperatorAttributes element provides Job Manager with information that can be
 * used to optimize the job performance.
 */
@Experimental
public class OperatorAttributes {
  
    /**
     * Returns true iff the operator can only emitemits records after inputs have reached EOF.
     *
     * <p>Here are the implications when it is true:
     *
     * <ul>
     *   <li> The results of this operator as well as its chained operators have blocking partition type.
     *   <li> This operator as well as its chained operators will be executed in batch mode.
     * </ul>
     */
    public boolean isOutputOnEOF() {...}
}

...

If a Transformation has isOutputOnEOF == true:

  • The process of operator chain chaining can still be done. After that, the results of its operator chain will be set blocking.
  • This operator will be executed in batch mode.

    More specifically, the checkpoint is disabled when these operators are running, such that these operators can do operations not compatible with checkpoints (e.g. sorting inputs). And operators should re-read the data from the upstream blocking edge or sources after failover. 


4) All Enumerate all existing operators can redefine operator attributes,isOutputOnEOF == true, to use new features metioned in 3).defined in Flink and set the isOutputOnEOF for those operators that only output at the end of the input to enable the optimization mentioned in 3).


5) 5) When DataStream API like DataStream#CoGroup is invoked with GlobalWindows andEndOfStreamTrigger as the window assigner and trigger, Flink should generate an operator The operator will have OperatorAttributes with isOutputOnEOF = true to achieve higher throughput by using the optimizations in batch mode.

...

  • DataStream#windowAll
  • KeyedStream#window
  • CoGroupedStreams#Where#EqualTo#window
  • JoinedStreams#Where#EqualTo#window

Benchmark results

To demonstrate our optimization improvements,we run each benchmark in different execution modes and configurations. Each result is measured by taking the average execution time across 5 runs with the given configuration.

We run benchmarks on a MacBook with the latest Flink 1.18-snapshot and parallelism=1. Default standalone cluster configuration is used except:

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

1) Execute DataStream#CoGroup

This benchmark uses DataStream#coGroup to process records from two bounded inputs. Each input will generate records with (key = i, value = i) for i from 1 to 8*10^7.

Below is the DataStream program code snippet.

Code Block
languagejava
DataStream<Tuple2<Integer, Double>> source1  = env.fromCollection(
                new DataGenerator(dataNum), Types.TUPLE(Types.INT, Types.DOUBLE));
DataStream<Tuple2<Integer, Double>> source2  = env.fromCollection(
                new DataGenerator(dataNum), Types.TUPLE(Types.INT, Types.DOUBLE));

source1.coGroup(source2)
    .where(tuple -> tuple.f0)
    .equalTo(tuple -> tuple.f0)
    .window(GlobalWindows.createWithEndOfStreamTrigger())
    .apply(new CustomCoGroupFunction())
    .addSink(...);

private static class CustomCoGroupFunction
            extends RichCoGroupFunction<Tuple2<Integer, Double>, Tuple2<Integer, Double>, Integer> {
        @Override
        public void coGroup(
                Iterable<Tuple2<Integer, Double>> iterableA,
                Iterable<Tuple2<Integer, Double>> iterableB,
                Collector<Integer> collector) {
            collector.collect(1);
        }
    }

The following result shows the throughput (records/sec) when the benchmark is executed in streaming mode, batch mode and optimized streaming mode after this PR.

The result shows that DataStream#coGroup in optimized streaming mode can be 22X as fast as streaming mode and 3X as fast as batch mode. 

STREAMINGBATCHOptimized STREAMING
66 ± 1 (100%, 1202426 ms)491 ± 5 (743%, 162731 ms)1506 ± 10 (2281%, 53098 ms)

2) Execute DataStream#Aggregate

This benchmark uses DataStream#aggregate to process 8*10^7 records. These records are evenly distributed across 8*10^5 keys. More specifically, the source will generate records with (key = i, value = i) for i from 1 to 8*10^5, and repeat this process 100 times.

Below is the DataStream program code snippet.

Code Block
languagejava
DataStreamSource<Tuple2<Long, Double>> source  = env.fromCollection(
                        new DataGenerator(dataNum, keyNum), Types.TUPLE(Types.LONG, Types.DOUBLE));
source.keyBy(value -> value.f0)
      .window(GlobalWindows.createWithEndOfStreamTrigger())
      .aggregate(new Aggregator())
      .addSink(...);

public static class Aggregator implements AggregateFunction<Tuple2<Long, Double>, Tuple2<Long, Double>, Double> {
        @Override
        public Tuple2<Long, Double> createAccumulator() {
            return new Tuple2<Long, Double>(0L, 0.0);
        }

        @Override
        public Tuple2<Long, Double> add(Tuple2<Long, Double> myData, Tuple2<Long, Double> accData) {
            accData.f1 = accData.f1 + myData.f1;
            return accData;
        }

        @Override
        public Double getResult(Tuple2<Long, Double> result) {
            return result.f1;
        }

        @Override
        public Tuple2<Long, Double> merge(Tuple2<Long, Double> acc1, Tuple2<Long, Double> acc2) {
            acc1.f1 = acc1.f1 + acc2.f1;
            return acc1;
        }
    }

The following result shows the throughput (records/sec) when the benchmark is executed in streaming mode, batch mode and optimized streaming mode after this PR.

The result shows that DataStream#aggregate in optimized streaming mode can be 10X as fast as streaming mode and 11% faster than batch mode

STREAMINGBATCHOptimized STREAMING
163 ± 0 (100%, 490478 ms)1561 ± 16 (957%, 51237 ms)1733 ± 9 (1063%, 46143 ms)

3) Execute a program that needs to fully process data from a bounded source before processing data from another unbounded source.

The following program demonstrates the scenario described in the motivation section. The program needs to pre-processes records from a bounded source (Source1) using an operator (Process1) which only emits results after its input has ended. Then anther operator(Process2) needs to process records emitted by Process1 with records from an unbounded source, and emit results with low processing latency in real-time.

Code Block
languagejava
 source1.keyBy(value -> value.f0)
  .window(GlobalWindows.createWithEndOfStreamTrigger())
  .aggregate(new MyAggregator()).name("Process1")
  .connect(source2.keyBy(value -> value.f0))
  .transform("Process2", Types.INT, new MyProcessOperator())
  .addSink(...); 

We can use this program to demonstrate that the program requires less slot resources. More specifically, suppose we configure the standalone cluster with taskmanager.numberOfTaskSlots = 2, and set the Source1,Process1, Source2 and Process2 in 4 different SlotSharingGroups, the program will fail to be deployed before this FLIP. And the program can be deployed successfully after this FLIP. This is because Source2 and Process2 can be deplyed after Source1Process1 finished and released their slots.

Additionally, we can use this program to demonstrate that it can achieve higher performance because Process2 will not need to keep buffer records emitted by Source2 in its memory while Process1 has not reached EOF. More specifically, the program can fail with OOM before this FLIP when the number of records in inputs is high. And the program can finish successfully without OOM after this FLIP.

Compatibility, Deprecation, and Migration Plan

Compatibility, Deprecation, and Migration Plan

The changes made in this FLIP are backward compatible. No deprecation or migration plan is needed.


As mentioned in Proposed Change 4), we will enumerate all the existing operators defined in Flink and set the isOutputOnEOF accordingly. For user-defined operators, their behavior stays the same as before. If the user-defined operators only output at the end of input, they have to set the isOutputOnEOF to true to enable the optimizationThe changes made in this FLIP are backward compatible. No deprecation or migration plan is needed.

Future Work

  • It would be useful to add an ExecutionState (e.g. Finishing) to specify whether the task has reached EOF for all its inputs. This allows JM to deploy its downstream tasks and possibly apply hybrid shuffle to increase job throughput.
  • Setting the upstream operators of EOF the dam operator in batch mode will also increase the job throughput, and Hybrid shuffle mode can also be used in batch mode part to further improve the performance when there are sufficient slot resources.
  • Developing Optimizing the implementation of frequently used EOF operatordam operators,  such as aggregate/CoGroup/Join/..., can achieve higher throughput by using the optimizations currently done in batch mode. We For example, we can instantiate an internal sorter in the operator, and flink it will not have to invoke WindowAssigner#assignWindows or triggerContext#onElement for each input record to reduce overhead.