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

Matthias Pohl edited comment on FLINK-27354 at 4/25/22 1:28 PM:
----------------------------------------------------------------

This issue continues to appear because of the retry mechanism using the 
{{AkkaRpcService}}'s {{internalScheduledExecutor}} which is a 
{{ActorSystemScheduledExecutorAdapter}} (see 
[AkkaRpcService:158|https://github.com/apache/flink/blob/aa06fbfb78ab9de8234e5be63dd66847bb319fdb/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java#L158]).
 This one uses the {{ActorSystem}}'s dispatcher threadpool which will be 
present until the {{RpcSystem}} shuts down (i.e. at the end of the 
ClusterEntrypoint's lifecycle).


was (Author: mapohl):
This issue continues to appear because of the retry mechanism using the 
{{AkkaRpcService}}'s {{internalScheduledExecutor}} which is a 
{{ActorSystemScheduledExecutorAdapter}} (see 
[AkkaRpcService:158|https://github.com/apache/flink/blob/aa06fbfb78ab9de8234e5be63dd66847bb319fdb/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java#L158]).
 This one uses the {{ActorSystem}}'s dispatcher threadpool which will be 
present until the {{RpcSystem}} shuts down (i.e. at the end of the Flink 
cluster's lifecycle).

> JobMaster still processes requests while terminating
> ----------------------------------------------------
>
>                 Key: FLINK-27354
>                 URL: https://issues.apache.org/jira/browse/FLINK-27354
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.15.0, 1.13.6, 1.14.4
>            Reporter: Matthias Pohl
>            Priority: Major
>         Attachments: flink-logs.zip
>
>
> An issue was reported in the [user 
> ML|https://lists.apache.org/thread/5pm3crntmb1hl17h4txnlhjz34clghrg] about 
> the JobMaster trying to reconnect to the ResourceManager during shutdown.
> The JobMaster is disconnecting from the ResourceManager during shutdown (see 
> [JobMaster:1182|https://github.com/apache/flink/blob/da532423487e0534b5fe61f5a02366833f76193a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java#L1182]).
>  This triggers the deregistration of the job in the {{ResourceManager}}. The 
> RM responses asynchronously at the end of this deregistration through 
> {{disconnectResourceManager}} (see 
> [ResourceManager:993|https://github.com/apache/flink/blob/da532423487e0534b5fe61f5a02366833f76193a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java#L993])
>  which will trigger a reconnect on the {{JobMaster}}'s side (see 
> [JobMaster::disconnectResourceManager|https://github.com/apache/flink/blob/da532423487e0534b5fe61f5a02366833f76193a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java#L789])
>  if it's still around because the {{resourceManagerAddress}} (used in 
> {{isConnectingToResourceManager}}) is not cleared. This would only happen 
> during a RM leader change.
> The {{disconnectResourceManager}} will be ignored if the {{JobMaster}} is 
> gone already.
> We should add a guard in some way to {{JobMaster}} to avoid reconnecting to 
> other components during shutdown. This might not only include the 
> ResourceManager connection but might also affect other parts of the 
> {{JobMaster}} API.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to