Discussion threadhttps://lists.apache.org/thread/qq39rmg3f23ysx5m094s4c4cq0m4tdj5
Vote threadhttps://lists.apache.org/thread/oy9mdmh6gk8pc0wjdk5kg8dz3jllz9ow
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).

[This FLIP proposal is a joint work of Ran JinhaoDong Lin and Xuannan Su ]


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. We will call such operator full-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.

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 task deployment and resource utilization 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 will be blocking, which effectively let Flink schedule and execute this operator in batch mode.

In addition, this FLIP also allows creating a GlobalWindows that the window will be triggered only at the end of its inputs. DataStream programs (e.g. coGroup and aggregate) can take advantage of this information to significantly increase throughput and reduce resource utilization.


Objectives

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


1) 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 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 to create a GlobalWindow that is only triggered at the end of its inputs, instead of writing tens of lines of code to define a 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. User-defined operators should be able to set the attributes accordingly so that they can achieve the benefit.


Public Interfaces

1) Add the createWithEndOfStreamTrigger method in GlobalWindows which allows users of the DataStream API to create GlobalWindows with a window trigger that only fires when the input ends.

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

    /**
     * Creates a {@link WindowAssigner} that assigns all elements to the same {@link GlobalWindow}.
     * The window is only useful if you also specify a custom trigger. Otherwise, the window will
     * never be triggered and no computation will be performed.
     */
	public static GlobalWindows create() {
        ...
    }      

    /**
     * Creates a {@link WindowAssigner} that assigns all elements to the same {@link GlobalWindow}
     * and the window is triggered if and only if the input stream is ended.
     */
	public static GlobalWindows createWithEndOfStreamTrigger() {
        ...
    }

 }


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 {
 
    public OperatorAttributesBuilder() {...}        

    /**
     * Set to true if and only if the operator only emits records after all its inputs have ended.
     * If it is not set, the default value false is used.
     */   
     public OperatorAttributesBuilder setOutputOnlyAfterEndOfStream(
            boolean outputOnlyAfterEndOfStream) {...}            

	/** If any operator attribute is not set explicitly, we will log it at DEBUG level. */
	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 if and only if the operator only emits records after all its inputs have ended.
     *
     * <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 isOutputOnlyAfterEndOfStream() {...}
}


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

public interface StreamOperator<OUT> extends CheckpointListener, KeyContext, Serializable {
    ...
	
    /**
     * Get the {@link OperatorAttributes} of the operator that the Flink runtime can use to optimize
     * the job performance.
     *
     * @return OperatorAttributes of the operator.
     */
    @Experimental
    default OperatorAttributes getOperatorAttributes() {
        return new OperatorAttributesBuilder().build();
    }
}
 

public interface StreamOperatorFactory<OUT> extends Serializable {
    ...

    /**
     * Get the {@link OperatorAttributes} of the operator created by the factory. Flink runtime can
     * use it to optimize the job performance.
     *
     * @return OperatorAttributes of the operator created by the factory.
     */
    @Experimental
    default OperatorAttributes getOperatorAttributes() {
        return new OperatorAttributesBuilder().build();
    }
}


Proposed Changes

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

@Internal 
public abstract class PhysicalTransformation<T> extends Transformation<T> {
    public boolean isOutputOnlyAfterEndOfStream() {
        return false;
    }
}


2) Update the following Transformation subclasses to override the newly added methods using the OperatorAttributes obtained from the underlying Operator:

  • OneInputTransformation
  • TwoInputTransformation
  • AbstractMultipleInputTransformation


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

If a Transformation has isOutputOnlyAfterEndOfStream == true:

  • The process of operator 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) Enumerate all existing operators defined in Flink and set the isOutputOnlyAfterEndOfStream accordingly for those operators that may only output at the end of the input to enable the optimization mentioned in 3):

  • WindowOperator: When used with GlobalWindows that is created with the method createWithEndOfStreamTrigger as the window assigner
  • StreamSortOperator


5) When DataStream API like DataStream#CoGroup is invoked with GlobalWindows, the WindowOperator will check if the window assigner is an instance of GlobalWindow and only triggers at the end of the inputs. If true, the operator will have OperatorAttributes with isOutputOnlyAfterEndOfStream = true to achieve higher throughput by using the optimizations in batch mode.

Analysis of APIs affected by this FLIP

This FLIP can potentially benefit all DataStream APIs that take WindowAssigner as the parameter.

More specifically, the following DataStream API can benefit from using GlobalWindows which is created with the method createWithEndOfStreamTrigger.

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

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 isOutputOnlyAfterEndOfStream 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 must set the isOutputOnlyAfterEndOfStream to true to enable the optimization.

Future Work

  • It would be useful to add an ExecutionState (e.g. Finishing) to specify whether the task has reached the end 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 the full-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.
  • Optimizing the implementation of frequently used full-dam operators,  such as aggregate/CoGroup/Join/..., can achieve higher throughput by using the optimizations currently done in batch mode. For example, we can instantiate an internal sorter in the operator, and it will not have to invoke WindowAssigner#assignWindows or triggerContext#onElement for each input record.