Status

Discussion threadhttps://lists.apache.org/thread.html/r8357d64b9cfdf5a233c53a20d9ac62b75c07c925ce2c43e162f1e39c%40%3Cdev.flink.apache.org%3E
Vote thread
JIRA

Unable to render Jira issues macro, execution error.

Release1.14


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

Motivation

StreamStatus is currently defined as follows:

A Stream Status element informs stream tasks whether or not they should continue to expect records and watermarks from the input stream that sent them. There are 2 kinds of status, namely IDLE and ACTIVE. Stream Status elements are generated at the sources, and may be propagated through the tasks of the topology. They directly infer the current status of the emitting task; a SourceStreamTask or StreamTask emits a IDLE if it will temporarily halt to emit any records or watermarks (i.e. is idle), and emits a ACTIVE once it resumes to do so (i.e. is active).

However that definition is quite problematic. Operators with delayed responses, such as AsyncIO, windows, or time-based process functions, might output elements when they are actually idle. So by that definition, they would need to switch from idle to active, emit an element, and go back to idle. It's not entirely clear when they would go back to idle (after each element or after some time?). Furthermore, under heavy backpressure, these additional StreamStatuses delay progress quite a bit.

At this point, it makes sense to revise the original intent of StreamStatus: being able to advance the watermark without data. Indeed, there is no reason to expand the definition towards records; we could simply limit the definition to watermarks as this is the only use case for StreamStatus in the code base currently. Moreover, that's exactly how it's implemented in Flink so far. So, in this FLIP we propose to adjust the definition to reality and act on that.

Idleness

A related questions is: What exactly is idleness? And how does a source instance become idle?

Idleness is a means to make progress when a source reader temporarily does not emit watermarks because it has no data. We can distinguish three cases where no data can be emitted at the source:

1. Static assignment + too few splits

The source uses a static assignment of splits to readers and there are more readers than sources.

In this case, readers will be closed and emit MAX_WATERMARK to unblock watermark propagation indefinitely. At this point, idleness is not needed.

2. Static assignment + temporary no data

The source uses a static assignment of splits to readers, every reader obtains at least one split, but some readers have no records in their splits temporarily (e.g. a Kafka partition became stale).

In this case, a user can use WatermarkStrategy#withIdleness to let the watermark advance after a specific amount of time. However, when a partition becomes active again, there is a risk of marking records as late that wouldn't have been late without idleness.

Here, it's important that a user is aware of the realistic potential for late records and can adjust the job accordingly to work with retractions. As part of this FLIP, we should improve the documentation of withIdleness to give the user all necessary tools to deal with late records.

3. Dynamic assignment + temporary no split

The last case is that a source uses dynamic assignment and a reader temporarily does not a have a split assigned.

In this case, a reader cannot be closed and thus holds back the watermark. Unfortunately, idleness on reader level would introduce systematic errors as in the previous case. However, in contrast to the previous case, the user may not be aware of such imprecision.

Hence, Flink needs a mechanism to sync the watermarks between source readers and source coordinator, such that no data may be potentially lost. For example, in systems that support watermarks natively, the coordinator extracts that global watermark and forwards it to the readers. Another example would be a file source that reads from time-bucketed folders where the coordinator can infer a global minimum watermark from the folder structure. This mechanism will be covered in a separate, later FLIP and is beyond the scope of this FLIP. Note that a user can still add idleness on top of the global minimum watermark to speed up processing.

Implications

With this FLIP, we need to adjust the APIs around idleness, such that it's clear that only users should be able to make a source instance go idle. We need to emphasize the potential of late records and subsequent data loss if no counter-measures are taken.

Public Interfaces

  • Output, Input to rename StreamStatus

  • WatermarkOutput

Proposed Changes

The biggest change includes StreamStatus albeit it's an internal class. We propose to adjust the definition (JavaDoc) to exclude records and rename the class to WatermarkStatus and move it to the watermarkstatus package. Finally, we need to rename emitStreamStatus and processStreamStatus in a few public evolving interfaces (Input, Output).

We should also update the definition of StreamStatus#IDLE to make sure that this a) may introduce late records and b) should be user-controlled. With that change, we also remove all code that automatically let's sources go idle without user-specified idleness.

Since sources may offer users the ability to go IDLE beyond the WatermarkStrategy#withIdleness, we propose to also add WatermarkOutput#markActive to quickly go active again without the need to explicitly emit a watermark first. This addition allows to reduce the number of late records when a source knows that records are available until it processed a batch and actually emit a watermark.

Compatibility, Deprecation, and Migration Plan

  • All changes are on internal classes or on public methods that been added only in master. So we can assume that this FLIP causes no visible change.

Test Plan

The change of StreamStatus  reflects the current implementation and is thus already covered by tests. We might need to add additional unit/integration tests if we discover paths where we automatically switch to IDLE (a recent change on 1.14-SNAPSHOT does it).

Changes to WatermarkOutput require a few additional unit tests. It's already covered in integration tests.

Rejected Alternatives

None yet.

1 Comment

  1. Thanks for creating this FLIP Arvid Heise. I think it looks good and could be published.