[ 
https://issues.apache.org/jira/browse/FLINK-21132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17281563#comment-17281563
 ] 

Kezhu Wang commented on FLINK-21132:
------------------------------------

{quote}I saw a hole in notifyCheckpointAbortAsync where there is no 
resetSynchronousSavepointId. I will verify it using test case. – from me
{quote}
[~roman_khachatryan] [~pnowojski] This does not hold as 
{{CheckpointCoordinator}} will fail the 
job({{CheckpointFailureManager.handleSynchronousSavepointFailure}}) after 
synchronous-savepoint(eg. stop-with-savepoint) failure. But the causal chain is 
a bit long, so I suggest to {{resetSynchronousSavepointId}} always. 
[pr#14815|https://github.com/apache/flink/pull/14815/files#diff-0c5fe245445b932fa83fdaf5c4802dbe671b73e8d76f39058c6eaaaffd9639faR1080]
 already have this, so there is no more concern from my side.
{quote}Data flow and events between TaskManagers are using the same channels. 
RPCs from JobManager to the TaskManagers are using a different channel, but 
they won't be processed if the Task's thread (mailbox) is blocked.
{quote}
I probably use wrong words. I use "data flow" to represent {{StreamElement}} 
and {{RuntimeEvent}}(including {{EndOfPartitionEvent}}, {{CheckpointBarrier}}, 
etc.) and "gateway event" to represent 
{{TaskExecutorGateway.triggerCheckpoint/confirmCheckpoint}} which are rpc 
methods. I think we are aligned on this.

Checkpoint completion and cancellation are delivered using 
{{TaskMailbox.MAX_PRIORITY}}, so they will be processed along with other 
control-mails in {{runSynchronousSavepointMailboxLoop}}. I don't think either 
approach change this part of code, so if there is deadlock after, it probably 
exists before.
{quote}To avoid deadlocks one would have to carefully thought this through.
{quote}
[~roman_khachatryan] [~pnowojski] Is it worth another dedicated issue to 
discuss this no end-of-partition style stop-with-savepoint ? I see values of 
this approach:
 # It is applicable to mailbox model. FLINK-21133 could be fixed without touch 
FLIP-27 source code. FLINK-21133 may be relative complicated due to source 
chaining with multiple input stream. Though, I think this could also be 
achieved in end-of-partition style by rearranging {{StreamTask.finishTask}} to 
path from {{StreamTask.triggerCheckpointAsync}}, but this will need more 
bookkeeping.
 # Clean code ? We don't need end-of-input detection during 
stop-with-savepoint. It could be tangled/confused in evaluation data-flow 
during stop-with-savepoint in future, eg. why end-of-input and then stripping 
of them.

> BoundedOneInput.endInput is called when taking synchronous savepoint
> --------------------------------------------------------------------
>
>                 Key: FLINK-21132
>                 URL: https://issues.apache.org/jira/browse/FLINK-21132
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Task
>    Affects Versions: 1.10.2, 1.10.3, 1.11.3, 1.12.1
>            Reporter: Kezhu Wang
>            Assignee: Roman Khachatryan
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.11.4, 1.12.2, 1.13.0
>
>
> [~elkhand](?) reported on project iceberg that {{BoundedOneInput.endInput}} 
> was 
> [called|https://github.com/apache/iceberg/issues/2033#issuecomment-765864038] 
> when [stopping job with 
> savepoint|https://github.com/apache/iceberg/issues/2033#issuecomment-765557995].
> I think it is a bug of Flink and was introduced in FLINK-14230. The 
> [changes|https://github.com/apache/flink/pull/9854/files#diff-0c5fe245445b932fa83fdaf5c4802dbe671b73e8d76f39058c6eaaaffd9639faL577]
>  rely on {{StreamTask.afterInvoke}} and {{OperatorChain.closeOperators}} will 
> only be invoked after *end of input*. But that is not true long before after 
> [FLIP-34: Terminate/Suspend Job with 
> Savepoint|https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=103090212].
>  Task could enter state called 
> [*finished*|https://github.com/apache/flink/blob/3a8e06cd16480eacbbf0c10f36b8c79a6f741814/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L467]
>  after synchronous savepoint, that is an expected job suspension and stopping.
> [~sunhaibotb] [~pnowojski] [~roman_khachatryan] Could you help confirm this ?
> For full context, see 
> [apache/iceberg#2033|https://github.com/apache/iceberg/issues/2033]. I have 
> pushed branch 
> [synchronous-savepoint-conflict-with-bounded-end-input-case|https://github.com/kezhuw/flink/commits/synchronous-savepoint-conflict-with-bounded-end-input-case]
>  in my repository. Test case 
> {{SavepointITCase.testStopSavepointWithBoundedInput}} failed due to 
> {{BoundedOneInput.endInput}} called.
> I am also aware of [FLIP-147: Support Checkpoints After Tasks 
> Finished|https://cwiki.apache.org/confluence/x/mw-ZCQ], maybe the three 
> should align on what *finished* means exactly. [~kkl0u] [~chesnay] 
> [~gaoyunhaii]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to