[
https://issues.apache.org/jira/browse/FLINK-8740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16411511#comment-16411511
]
ASF GitHub Bot commented on FLINK-8740:
---------------------------------------
GitHub user tillrohrmann opened a pull request:
https://github.com/apache/flink/pull/5755
[FLINK-8740] [metrics] Create new JobManagerJobMetricGroup when creating a
new ExecutionGraph
## What is the purpose of the change
Closes the JobManagerJobMetricGroup when suspending the `JobManager`. This
allows to reregister the job metrics when the `JobManager` regains its
leadership.
## Verifying this change
- Tested manually
## 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: (no)
- 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)
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/tillrohrmann/flink jobLevelMetricsHA
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/5755.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #5755
----
commit ef2de68bcad3c4715941a1e61d0a460b4c609520
Author: Till Rohrmann <trohrmann@...>
Date: 2018-03-23T13:08:49Z
[FLINK-8740] [metrics] Create new JobManagerJobMetricGroup when creating a
new ExecutionGraph
commit 98a49cd032346a2b09641ee6d30baadf9a98855f
Author: Till Rohrmann <trohrmann@...>
Date: 2018-03-23T14:43:20Z
[hotfix] Create ExecutionGraph when JobMaster is started
The ExecutionGraph is not a final resource in the JobMaster. For example,
it is necessary
to create a new ExecutionGraph when rescaling the job or when the JobMaster
loses and
regains its leadership.
commit dcddfeb0b8883964505bed55d5b3730cae9abe60
Author: Till Rohrmann <trohrmann@...>
Date: 2018-03-23T14:46:29Z
[hotfix] Remove unused fields in JobMaster
----
> Job-level metrics lost during job re-submission in HA mode
> ----------------------------------------------------------
>
> Key: FLINK-8740
> URL: https://issues.apache.org/jira/browse/FLINK-8740
> Project: Flink
> Issue Type: Bug
> Components: JobManager
> Affects Versions: 1.4.0
> Reporter: Joshua DeWald
> Assignee: Till Rohrmann
> Priority: Blocker
> Fix For: 1.5.0
>
>
> When Flink is running in High Availability and a leader re-election occurs to
> the same job manager, the job is unable to register the job-level metrics due
> to a name collision.
> This may occur even if a different Job Manager is elected, but as it is a
> local JobManagerMetricsGroup which spits out the error, that is unlikely the
> case.
>
> *Expected Behavior*
> When a job is forced to re-submit due to Job Manager re-election, job-level
> metrics should be available in the new instance of the job (uptime,
> checkpoints size, checkpoint duration, etc)
> *Actual Behavior*
> When job gets re-submitted, it is unable to register job-level metrics due to
> collision in the JobManagerMetricGroup, which leads to situation where even
> though job is running the metrics around checkpoints and uptime are not
> available
> *Steps to reproduce*
> # Start up Flink in HA mode using ZooKeeper, single node is fine
> # Submit a job to the cluster
> # Stop and restart ZooKeeper
> # In Job Manager logs you will see the following errors:
> #
> {noformat}
> 79043 2018-02-19 21:58:15,928 WARN org.apache.flink.metrics.MetricGroup -
> Name collision: Group already contains a Metric with the name
> 'totalNumberOfCheckpoints'. Metric will not be reported....
> 79044 2018-02-19 21:58:15,928 WARN org.apache.flink.metrics.MetricGroup -
> Name collision: Group already contains a Metric with the name
> 'numberOfInProgressCheckpoints'. Metric will not be reported....
> 79045 2018-02-19 21:58:15,928 WARN org.apache.flink.metrics.MetricGroup -
> Name collision: Group already contains a Metric with the name
> 'numberOfCompletedCheckpoints'. Metric will not be reported....{noformat}
> *Proposed Solution*
> I suspect that there may be other related issues than just the metrics, but a
> code change that seems to fix the issue is that, during recovery, to remove
> the existing registered Job Metrics:
> {code:java}
> if (isRecovery) {
> log.info(s"Removing metrics for $jobId, new will be added during recover")
> jobManagerMetricGroup.removeJob(jobId)
> }{code}
> I'd be happy to submit this in a PR if that is acceptable to open up the
> discussion, but I am not sure the consequences of not closing the previous
> JMMG or perhaps simply not re-registering job-level metrics during recovery.
> Doing this would seem to entail informing lower levels about the recovery.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)