kkdoon opened a new issue, #28554:
URL: https://github.com/apache/beam/issues/28554
### What happened?
**Issue:**
Flink pipeline does not get drained when RequiresStableInput annotation is
used. Stack-trace when drain is triggered:
`Caused by: java.lang.RuntimeException: There are still watermark holds left
when terminating operator
KVStoreTransform/GroupIntoBatches/ParDo(GroupIntoBatches)/ParMultiDo(GroupIntoBatches)
->
PersistenceTransform/WriteSpendStateToDb/ParMultiDo(WriteSpendStateDbStreaming)
(1/1)#0 Watermark held 1695147850921
at
org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.flushData(DoFnOperator.java:631)
at
org.apache.beam.runners.flink.translation.wrappers.streaming.AbstractStreamOperatorCompat.finish(AbstractStreamOperatorCompat.java:67)
at
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50)
at
org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.finishOperator(StreamOperatorWrapper.java:239)
at
org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.lambda$deferFinishOperatorToMailbox$3(StreamOperatorWrapper.java:213)
at
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50)
at
org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90)
at
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl.tryYield(MailboxExecutorImpl.java:97)
at
org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.quiesceTimeServiceAndFinishOperator(StreamOperatorWrapper.java:186)
... 13 more
`
**Cause:**
When drain is triggered, MAX_WATERMARK gets emitted before the last
checkpoint barrier. This helps in triggering all the registered event-time
timers and flushing out any state. Therefore, the expectation is that when
[flushData](https://github.com/apache/beam/blob/79d0a8d11095522a036a6b3007f5fed4f6f46b3b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java#L608)
is finally invoked in DoFnOperator, all the event timers should be fired and
the watermark should proceed.
However, when RequiresStableInput annotation is used, the behavior is to
process the DoFn **after** the checkpoint operation is complete. Since, flush
is invoked when the final checkpoint/savepoint operation is in progress, the
watermark is held by the DoFn with the RequiresStableInput annotation, sine
data is unprocessed and is waiting for the checkpoint to complete.
**Potential Solutions:**
1. Skip
[this](https://github.com/apache/beam/blob/79d0a8d11095522a036a6b3007f5fed4f6f46b3b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java#L629)
check in case the DoFn has RequiresStableInput set, since we know that all the
final pending data will be processed after the final savepoint operation
completes.
2. Trigger `bufferingDoFnRunner.checkpointCompleted` within `flushData` to
ensure that checkpointing is completed and all pending processing of buffered
data is done within flushData itself.
### Issue Priority
Priority: 2 (default / most bugs should be filed as P2)
### Issue Components
- [ ] Component: Python SDK
- [ ] Component: Java SDK
- [ ] Component: Go SDK
- [ ] Component: Typescript SDK
- [ ] Component: IO connector
- [ ] Component: Beam examples
- [ ] Component: Beam playground
- [ ] Component: Beam katas
- [ ] Component: Website
- [ ] Component: Spark Runner
- [X] Component: Flink Runner
- [ ] Component: Samza Runner
- [ ] Component: Twister2 Runner
- [ ] Component: Hazelcast Jet Runner
- [ ] Component: Google Cloud Dataflow Runner
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]