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

Wei-Che Wei edited comment on FLINK-4714 at 3/19/18 2:46 PM:
-------------------------------------------------------------

Hi [~till.rohrmann]
Since FLINK-5982 has been merged, I would like to go back to this work.
As we discussed before, I prepared to move those initialization part before 
{{isRunning = true;}} to the {{StreamTask}}'s constructor.

However, I found that it didn't work as I thought.
First, I can't move {{StreamTask#initializeState()}} to the constructor, 
because it would fail {{InterruptSensitiveRestoreTest}}.
I think the reason is due to that the constructor was blocked and had not been 
created, so that {{TaskCanceler}} couldn't stop it by calling 
{{StreamTask#cancel()}}.
Second, {{StreamTask#init()}} can't be moved to constructor as well, because 
{{spotbugs-maven-plugin}} will make compilation failed due to 
{{UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR}}.

I have no idea how I could make all of them work in the {{StreamTask}}'s 
constructor, so I will prefer to turn into the original discussion: adding a 
{{initialization}} or {{open}} method.
What do you think? Please advise me if I made any mistake. Thank you.

Here is my [repo|https://github.com/tony810430/flink/tree/FLINK-4714-WIP]. 
Currently, I made {{initializeState()}} and {{init()}} remain in {{invoke()}} 
to pass travis ci.


was (Author: tonywei):
Hi [~till.rohrmann]
Since FLINK-5982 has been merged, I would like to go back to this work.
As we discussed before, I prepared to move those initialization part before 
{{isRunning = true;}} to the {{StreamTask}}'s constructor.

However, I found that it didn't work as I thought.
First, I can't move {{StreamTask#initializeState()}} to the constructor, 
because it would fail {{InterruptSensitiveRestoreTest}}.
I think the reason is due to that the constructor was blocked and had not been 
created, so that {{TaskCanceler}} couldn't stop it by calling 
{{StreamTask#cancel()}}.
Second, {{StreamTask#init()}} can't be moved to constructor as well, because 
{{spotbugs-maven-plugin}} will make compilation failed due to 
{{UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR}}.

I have no idea how I could make all of them work in the {{StreamTask}}'s 
constructor, so I will prefer to turn into the original discussion: adding a 
{{initialization}} or {{open}} method.
What do you think? Please advise me if I made any mistake. Thank you.

This is my [repo|https://github.com/tony810430/flink/tree/FLINK-4714-WIP].

> Set task state to RUNNING after state has been restored
> -------------------------------------------------------
>
>                 Key: FLINK-4714
>                 URL: https://issues.apache.org/jira/browse/FLINK-4714
>             Project: Flink
>          Issue Type: Improvement
>          Components: Distributed Coordination, State Backends, Checkpointing
>    Affects Versions: 1.2.0
>            Reporter: Till Rohrmann
>            Assignee: Wei-Che Wei
>            Priority: Major
>
> The task state is set to {{RUNNING}} as soon as the {{Task}} is executed. 
> That, however, happens before the state of the {{StreamTask}} invokable has 
> been restored. As a result, the {{CheckpointCoordinator}} starts to trigger 
> checkpoints even though the {{StreamTask}} is not ready.
> In order to avoid aborting checkpoints and properly start it, we should 
> switch the task state to {{RUNNING}} after the state has been restored.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to