morhidi commented on code in PR #311:
URL: 
https://github.com/apache/flink-kubernetes-operator/pull/311#discussion_r919663834


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/lifecycle/LifecycleMetrics.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.kubernetes.operator.metrics.lifecycle;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
+import org.apache.flink.kubernetes.operator.crd.AbstractFlinkResource;
+import 
org.apache.flink.kubernetes.operator.metrics.KubernetesOperatorMetricGroup;
+import org.apache.flink.metrics.Histogram;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.metrics.DescriptiveStatisticsHistogram;
+
+import lombok.RequiredArgsConstructor;
+import lombok.ToString;
+
+import java.time.Clock;
+import java.time.Instant;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
+
+import static 
org.apache.flink.kubernetes.operator.metrics.lifecycle.ResourceLifecycleState.CREATED;
+import static 
org.apache.flink.kubernetes.operator.metrics.lifecycle.ResourceLifecycleState.DEPLOYED;
+import static 
org.apache.flink.kubernetes.operator.metrics.lifecycle.ResourceLifecycleState.ROLLED_BACK;
+import static 
org.apache.flink.kubernetes.operator.metrics.lifecycle.ResourceLifecycleState.ROLLING_BACK;
+import static 
org.apache.flink.kubernetes.operator.metrics.lifecycle.ResourceLifecycleState.STABLE;
+import static 
org.apache.flink.kubernetes.operator.metrics.lifecycle.ResourceLifecycleState.SUSPENDED;
+import static 
org.apache.flink.kubernetes.operator.metrics.lifecycle.ResourceLifecycleState.UPGRADING;
+
+/**
+ * Utility for tracking resource lifecycle metrics globally and per namespace.
+ *
+ * @param <CR> Flink resource type.
+ */
+public class LifecycleMetrics<CR extends AbstractFlinkResource<?, ?>> {
+
+    private static final String TRANSITION_FIRST_DEPLOYMENT = 
"FirstDeployment";
+    private static final String TRANSITION_RESUME = "Resume";
+    private static final String TRANSITION_UPGRADE = "Upgrade";
+    private static final String TRANSITION_SUSPEND = "Suspend";
+    private static final String TRANSITION_SUBMISSION = "Submission";
+    private static final String TRANSITION_STABILIZATION = "Stabilization";
+    private static final String TRANSITION_ROLLBACK = "Rollback";
+
+    public static final List<Transition> TRACKED_TRANSITIONS = 
getTrackedTransitions();
+
+    private final Map<Tuple2<String, String>, ResourceLifecycleMetricTracker> 
lifecycleTrackers =
+            new ConcurrentHashMap<>();
+    private final Set<String> namespaces = Collections.newSetFromMap(new 
ConcurrentHashMap<>());
+
+    private final FlinkConfigManager configManager;
+    private final Clock clock;
+    private final KubernetesOperatorMetricGroup operatorMetricGroup;
+
+    private Map<String, Tuple2<Histogram, Map<String, Histogram>>> 
transitionMetrics;
+    private Function<MetricGroup, MetricGroup> metricGroupFunction;
+
+    public LifecycleMetrics(
+            FlinkConfigManager configManager,
+            Clock clock,
+            KubernetesOperatorMetricGroup operatorMetricGroup) {
+        this.configManager = configManager;
+        this.clock = clock;
+        this.operatorMetricGroup = operatorMetricGroup;
+    }
+
+    public void onUpdate(CR cr) {
+        
getLifecycleMetricTracker(cr).onUpdate(cr.getStatus().getLifecycleState(), 
clock.instant());
+    }
+
+    public void onRemove(CR cr) {
+        lifecycleTrackers.remove(
+                Tuple2.of(cr.getMetadata().getNamespace(), 
cr.getMetadata().getName()));
+    }
+
+    private ResourceLifecycleMetricTracker getLifecycleMetricTracker(CR cr) {
+        init(cr);
+        createNamespaceStateCountIfMissing(cr.getMetadata().getNamespace());
+        return lifecycleTrackers.computeIfAbsent(
+                Tuple2.of(cr.getMetadata().getNamespace(), 
cr.getMetadata().getName()),
+                k -> {
+                    var initialState = cr.getStatus().getLifecycleState();
+                    var time =
+                            initialState == CREATED
+                                    ? 
Instant.parse(cr.getMetadata().getCreationTimestamp())
+                                    : clock.instant();
+                    return new ResourceLifecycleMetricTracker(
+                            initialState, time, getTransitionHistograms(cr));
+                });
+    }
+
+    private void createNamespaceStateCountIfMissing(String namespace) {
+        if (!namespaces.add(namespace)) {
+            return;
+        }
+
+        MetricGroup lifecycleGroup =
+                metricGroupFunction.apply(
+                        operatorMetricGroup.createResourceNamespaceGroup(
+                                configManager.getDefaultConfig(), namespace));
+        for (ResourceLifecycleState state : ResourceLifecycleState.values()) {
+            lifecycleGroup
+                    .addGroup("State")
+                    .addGroup(state.name())
+                    .gauge(
+                            "Count",
+                            () ->
+                                    lifecycleTrackers.values().stream()
+                                            
.map(ResourceLifecycleMetricTracker::getCurrentState)
+                                            .filter(s -> s == state)
+                                            .count());
+        }
+    }
+
+    private synchronized void init(CR cr) {
+        if (transitionMetrics != null) {
+            return;
+        }
+        this.metricGroupFunction =
+                mg -> 
mg.addGroup(cr.getClass().getSimpleName()).addGroup("Lifecycle");
+
+        this.transitionMetrics = new ConcurrentHashMap<>();
+        TRACKED_TRANSITIONS.forEach(
+                t ->
+                        transitionMetrics.computeIfAbsent(
+                                t.metricName,
+                                name ->
+                                        Tuple2.of(
+                                                createTransitionHistogram(
+                                                        name, 
operatorMetricGroup),
+                                                new ConcurrentHashMap<>())));
+    }
+
+    private Map<String, List<Histogram>> getTransitionHistograms(CR cr) {
+        var histos = new HashMap<String, List<Histogram>>();
+        transitionMetrics.forEach(
+                (metricName, t) -> {
+                    histos.put(
+                            metricName,
+                            List.of(
+                                    t.f0,
+                                    t.f1.computeIfAbsent(
+                                            cr.getMetadata().getNamespace(),
+                                            ns ->
+                                                    createTransitionHistogram(
+                                                            metricName,
+                                                            operatorMetricGroup
+                                                                    
.createResourceNamespaceGroup(
+                                                                            
configManager
+                                                                               
     .getDefaultConfig(),
+                                                                            
ns)))));
+                });
+        return histos;
+    }
+
+    private Histogram createTransitionHistogram(String metricName, MetricGroup 
group) {
+        return metricGroupFunction
+                .apply(group)
+                .addGroup("Transition")
+                .addGroup(metricName)
+                .histogram(
+                        "TimeSeconds",
+                        new DescriptiveStatisticsHistogram(
+                                configManager
+                                        .getOperatorConfiguration()
+                                        .getMetricsHistogramSampleSize()));
+    }
+
+    private static List<Transition> getTrackedTransitions() {
+        return List.of(
+                new Transition(CREATED, DEPLOYED, false, 
TRANSITION_FIRST_DEPLOYMENT),
+                new Transition(SUSPENDED, STABLE, true, TRANSITION_RESUME),
+                new Transition(STABLE, STABLE, true, TRANSITION_UPGRADE),
+                new Transition(DEPLOYED, UPGRADING, true, TRANSITION_SUSPEND),
+                new Transition(STABLE, UPGRADING, true, TRANSITION_SUSPEND),
+                new Transition(ROLLED_BACK, UPGRADING, true, 
TRANSITION_SUSPEND),
+                new Transition(DEPLOYED, SUSPENDED, true, TRANSITION_SUSPEND),
+                new Transition(STABLE, SUSPENDED, true, TRANSITION_SUSPEND),
+                new Transition(ROLLED_BACK, SUSPENDED, true, 
TRANSITION_SUSPEND),
+                new Transition(DEPLOYED, STABLE, false, 
TRANSITION_STABILIZATION),
+                new Transition(DEPLOYED, ROLLED_BACK, false, 
TRANSITION_ROLLBACK),
+                new Transition(UPGRADING, DEPLOYED, true, 
TRANSITION_SUBMISSION),
+                new Transition(ROLLING_BACK, ROLLED_BACK, true, 
TRANSITION_SUBMISSION));
+    }
+
+    /**
+     * Pojo for encapsulating state transitions and whether we should measure 
time from the
+     * beginning of from or since the last update.
+     */
+    @ToString
+    @RequiredArgsConstructor
+    protected static class Transition {

Review Comment:
   Have you considered defining `Transition` as enum?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to