You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 15 Next »

Discussion thread-
Vote thread-
JIRA

Unable to render Jira issues macro, execution error.

Release-

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

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. 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.

Currently, supporting the above use-case requires all operators to be deployed at the start of the job. This approach wastes slot and memory resources because operatorB can not do any useful work until operatorA's input has ended. Even worse, operatorB might use a lot of disk space only to cache and spill records received from the unbounded source to its local disk while it is waiting for operatorA's output.

In this FLIP, we propose to optimize performance for the above use-case by allowing an operator to explicitly specify whether it only emits records after all its inputs have ended. JM will leverage this information to optimize job scheduling such that the partition type of the results emitted by this operator, as well as the results emitted by its upstream operators, will all be blocking, which effectively let Flink schedule and execute this operator as well as its upstream operators in batch mode. Hybrid shuffle mode(FLIP-235: Hybrid Shuffle Mode) can be used in batch mode part to further improve the performance when there are sufficient slot resources.


Public Interfaces

1) Add EndOfStreamWindows which is a subclass of WindowAssigner. This class allows users of the DataStream API to specify whether the computation (e.g. co-group, aggregate) should emit data only after end-of-input.

/**
 * This WindowAssigner assigns all elements to the same window that is fired iff the input
 * stream reaches EOF.
 */
@PublicEvolving
public class EndOfStreamWindows extends WindowAssigner<Object, TimeWindow> {
 
    private static final TimeWindow TIME_WINDOW_INSTANCE =
            new TimeWindow(Long.MIN_VALUE, Long.MAX_VALUE);
 
    private EndOfStreamWindows() {}
 
    public static EndOfStreamWindows get() {
        return INSTANCE;
    }
 
    @Override
    public Collection<TimeWindow> assignWindows(
            Object element, long timestamp, WindowAssignerContext context) {
        return Collections.singletonList(TIME_WINDOW_INSTANCE);
    }
 
    @Override
    public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
        return new EndOfStreamTrigger();
    }
 
    @Override
    public boolean isEventTime() {
        return true;
    }
 
    private static class EndOfStreamTrigger extends Trigger<Object, TimeWindow> {
        @Override
        public TriggerResult onElement(
                Object element, long timestamp, TimeWindow window, TriggerContext ctx)
                throws Exception {
            return TriggerResult.CONTINUE;
        }
 
        @Override
        public TriggerResult onEventTime(long time, TimeWindow window, TriggerContext ctx) {
            return time == window.maxTimestamp() ? TriggerResult.FIRE : TriggerResult.CONTINUE;
        }
 
        @Override
        public TriggerResult onProcessingTime(long time, TimeWindow window, TriggerContext ctx) {
            return TriggerResult.CONTINUE;
        }
        ...
    }
}


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

package org.apache.flink.streaming.api.operators;
 
/** The builder class for {@link OperatorAttributes}. */
@Experimental
public class OperatorAttributesBuilder {
    @Nullable private Boolean outputOnEOF = null;
    @Nullable private Boolean outputOnCheckpoint = null;
 
    public OperatorAttributesBuilder() {...}
 
    public OperatorAttributesBuilder setOutputOnEOF(boolean outputOnEOF) {...}
 
    public OperatorAttributesBuilder setOutputOnCheckpoint(boolean outputOnCheckpoint) {...}
  
    /**
     * If any operator attribute is null, we will log it at DEBUG level and use the following
     * default values.
     * - outputOnEOF defaults to false
     * - outputOnCheckpoint defaults to false
     */
     public OperatorAttributes build() {...}
}
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 emit 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 upstream operators have blocking partition type.
     *   <li> This operator as well as its upstream operators will be executed in batch mode.
     * </ul>
     */
    public boolean isOutputOnEOF() {...}
 
    /**
     * Returns true iff the operator can only emit records when checkpoint is triggered.
     */
    public boolean isOutputOnCheckpoint() {...}
}


3) Add the getOperatorAttributes() API to the StreamOperator and StreamOperatorFactory interfaces.

@Experimental
public interface StreamOperator<OUT> extends CheckpointListener, KeyContext, Serializable {
    ...
 
