Discussion thread | - |
---|---|
Vote thread | - |
JIRA | - |
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 between Yunfeng Zhou and Dong Lin ]
Motivation
In scenarios where output records represent updates to existing values, there are cases in which output records are required to be updated at a minute-level latency. For instance, a user may want to track the top 5 most-viewed tweets and update the results every 2 minutes. Currently, Flink emits newly generated output results as soon as possible after it finishes processing input records, since it lacks knowledge of the specific update latency required for the use-case (i.e., 2 minutes). This has unneccessarily limited the throughput of a Flink job and consumed more network and computation than actually needed.
To address this limitation, we propose the inclusion of support for user-configurable flush interval in this FLIP. This configuration would enable Flink to optimize its throughput by buffering output records and emitting only the latest updates at the user-defined interval. By doing so, certain operators, such as non-window aggregations, could achieve much higher throughput by reducing the total number of records emitted by the operator.
Public API
1. Add job-level configuration execution.max-flush-interval
Name: execution.max-flush-interval
Type: Duration
Default value: null
Description:
If this configuration is not null, operators with idempotent output will have their output results temporarily buffered in the state backend, and flush the latest output (of each key, if the operator's upstream is a keyed stream) that have been collected to downstream operators periodically before the configured interval passes. This optimization helps to improve throughput by reducing the output of intermediate results at the cost of increased latency and state size.
2. Add OperatorAttributesBuilder and OperatorAttributes for operator developers to specify operator attributes that can be used for performance optimization, and add the property isOutputIdempotent to OperatorAttribute and the Function interface to indicate whether buffering and flushing optimization can be enabled on an operator.
package org.apache.flink.streaming.api.operators; /** The builder class for {@link OperatorAttributes}. */ @Experimental public class OperatorAttributesBuilder { private boolean isOutputIdempotent = false; /** @see OperatorAttributes#isOutputIdempotent */ public OperatorAttributesBuilder setOutputIdempotent(boolean isOutputIdempotent) {...} public OperatorAttributes build() {...} }
package org.apache.flink.streaming.api.operators; /** * OperatorAttributes element provides information about the operator that can be * used for performance optimization. */ @Experimental public class OperatorAttributes { /** * Whether the output results of this operator is idempotent. To be exact, idempotency here * means that it would yield the same final effect to collect a list of output results or to * collect only the last result (of each key, if the operator is applied on keyed stream) in the * list. * * <p>By setting this attribute to true, it offers an optimization hint that so long as a newer * output result is available, the older result (with the same key) can be discarded. This * allows Flink to buffer the outputs of the operator and periodically emit the latest ones ever * collected. The operator's throughput can be improved in this way by reducing the total number * of records emitted by the operator. */ public boolean isOutputIdempotent() {...} }
public interface Function extends java.io.Serializable { /** * Whether the output results of this function is idempotent. To be exact, idempotency here * means that it would yield the same final effect to collect a list of output results or to * collect only the last result (of each key, if the function is applied on keyed stream) in the * list. * * <p>By setting this attribute to true, it offers an optimization hint that so long as a newer * output result is available, the older result (with the same key) can be discarded. This * allows Flink to buffer the outputs of the function and periodically emit the latest ones ever * collected. The function's throughput can be improved in this way by reducing the total number * of records emitted by the function. */ default public boolean isOutputIdempotent() { return false; } }
3. Add the getOperatorAttributes() API to the StreamOperator and StreamOperatorFactory interfaces.
@PublicEvolving public interface StreamOperator<OUT> extends CheckpointListener, KeyContext, Serializable { ... @Experimental default OperatorAttributes getOperatorAttributes() { return new OperatorAttributesBuilder().build(); } }
@PublicEvolving public interface StreamOperatorFactory<OUT> extends Serializable { ... @Experimental default OperatorAttributes getOperatorAttributes() { return new OperatorAttributesBuilder().build(); } }
Proposed Changes
We propose to add a subclass of the Output interface as follows. When operators output results through this class, this class will buffer the latest results in the state backend until flush() is invoked. A similar subclass of Output will also be introduced in Table/SQL API, which also supports emitting the latest StreamRecord while providing additional support to the Insert/Delete/Update semantics.
/** * An {@link Output} that buffers the latest output results from an operator in the state backend. * * <ul> * <li>If the output result is a StreamRecord, the latest record (of each key, if the operator is * applied on keyed stream, same as below) would be buffered. This means if there has existed * a record (with the same key) in buffer, the record will be overridden by the newly arrived * record. * <li>If the output result is a Watermark, it would be directly emitted if the buffer is still * empty. Otherwise, it would be buffered until flush. Only the latest watermark would be * buffered. * <li>If the output result is a WatermarkStatus, it would be directly emitted if the buffer is * still empty. Otherwise, it would be buffered until flush. Only the latest watermark status * would be buffered. * <li>If the output result is a LatencyMarker, it would be directly emitted if the buffer is * still empty. Otherwise, it would be buffered until flush. All latency marker would be * buffered. * </ul> * * <p>When {@link #flush()} is invoked, all StreamElements in the buffer would be emitted to the * actual output, and the buffer would be cleared. If there is no watermark and stream records do * not contain timestamp, the buffered stream records and latency markers would be emitted in a * random order. Otherwise, elements would be emitted in the following order. * * <ul> * <li>All stream records whose timestamp is smaller than or equal to the watermark. * <li>The watermark status and watermark, in the order they were buffered. * <li>All stream records whose timestamp is larger than the watermark. * <li>All latency markers. * </ul> */ @Internal public class BufferOutput<T> implements Output<StreamRecord<T>> { private final Output<StreamRecord<T>> output; /** @param output The actual output to flush buffered records to. */ public BufferOutput( ... Output<StreamRecord<T>> output ) { this.output = output; } @Override public void collect(StreamRecord<T> record) {...} @Override public void close() {...} @Override public void emitWatermark(Watermark mark) {...} @Override public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {...} @Override public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> record) {...} @Override public void emitLatencyMarker(LatencyMarker latencyMarker) {...} public void flush() { for (...) { output.collect(...) } } }
If all of the following conditions are met, OperatorChain would decorate the original mainOutput variable with BufferedOutput before using it to create and setup the operator.
The operator is applied on a keyed stream, or the parallelism of the operator is 1.
execution.max-flush-interval is not null.
The operator reports isOutputIdempotent = true
If buffering is enabled according to the requirements above, StreamTask would order the OperatorChain to flush all BufferOutput when any of the following conditions is met during runtime.
All input channels are empty and there have been records given to the operator since the last flush.
execution.max-flush-interval has passed since the last flush.
Analysis
Performance benefits and overheads brought by this FLIP
If the optimizations proposed in this FLIP is enabled on an operator, its downstream operators would be able to reduce the computation, storage and network resources needed to process the intermediate output results generated by the operator. Suppose the operator used to generate N output records for each key between a max-flush-interval on average, the downstream operators now would only need 1/N of the original resources to handle the incoming data stream, or increase its throughput by N times with the same resource set.
On the other hand, this operator would increate the end-to-end latency of the Flink job, as the latest records would only be output when flush() is triggered. In the worst case where the latest record of a key is buffered right after a flush, the latency of the record would increase by max-flush-interval. Besides, each buffering operation would bring a read & write state access.
Built-in operators and functions that would be affected by this FLIP
With regard to the scenarios in which the optimizations proposed in this FLIP can come to effect, the group-reduce function seems to be the only one among Flink's built-in functions that can utilize these optimizations so far. This would be achieved by having the following operator/function report isOutputIdempotent=true.
StreamGroupedReduceOperator
GroupAggFunction
After this is applied, the following public API would be able to utilize these optimizations when they are applied on non-windowed stream.
DataStream API
KeyedStream#reduce
KeyedStream#sum
KeyedStream#min
KeyedStream#max
KeyedStream#minBy
KeyedStream#maxBy
Table API (similar for SQL)
BaseExpressions#distinct
BaseExpressions#sum
BaseExpressions#sum0
BaseExpressions#min
BaseExpressions#max
BaseExpressions#count
BaseExpressions#avg
BaseExpressions#firstValue
BaseExpressions#lastValue
BaseExpressions#listAgg
BaseExpressions#stddevPop
BaseExpressions#stddevSamp
BaseExpressions#varPop
BaseExpressions#varSamp
BaseExpressions#collect
Compatibility, Deprecation, and Migration Plan
The design proposed in this FLIP is backward compatible.
If this FLIP and FLIP-325 could be implemented, we propose to deprecate the table.exec.mini-batch.* configurations in Table/SQL programs. As we can see, table.exec.mini-batch brings two optimizations: reduced state backend access and reduced number of output records, and the design proposed in this FLIP and FLIP-325 can cover these optimizations with the following additional advantages:
Less heap memory usage. The operator only needs to store the aggregated value and one output record for each unique key, rather than the full list of the original elements.
Better cache hit rate. Since hot key does not have to be evicted from the cache periodically (due to mini-batch processing).
No need to increase time of the synchronous checkpoint stage.
Applicable to operators in DataStream API as well, instead of only Table/SQL API.
Thus the table.exec.mini-batch.* configurations should be deprecated and replaced by the newly proposed designs.
Future Work
When a Flink job contains multiple operators in series and more than one of them configure isOutputIdempotent=true, the end-to-end latency brought by each operator might be accumulated and the overall latency overhead could be max-flush-interval * num-operators. This problem is now alleviated by triggering flush when input channels are empty, as described in "Proposed Changes" section. In future we may need to introduce a control flow message that travels through the job graph in topological order and triggers flush on each StreamTask, and resolve the accumulation of the buffering latency with this message.