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

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


The following commit(s) were added to refs/heads/main by this push:
     new 844a85bf [hotfix] Fix JobDetailsInfo json extraction for 1.17
844a85bf is described below

commit 844a85bf7b8c142ddee22e3b481ce08020cbec51
Author: Gyula Fora <[email protected]>
AuthorDate: Thu Jun 8 11:22:09 2023 +0200

    [hotfix] Fix JobDetailsInfo json extraction for 1.17
---
 .../autoscaler/ScalingMetricCollector.java         |  8 +++-
 .../autoscaler/ScalingMetricCollectorTest.java     | 45 ++++++++++++++++++++++
 2 files changed, 52 insertions(+), 1 deletion(-)

diff --git 
a/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/ScalingMetricCollector.java
 
b/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/ScalingMetricCollector.java
index 0fb65242..365da10e 100644
--- 
a/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/ScalingMetricCollector.java
+++ 
b/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/ScalingMetricCollector.java
@@ -184,7 +184,13 @@ public abstract class ScalingMetricCollector {
                                             JobDetailsInfo.JobVertexDetailsInfo
                                                     ::getMaxParallelism));
 
-            return JobTopology.fromJsonPlan(jobDetailsInfo.getJsonPlan(), 
maxParallelismMap);
+            // Flink 1.17 introduced a strange behaviour where the 
JobDetailsInfo#getJsonPlan()
+            // doesn't return the jsonPlan correctly but it wraps with 
RawJson{json='plan'}
+            var rawPlan = jobDetailsInfo.getJsonPlan();
+            var json =
+                    rawPlan.substring("RawJson{json='".length(), 
rawPlan.length() - "'}".length());
+
+            return JobTopology.fromJsonPlan(json, maxParallelismMap);
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
diff --git 
a/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/ScalingMetricCollectorTest.java
 
b/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/ScalingMetricCollectorTest.java
new file mode 100644
index 00000000..a6e5665c
--- /dev/null
+++ 
b/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/ScalingMetricCollectorTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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.TestingClusterClient;
+import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo;
+
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.CompletableFuture;
+
+/** Tests for {@link ScalingMetricCollector}. */
+public class ScalingMetricCollectorTest {
+
+    @Test
+    public void testJobTopologyParsingFromJobDetails() throws Exception {
+        String s =
+                
"{\"jid\":\"8b6cdb9a1db8876d3dd803d5e6108ae3\",\"name\":\"State machine 
job\",\"isStoppable\":false,\"state\":\"RUNNING\",\"start-time\":1686216314565,\"end-time\":-1,\"duration\":25867,\"maxParallelism\":-1,\"now\":1686216340432,\"timestamps\":{\"INITIALIZING\":1686216314565,\"RECONCILING\":0,\"CANCELLING\":0,\"FAILING\":0,\"CREATED\":1686216314697,\"SUSPENDED\":0,\"RUNNING\":1686216314900,\"FAILED\":0,\"CANCELED\":0,\"FINISHED\":0,\"RESTARTING\":0},\"vertices\":[{\"id\"
 [...]
+        JobDetailsInfo jobDetailsInfo = new ObjectMapper().readValue(s, 
JobDetailsInfo.class);
+
+        var metricsCollector = new RestApiMetricsCollector();
+        var client = new TestingClusterClient<>(new Configuration(), "test");
+        client.setRequestProcessor((h, p, b) -> 
CompletableFuture.completedFuture(jobDetailsInfo));
+        metricsCollector.queryJobTopology(client, new JobID());
+    }
+}

Reply via email to