    default OperatorAttributes getOperatorAttributes() {
        return new OperatorAttributesBuilder().build();
    }
}
 
@Experimental
public interface StreamOperatorFactory<OUT> extends Serializable {
    ...
 
    default OperatorAttributes getOperatorAttributes() {
        return new OperatorAttributesBuilder().build();
    }
}


Proposed Changes

1) Add the APIs on Transformation interface to get the corresponding operator attributes.

@Internal
public abstract class Transformation<T> {
    public boolean isOutputOnEOF() {
        return false;
    }
 
    public boolean isOutputOnCheckpoint() {
        return false;
    }
}


2) Update Transformation subclasses (e.g. OneInputTransformation and TwoInputTransformation) to override the newly added methods using the OperatorAttributes obtained from the underlying Operator.


3) Update JM to make use of the following operator attributes when compiling the Transformation graph into the JobGraph.

  • If a Transformation has isOutputOnEOF == true:
    • The results of this operator as well as its upstream operators have blocking (by default) or hybrid shuffle partition type which is controlled by configuring ExecutionOptions.BATCH_SHUFFLE_MODE.
    • This operator as well as its upstream operators will be executed in batch mode (e.g checkpoint is disabled when these operators are running).
  • If all Transformation has isOutputOnCheckpoint == false:
    • In FLIP-325, JM will not trigger an extra flush() before triggering a checkpoint.

4) A blocking input edge with pending records is same as a source with isBacklog=true when an operator determines its RecordAttributes for downstream nodes.


5) When DataStream#coGroup is invoked with EndOfStreamWindows as the window assigner, Flink should generate an operator with isOutputOnEOF = true.

In addition, after FLIP-327 is accepted, this operator should also have isInternalSorterSupported = true

This operator will use the follow optimization to achieve much higher throughput than the existing DataStream#coGroup in both stream and batch mode:

  • It will instantiate two internal sorter to sorts records from its two inputs separately. Then it can pull the sorted records from these two sorters. This can be done without wrapping input records with TaggedUnion<...>. In comparison, the existing DataStream#coGroup needs to wrap input records with TaggedUnion<...> before sorting them using one external sorter, which introduces higher overhead.
  • It will not invoke WindowAssigner#assignWindows or triggerContext#onElement for input records. In comparison, the existing WindowOperator#processElement invokes these methods for every records.


6) When DataStream#aggregate is invoked with EndOfStreamWindows as the window assigner, Flink should generate an operator with isOutputOnEOF = true.

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

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


jobmanager.memory.process.size: 6400m
taskmanager.memory.process.size: 6912m

Benchmark: CoGroup

Use DataStream#coGroup to process records from two bounded inputs both with Data Count number of records distributed in Data Count number of keys. CoGroup results will be emitted after input have ended.

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
1e763 ± 1 (100%, 158245 ms)532 ± 12 (844%, 18768 ms)1359 ± 41 (2157%, 7357 ms)1545 ± 35 (2452%, 6472 ms)
3e760 ± 0 (100%, 497587 ms)554 ± 23 (923%, 54112 ms)1202 ± 9 (2003%, 24958 ms)1334 ± 17 (2223%, 22485 ms)
5e756 ± 1 (100%, 878518 ms)572 ± 5 (1021%, 87296 ms)1163 ± 2 (2076%, 42961 ms)1283 ± 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. Aggregate results will be emitted after input have ended.

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 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)

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

We have a job that 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.

 source1.keyBy(value -> value.f0)
  .window(EndOfStreamWindows.get())
  .aggregate(new MyAggregator()).name("Process1")
  .connect(source2.keyBy(value -> value.f0))
  .transform("Process2", Types.INT, new MyProcessOperator())
  .addSink(new DiscardingSink<>()); 

Resource Utilization

Suppose we set an extra standalone cluster configuration:

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
1e7183 ± 19 (100%, 54370 ms)1371 ± 43 (749%, 7289 ms)1568 ± 49 (856%, 6376 ms)
3e740 ± 10 (747765 ms)1461 ± 19 (20531 ms)1674 ± 41 (17912 ms)
5e7OOM1522 ± 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

The 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.


  • No labels