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

ASF GitHub Bot commented on FLINK-8740:
---------------------------------------

Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5755#discussion_r177090323
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobManagerJobMetricGroupFactory.java
 ---
    @@ -0,0 +1,46 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.jobmaster.factories;
    +
    +import org.apache.flink.api.common.JobID;
    +import org.apache.flink.runtime.jobgraph.JobGraph;
    +import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup;
    +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
    +
    +import javax.annotation.Nonnull;
    +
    +/**
    + * Implementation of {@link JobManagerJobMetricGroupFactory} which makes 
sure that there is always
    + * at most one {@link JobManagerJobMetricGroup} for a given {@link JobID} 
registered.
    + */
    +public class DefaultJobManagerJobMetricGroupFactory implements 
JobManagerJobMetricGroupFactory {
    +
    +   private final JobManagerMetricGroup jobManagerMetricGroup;
    +
    +   public DefaultJobManagerJobMetricGroupFactory(@Nonnull 
JobManagerMetricGroup jobManagerMetricGroup) {
    +           this.jobManagerMetricGroup = jobManagerMetricGroup;
    +   }
    +
    +   @Override
    +   public JobManagerJobMetricGroup create(@Nonnull JobGraph jobGraph) {
    +           jobManagerMetricGroup.removeJob(jobGraph.getJobID());
    --- End diff --
    
    This should only be necessary if the group wasn't properly cleaned up 
previously by the JobMaster, correct? If so I suggest for debugging purposes to 
modify `removeJob` to return a boolean indicating whether something was 
actually deleted and log that as a warning.


> 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)

Reply via email to