This is an automated email from the ASF dual-hosted git repository.

mxm pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/flink-kubernetes-operator.git

commit 39d37eb6ba971b67078b79214e409c3e91f7114c
Author: Maximilian Michels <m...@apache.org>
AuthorDate: Thu May 4 19:24:17 2023 +0200

    [FLINK-32005] Add a per-deployment error metric to signal about potential 
issues
    
    If any of the autoscaled deployment produce errors they are only visible in 
the logs or in the k8s events. Additionally,
    it would be good to have metrics to detect any potential issues.
---
 .../operator/autoscaler/JobAutoScalerImpl.java     | 19 ++++--
 .../operator/autoscaler/JobAutoScalerImplTest.java | 73 ++++++++++++++++++++++
 2 files changed, 86 insertions(+), 6 deletions(-)

diff --git 
a/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/JobAutoScalerImpl.java
 
b/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/JobAutoScalerImpl.java
index 8023d7c2..041c03e9 100644
--- 
a/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/JobAutoScalerImpl.java
+++ 
b/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/JobAutoScalerImpl.java
@@ -22,9 +22,10 @@ import 
org.apache.flink.kubernetes.operator.api.AbstractFlinkResource;
 import 
org.apache.flink.kubernetes.operator.autoscaler.metrics.EvaluatedScalingMetric;
 import org.apache.flink.kubernetes.operator.autoscaler.metrics.ScalingMetric;
 import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext;
-import 
org.apache.flink.kubernetes.operator.metrics.KubernetesResourceMetricGroup;
 import 
org.apache.flink.kubernetes.operator.reconciler.deployment.JobAutoScaler;
 import org.apache.flink.kubernetes.operator.utils.EventRecorder;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 import io.fabric8.kubernetes.client.KubernetesClient;
@@ -55,6 +56,8 @@ public class JobAutoScalerImpl implements JobAutoScaler {
             lastEvaluatedMetrics = new ConcurrentHashMap<>();
     private final Map<ResourceID, Set<JobVertexID>> registeredMetrics = new 
ConcurrentHashMap<>();
 
+    final Map<ResourceID, Counter> errorCounters = new ConcurrentHashMap<>();
+
     public JobAutoScalerImpl(
             KubernetesClient kubernetesClient,
             ScalingMetricCollector metricsCollector,
@@ -82,8 +85,11 @@ public class JobAutoScalerImpl implements JobAutoScaler {
 
         var conf = ctx.getObserveConfig();
         var resource = ctx.getResource();
+        var resouceId = ResourceID.fromResource(resource);
+        var autoscalerMetricGroup = 
ctx.getResourceMetricGroup().addGroup("AutoScaler");
 
         try {
+
             if (resource.getSpec().getJob() == null || 
!conf.getBoolean(AUTOSCALER_ENABLED)) {
                 LOG.info("Job autoscaler is disabled");
                 return false;
@@ -108,8 +114,8 @@ public class JobAutoScalerImpl implements JobAutoScaler {
             LOG.debug("Evaluating scaling metrics for {}", collectedMetrics);
             var evaluatedMetrics = evaluator.evaluate(conf, collectedMetrics);
             LOG.debug("Scaling metrics evaluated: {}", evaluatedMetrics);
-            lastEvaluatedMetrics.put(ResourceID.fromResource(resource), 
evaluatedMetrics);
-            registerResourceScalingMetrics(resource, 
ctx.getResourceMetricGroup());
+            lastEvaluatedMetrics.put(resouceId, evaluatedMetrics);
+            registerResourceScalingMetrics(resource, autoscalerMetricGroup);
 
             var specAdjusted =
                     scalingExecutor.scaleResource(resource, autoScalerInfo, 
conf, evaluatedMetrics);
@@ -117,6 +123,9 @@ public class JobAutoScalerImpl implements JobAutoScaler {
             return specAdjusted;
         } catch (Throwable e) {
             LOG.error("Error while scaling resource", e);
+            errorCounters
+                    .computeIfAbsent(resouceId, _id -> 
autoscalerMetricGroup.counter("errors"))
+                    .inc();
             eventRecorder.triggerEvent(
                     resource,
                     EventRecorder.Type.Warning,
@@ -128,10 +137,8 @@ public class JobAutoScalerImpl implements JobAutoScaler {
     }
 
     private void registerResourceScalingMetrics(
-            AbstractFlinkResource<?, ?> resource,
-            KubernetesResourceMetricGroup resourceMetricGroup) {
+            AbstractFlinkResource<?, ?> resource, MetricGroup scalerGroup) {
         var resourceId = ResourceID.fromResource(resource);
-        var scalerGroup = resourceMetricGroup.addGroup("AutoScaler");
 
         lastEvaluatedMetrics
                 .get(resourceId)
diff --git 
a/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/JobAutoScalerImplTest.java
 
b/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/JobAutoScalerImplTest.java
new file mode 100644
index 00000000..5303e5c7
--- /dev/null
+++ 
b/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/JobAutoScalerImplTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.autoscaler;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.OperatorTestBase;
+import org.apache.flink.kubernetes.operator.TestUtils;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
+import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext;
+import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
+import io.javaoperatorsdk.operator.processing.event.ResourceID;
+import lombok.Getter;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static 
org.apache.flink.kubernetes.operator.autoscaler.config.AutoScalerOptions.AUTOSCALER_ENABLED;
+
+/** Tests for JobAutoScalerImpl. */
+@EnableKubernetesMockClient(crud = true)
+public class JobAutoScalerImplTest extends OperatorTestBase {
+
+    @Getter private KubernetesClient kubernetesClient;
+
+    private FlinkDeployment app;
+
+    @BeforeEach
+    public void setup() {
+        app = TestUtils.buildApplicationCluster();
+        app.getMetadata().setGeneration(1L);
+        app.getStatus().getJobStatus().setJobId(new JobID().toHexString());
+        kubernetesClient.resource(app).createOrReplace();
+
+        var defaultConf = new Configuration();
+        defaultConf.set(AUTOSCALER_ENABLED, true);
+        configManager = new FlinkConfigManager(defaultConf);
+        ReconciliationUtils.updateStatusForDeployedSpec(
+                app, configManager.getDeployConfig(app.getMetadata(), 
app.getSpec()));
+    }
+
+    @Test
+    void testErrorReporting() {
+        var autoscaler = new JobAutoScalerImpl(kubernetesClient, null, null, 
null, eventRecorder);
+        FlinkResourceContext<FlinkDeployment> resourceContext = 
getResourceContext(app);
+        ResourceID resourceId = ResourceID.fromResource(app);
+
+        autoscaler.scale(resourceContext);
+        Assertions.assertEquals(1, 
autoscaler.errorCounters.get(resourceId).getCount());
+
+        autoscaler.scale(resourceContext);
+        Assertions.assertEquals(2, 
autoscaler.errorCounters.get(resourceId).getCount());
+    }
+}

Reply via email to