Skip to content

[FLINK-39437][table] Support interruptible timers in PTFs#27962

Open
fhueske wants to merge 3 commits intoapache:masterfrom
fhueske:fhueske-FLINK-39437-Support-interruptible-timers-in-PTFs
Open

[FLINK-39437][table] Support interruptible timers in PTFs#27962
fhueske wants to merge 3 commits intoapache:masterfrom
fhueske:fhueske-FLINK-39437-Support-interruptible-timers-in-PTFs

Conversation

@fhueske
Copy link
Copy Markdown
Contributor

@fhueske fhueske commented Apr 17, 2026

What is the purpose of the change

Enable interruptible timers for PTFs.

Brief change log

  • Override useInterruptibleTimers() in AbstractProcessTableOperator to return true, activating the MailboxWatermarkProcessor for PTF operators. This allows timer firing to be interrupted between mailbox iterations, improving throughput by not blocking mailbox processing during large timer batches.

  • Also reorder processWatermark() to call ingestWatermarkEvent() before super.processWatermark(), ensuring all timer callbacks (including those deferred across mailbox iterations) see a consistent watermark in the runner. This matches the behavior of WritableInternalTimeContext.currentWatermark(), which reads from the timer service and already sees the new watermark before any timer fires.

Verifying this change

  • added a semantic test to assert that timers on the same timestamp see the same watermark

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): no
  • The public API, i.e., is any changed class annotated with @Public(Evolving): no
  • The serializers: no
  • The runtime per-record code paths (performance sensitive): no
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
  • The S3 file system connector: no

Documentation

  • Does this pull request introduce a new feature? no
  • If yes, how is the feature documented? n/a

@flinkbot
Copy link
Copy Markdown
Collaborator

flinkbot commented Apr 17, 2026

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

@fhueske fhueske force-pushed the fhueske-FLINK-39437-Support-interruptible-timers-in-PTFs branch from 807ebc4 to 2c38910 Compare April 17, 2026 17:39
@fhueske fhueske requested review from Copilot and twalthr April 17, 2026 17:40
@fhueske fhueske marked this pull request as ready for review April 17, 2026 17:41
Copy link
Copy Markdown

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Enables interruptible timer processing for Process Table Function (PTF) operators and adjusts watermark handling so timer callbacks observe a consistent watermark when timers are fired incrementally via the mailbox.

Changes:

  • Enable interruptible timers for PTF operators by overriding useInterruptibleTimers(...) in AbstractProcessTableOperator.
  • Update processWatermark(...) ordering to ingest the new watermark into the PTF runner before delegating to the superclass (which advances the timer service and may fire timers).
  • Add a new planner semantic test program and test function intended to validate watermark consistency across multiple same-timestamp named timers.

Reviewed changes

Copilot reviewed 4 out of 4 changed files in this pull request and generated 2 comments.

File Description
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/process/AbstractProcessTableOperator.java Enables interruptible timers for PTF operators and reorders watermark ingestion vs. timer firing.
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ProcessTableFunctionTestUtils.java Adds a new PTF test function that registers multiple same-timestamp named timers.
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ProcessTableFunctionTestPrograms.java Adds a new TableTestProgram with expected outputs for consistent watermark observation in timer callbacks.
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/ProcessTableFunctionSemanticTests.java Registers the new semantic test program in the PTF semantic test suite.

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

}

@Override
public final boolean useInterruptibleTimers(ReadableConfig config) {
Copy link
Copy Markdown
Contributor

@spuru9 spuru9 Apr 17, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[nit] final could be dropped on useInterruptibleTimers? For consistency, all seven analogous overrides (WindowOperator, TimeIntervalJoin, BaseTemporalSortOperator, CepOperator, and others) have the similar function.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added the final on purpose to prevent this function from being overwritten by extending classes.
AbstractProcessTableOperator is an internal class. If it becomes necessary for a child class to overwrite the the method, we can always remove the final.
Until then, it documents that all child classes must be able to handle interruptible timers.

@github-actions github-actions bot added the community-reviewed PR has been reviewed by the community. label Apr 18, 2026
@fhueske fhueske force-pushed the fhueske-FLINK-39437-Support-interruptible-timers-in-PTFs branch from c12100e to 82c8ca6 Compare April 20, 2026 07:45
.build())
.runSql(
"INSERT INTO sink SELECT * FROM f(r => TABLE t PARTITION BY name, on_time => DESCRIPTOR(ts))")
.build();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Perhaps we need to create a dedicated HarnessTest to test whether interruptions are still correct

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've added ProcessSetTableOperatorInterruptibleTimersTest to assert the call order or timers.

fhueske added 3 commits April 20, 2026 16:12
Override `useInterruptibleTimers()` in `AbstractProcessTableOperator` to
return `true`, activating the `MailboxWatermarkProcessor` for PTF operators.
This allows timer firing to be interrupted between mailbox iterations,
improving throughput by not blocking mailbox processing during large timer
batches.

Also reorder `processWatermark()` to call `ingestCurrentWatermarkEvent()`
before `super.processWatermark()` to keep the runner's watermark consistent
with the timer service watermark, which is advanced before any timer fires.

Add a test (`PROCESS_CONSISTENT_WATERMARK_TIMERS`) that validates multiple
named timers registered at the same timestamp all see a consistent watermark
in their callbacks.
@fhueske fhueske force-pushed the fhueske-FLINK-39437-Support-interruptible-timers-in-PTFs branch from 3efca16 to 869e474 Compare April 20, 2026 17:16
@fhueske
Copy link
Copy Markdown
Contributor Author

fhueske commented Apr 21, 2026

Thanks for the reviews @spuru9 and @Au-Miner.
I've updated the PR.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-reviewed PR has been reviewed by the community.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants