[
https://issues.apache.org/jira/browse/FLINK-10255?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16609200#comment-16609200
]
ASF GitHub Bot commented on FLINK-10255:
----------------------------------------
tillrohrmann opened a new pull request #6678: [FLINK-10255] Only react to
onAddedJobGraph signal when being leader
URL: https://github.com/apache/flink/pull/6678
## What is the purpose of the change
The Dispatcher should only react to the onAddedJobGraph signal if it is the
leader.
In all other cases the signal should be ignored since the jobs will be
recovered once
the Dispatcher becomes the leader.
In order to still support non-blocking job recoveries, this commit
serializes all
recovery operations by introducing a recoveryOperation future which first
needs to
complete before a subsequent operation is started. That way we can avoid
race conditions
between granting and revoking leadership as well as the onAddedJobGraph
signals. This is
important since we can only lock each JobGraph once and, thus, need to make
sure that
we don't release a lock of a properly recovered job in a concurrent
operation.
cc @GJL
## Brief change log
- Only react to `SubmittedJobGraphListener#onAddedJobGraph` when being the
leader
- Serialize recovery operations by introducing a `recoveryOperation` future
in order to avoid wrong unlocking of guarded resources
## Verifying this change
- Added `ZooKeeperHADispatcherTest#testStandbyDispatcherJobExecution` and
`ZooKeeperHADispatcherTest#testStandbyDispatcherJobRecovery`
## 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, Yarn/Mesos, ZooKeeper: (yes)
- The S3 file system connector: (no)
## Documentation
- Does this pull request introduce a new feature? (no)
- If yes, how is the feature documented? (not applicable)
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> Standby Dispatcher locks submitted JobGraphs
> --------------------------------------------
>
> Key: FLINK-10255
> URL: https://issues.apache.org/jira/browse/FLINK-10255
> Project: Flink
> Issue Type: Bug
> Components: Distributed Coordination
> Affects Versions: 1.5.3, 1.6.0, 1.7.0
> Reporter: Till Rohrmann
> Assignee: Till Rohrmann
> Priority: Blocker
> Labels: pull-request-available
> Fix For: 1.6.1, 1.7.0, 1.5.4
>
>
> Currently, standby {{Dispatchers}} lock submitted {{JobGraphs}} which are
> added to the {{SubmittedJobGraphStore}} if HA mode is enabled. Locking the
> {{JobGraphs}} can prevent their cleanup leaving the system in an inconsistent
> state.
> The problem is that we recover in the
> {{SubmittedJobGraphListener#onAddedJobGraph}} callback which is also called
> if don't have the leadership the newly added {{JobGraph}}. Recovering the
> {{JobGraph}} currently locks the {{JobGraph}}. In case that the
> {{Dispatcher}} is not the leader, then we won't start that job after its
> recovery. However, we also don't release the {{JobGraph}} leaving it locked.
> There are two possible solutions to the problem. Either we check whether we
> are the leader before recovering jobs or we say that recovering jobs does not
> lock them. Only if we can submit the recovered job we lock them. The latter
> approach has the advantage that it follows a quite similar code path as an
> initial job submission. Moreover, jobs are currently also recovered at other
> places. In all these places we currently would need to release the
> {{JobGraphs}} if we cannot submit the recovered {{JobGraph}} (e.g.
> {{Dispatcher#grantLeadership}}).
> An extension of the first solution could be to stop the
> {{SubmittedJobGraphStore}} while the {{Dispatcher}} is not the leader. Then
> we would have to make sure that no concurrent callback from the
> {{SubmittedJobGraphStore#SubmittedJobGraphListener}} can be executed after
> revoking leadership from the {{Dispatcher}}.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)