[FLINK-32102] Aggregate multiple pendingRecords metric per source if present (#598)
Some source expose multiple .pendingRecords metrics. If that is the case, we must sum up these
records to yield the correct internal pending records count.
diff --git a/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/RestApiMetricsCollector.java b/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/RestApiMetricsCollector.java
index 40485ac..68924e9 100644
--- a/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/RestApiMetricsCollector.java
+++ b/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/RestApiMetricsCollector.java
@@ -53,7 +53,7 @@
return filteredVertexMetricNames.entrySet().stream()
.collect(
Collectors.toMap(
- e -> e.getKey(),
+ Map.Entry::getKey,
e ->
queryAggregatedVertexMetrics(
flinkService, cr, conf, e.getKey(), e.getValue())));
@@ -94,7 +94,18 @@
.get();
return responseBody.getMetrics().stream()
- .collect(Collectors.toMap(m -> metrics.get(m.getId()), m -> m));
+ .collect(
+ Collectors.toMap(
+ m -> metrics.get(m.getId()),
+ m -> m,
+ (m1, m2) ->
+ new AggregatedMetric(
+ m1.getId() + " merged with " + m2.getId(),
+ Math.min(m1.getMin(), m2.getMin()),
+ Math.max(m1.getMax(), m2.getMax()),
+ // Average can't be computed
+ Double.NaN,
+ m1.getSum() + m2.getSum())));
}
}
}
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 cb094e7..18478a9 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
@@ -53,6 +53,7 @@
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
@@ -371,14 +372,13 @@
// the first record. If this is a fresh topic or no new data has been read since
// the last checkpoint, the pendingRecords metrics won't be available. Also, legacy
// sources do not have this metric.
- Optional<String> pendingRecordsMetric =
- FlinkMetric.PENDING_RECORDS.findAny(allMetricNames);
- pendingRecordsMetric.ifPresentOrElse(
- m -> filteredMetrics.put(m, FlinkMetric.PENDING_RECORDS),
- () ->
- LOG.warn(
- "pendingRecords metric for {} could not be found. Either a legacy source or an idle source. Assuming no pending records.",
- jobVertexID));
+ List<String> pendingRecordsMetric = FlinkMetric.PENDING_RECORDS.findAll(allMetricNames);
+ if (pendingRecordsMetric.isEmpty()) {
+ LOG.warn(
+ "pendingRecords metric for {} could not be found. Either a legacy source or an idle source. Assuming no pending records.",
+ jobVertexID);
+ }
+ pendingRecordsMetric.forEach(m -> filteredMetrics.put(m, FlinkMetric.PENDING_RECORDS));
FlinkMetric.SOURCE_TASK_NUM_RECORDS_OUT_PER_SEC
.findAny(allMetricNames)
.ifPresent(
diff --git a/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/metrics/FlinkMetric.java b/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/metrics/FlinkMetric.java
index 76ba7a9..703e4e7 100644
--- a/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/metrics/FlinkMetric.java
+++ b/flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/metrics/FlinkMetric.java
@@ -20,8 +20,10 @@
import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedMetric;
import java.util.Collection;
+import java.util.List;
import java.util.Optional;
import java.util.function.Predicate;
+import java.util.stream.Collectors;
/**
* Enum representing the collected Flink metrics for autoscaling. The actual metric names depend on
@@ -46,4 +48,11 @@
public Optional<String> findAny(Collection<AggregatedMetric> metrics) {
return metrics.stream().map(AggregatedMetric::getId).filter(predicate).findAny();
}
+
+ public List<String> findAll(Collection<AggregatedMetric> metrics) {
+ return metrics.stream()
+ .map(AggregatedMetric::getId)
+ .filter(predicate)
+ .collect(Collectors.toList());
+ }
}
diff --git a/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/RestApiMetricsCollectorTest.java b/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/RestApiMetricsCollectorTest.java
new file mode 100644
index 0000000..04d8310
--- /dev/null
+++ b/flink-kubernetes-operator-autoscaler/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/RestApiMetricsCollectorTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.operator.TestingFlinkService;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.autoscaler.metrics.FlinkMetric;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedMetric;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Map;
+
+/** Tests for RestApiMetrisCollector. */
+public class RestApiMetricsCollectorTest {
+
+ @Test
+ public void testAggregateMultiplePendingRecordsMetricsPerSource() {
+ var collector = new RestApiMetricsCollector();
+
+ JobVertexID jobVertexID = new JobVertexID();
+ Map<String, FlinkMetric> flinkMetrics =
+ Map.of(
+ "a.pendingRecords", FlinkMetric.PENDING_RECORDS,
+ "b.pendingRecords", FlinkMetric.PENDING_RECORDS);
+ Map<JobVertexID, Map<String, FlinkMetric>> metrics = Map.of(jobVertexID, flinkMetrics);
+
+ FlinkDeployment cr = new FlinkDeployment();
+ cr.getStatus().getJobStatus().setJobId(new JobID().toHexString());
+
+ TestingFlinkService flinkService = new TestingFlinkService();
+ flinkService.setAggregatedMetricsResponse(
+ List.of(
+ new AggregatedMetric(
+ "a.pendingRecords", Double.NaN, Double.NaN, Double.NaN, 100.),
+ new AggregatedMetric(
+ "b.pendingRecords", Double.NaN, Double.NaN, Double.NaN, 100.),
+ new AggregatedMetric(
+ "c.unrelated", Double.NaN, Double.NaN, Double.NaN, 100.)));
+
+ Configuration conf = new Configuration();
+ conf.set(KubernetesConfigOptions.CLUSTER_ID, "id");
+
+ Map<JobVertexID, Map<FlinkMetric, AggregatedMetric>> jobVertexIDMapMap =
+ collector.queryAllAggregatedMetrics(cr, flinkService, conf, metrics);
+
+ System.out.println(jobVertexIDMapMap);
+
+ Assertions.assertEquals(1, jobVertexIDMapMap.size());
+ Map<FlinkMetric, AggregatedMetric> vertexMetrics = jobVertexIDMapMap.get(jobVertexID);
+ Assertions.assertNotNull(vertexMetrics);
+ AggregatedMetric pendingRecordsMetric = vertexMetrics.get(FlinkMetric.PENDING_RECORDS);
+ Assertions.assertNotNull(pendingRecordsMetric);
+ Assertions.assertEquals(pendingRecordsMetric.getSum(), 200);
+ }
+}
diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java
index d6965b0..c0dc3da 100644
--- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java
+++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java
@@ -60,6 +60,9 @@
import org.apache.flink.runtime.rest.messages.DashboardConfiguration;
import org.apache.flink.runtime.rest.messages.EmptyResponseBody;
import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders;
+import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedMetric;
+import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedMetricsResponseBody;
+import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedSubtaskMetricsHeaders;
import org.apache.flink.util.SerializedThrowable;
import io.fabric8.kubernetes.api.model.DeletionPropagation;
@@ -75,6 +78,7 @@
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -125,6 +129,9 @@
private Map<String, String> metricsValues = new HashMap<>();
+ @Setter
+ private Collection<AggregatedMetric> aggregatedMetricsResponse = Collections.emptyList();
+
public TestingFlinkService() {
super(null, new FlinkConfigManager(new Configuration()));
}
@@ -334,6 +341,8 @@
(messageHeaders, messageParameters, requestBody) -> {
if (messageHeaders instanceof JobsOverviewHeaders) {
return CompletableFuture.completedFuture(getMultipleJobsDetails());
+ } else if (messageHeaders instanceof AggregatedSubtaskMetricsHeaders) {
+ return CompletableFuture.completedFuture(getSubtaskMetrics());
}
return CompletableFuture.completedFuture(EmptyResponseBody.getInstance());
});
@@ -348,6 +357,10 @@
.collect(Collectors.toList()));
}
+ private AggregatedMetricsResponseBody getSubtaskMetrics() {
+ return new AggregatedMetricsResponseBody(aggregatedMetricsResponse);
+ }
+
private static JobDetails toJobDetails(JobStatusMessage jobStatus) {
return new JobDetails(
jobStatus.getJobId(),