tweise commented on code in PR #484:
URL: 
https://github.com/apache/flink-kubernetes-operator/pull/484#discussion_r1050877759


##########
examples/autoscaling/src/main/java/autoscaling/AutoscalingExample.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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 autoscaling;
+
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+/** Autoscaling Example. */
+public class AutoscalingExample {
+    public static void main(String[] args) throws Exception {
+        final long start = System.currentTimeMillis();
+        var env = StreamExecutionEnvironment.getExecutionEnvironment();
+        DataStream<Long> stream = env.fromSequence(Long.MIN_VALUE, 
Long.MAX_VALUE);
+        stream =
+                stream.shuffle()
+                        .map(
+                                i -> {
+                                    long period = (System.currentTimeMillis() 
- start) / 120_000;
+                                    long sleep = period * 100;
+                                    // Thread.sleep(sleep);

Review Comment:
   Please document how this is supposed to work.



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/config/AutoScalerOptions.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.config;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+import java.time.Duration;
+
+import static 
org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.operatorConfig;
+
+/** Config options related to the autoscaler module. */
+public class AutoScalerOptions {
+
+    private static ConfigOptions.OptionBuilder autoScalerConfig(String key) {
+        return operatorConfig("job.autoscaler." + key);
+    }
+
+    public static final ConfigOption<Boolean> AUTOSCALER_ENABLED =
+            autoScalerConfig("enabled")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription("Enable job autoscaler module.");
+
+    public static final ConfigOption<Boolean> SCALING_ENABLED =

Review Comment:
   The difference between this and above option could be more explicit with 
expanded description.



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/ScalingMetricCollector.java:
##########
@@ -0,0 +1,443 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.client.program.rest.RestClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource;
+import 
org.apache.flink.kubernetes.operator.autoscaler.config.AutoScalerOptions;
+import 
org.apache.flink.kubernetes.operator.autoscaler.metrics.CollectedMetrics;
+import org.apache.flink.kubernetes.operator.autoscaler.metrics.FlinkMetric;
+import org.apache.flink.kubernetes.operator.autoscaler.metrics.ScalingMetric;
+import org.apache.flink.kubernetes.operator.autoscaler.metrics.ScalingMetrics;
+import org.apache.flink.kubernetes.operator.autoscaler.topology.JobTopology;
+import org.apache.flink.kubernetes.operator.service.FlinkService;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.JobIDPathParameter;
+import org.apache.flink.runtime.rest.messages.JobVertexIdPathParameter;
+import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo;
+import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedMetric;
+import 
org.apache.flink.runtime.rest.messages.job.metrics.AggregatedSubtaskMetricsHeaders;
+import 
org.apache.flink.runtime.rest.messages.job.metrics.AggregatedSubtaskMetricsParameters;
+import org.apache.flink.util.Preconditions;
+
+import io.javaoperatorsdk.operator.processing.event.ResourceID;
+import lombok.SneakyThrows;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Clock;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.kubernetes.operator.autoscaler.config.AutoScalerOptions.SOURCE_SCALING_ENABLED;
+
+/** Metric collector using flink rest api. */
+public abstract class ScalingMetricCollector implements Cleanup {
+    private static final Logger LOG = 
LoggerFactory.getLogger(ScalingMetricCollector.class);
+
+    private final Map<ResourceID, Tuple2<Long, Map<JobVertexID, Map<String, 
FlinkMetric>>>>
+            availableVertexMetricNames = new ConcurrentHashMap<>();
+
+    private final Map<ResourceID, SortedMap<Instant, Map<JobVertexID, 
Map<ScalingMetric, Double>>>>
+            histories = new ConcurrentHashMap<>();
+
+    private final Map<ResourceID, JobTopology> topologies = new 
ConcurrentHashMap<>();
+
+    private Clock clock = Clock.systemDefaultZone();
+
+    public CollectedMetrics getMetricsHistory(
+            AbstractFlinkResource<?, ?> cr,
+            AutoScalerInfo scalingInformation,
+            FlinkService flinkService,
+            Configuration conf)
+            throws Exception {
+
+        var resourceID = ResourceID.fromResource(cr);
+        var currentJobStartTs =
+                
Instant.ofEpochMilli(Long.parseLong(cr.getStatus().getJobStatus().getStartTime()));
+
+        if (!currentJobStartTs.equals(
+                scalingInformation.getJobStartTs().orElse(currentJobStartTs))) 
{
+            scalingInformation.clearMetricHistory();
+            cleanup(cr);
+        }
+
+        // Initialize metric history
+        var scalingMetricHistory =
+                histories.compute(
+                        resourceID,
+                        (k, h) -> {
+                            if (h == null) {
+                                h = scalingInformation.getMetricHistory();
+                            }
+                            return h.tailMap(
+                                    clock.instant()
+                                            
.minus(conf.get(AutoScalerOptions.METRICS_WINDOW)));
+                        });
+
+        var topology = getJobTopology(flinkService, cr, conf);
+
+        // The filtered list of metrics we want to query for each vertex
+        var filteredVertexMetricNames = queryFilteredMetricNames(flinkService, 
cr, conf, topology);
+
+        // Aggregated job vertex metrics collected from Flink based on the 
filtered metric names
+        var collectedVertexMetrics =
+                queryAllAggregatedMetrics(cr, flinkService, conf, 
filteredVertexMetricNames);
+
+        // The computed scaling metrics based on the collected aggregated 
vertex metrics
+        var scalingMetrics =
+                convertToScalingMetrics(resourceID, collectedVertexMetrics, 
topology, conf);
+
+        // Add scaling metrics to history if they were computed successfully
+        scalingMetricHistory.put(clock.instant(), scalingMetrics);
+        scalingInformation.updateMetricHistory(currentJobStartTs, 
scalingMetricHistory);
+
+        return new CollectedMetrics(topology, scalingMetricHistory);
+    }
+
+    protected JobTopology getJobTopology(
+            FlinkService flinkService, AbstractFlinkResource<?, ?> cr, 
Configuration conf)
+            throws Exception {
+
+        try (var restClient = (RestClusterClient<String>) 
flinkService.getClusterClient(conf)) {
+            var jobId = 
JobID.fromHexString(cr.getStatus().getJobStatus().getJobId());
+            var topology =
+                    topologies.computeIfAbsent(
+                            ResourceID.fromResource(cr), r -> 
queryJobTopology(restClient, jobId));
+            updateKafkaSourceMaxParallelisms(restClient, jobId, topology);
+            return topology;
+        }
+    }
+
+    @VisibleForTesting
+    protected JobTopology queryJobTopology(RestClusterClient<String> 
restClient, JobID jobId) {
+        try {
+            var jobDetailsInfo = restClient.getJobDetails(jobId).get();
+
+            Map<JobVertexID, Integer> maxParallelismMap =
+                    jobDetailsInfo.getJobVertexInfos().stream()
+                            .collect(
+                                    Collectors.toMap(
+                                            
JobDetailsInfo.JobVertexDetailsInfo::getJobVertexID,
+                                            JobDetailsInfo.JobVertexDetailsInfo
+                                                    ::getMaxParallelism));
+
+            return JobTopology.fromJsonPlan(jobDetailsInfo.getJsonPlan(), 
maxParallelismMap);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private void updateKafkaSourceMaxParallelisms(
+            RestClusterClient<String> restClient, JobID jobId, JobTopology 
topology)
+            throws Exception {
+        for (Map.Entry<JobVertexID, Set<JobVertexID>> entry : 
topology.getInputs().entrySet()) {
+            if (entry.getValue().isEmpty()) {
+                var sourceVertex = entry.getKey();
+                queryAggregatedMetricNames(restClient, jobId, 
sourceVertex).stream()
+                        .map(AggregatedMetric::getId)
+                        .filter(s -> s.endsWith(".currentOffset"))
+                        .mapToInt(
+                                s -> {
+                                    // We extract the partition from the 
pattern:

Review Comment:
   Should this be marked as Kafka-specific?



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/Cleanup.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.kubernetes.operator.api.AbstractFlinkResource;
+
+/** Cleanup interface for autoscaling related metadata. */
+public interface Cleanup {

Review Comment:
   nit: Although I understand it is scoped in a package, this is a rather 
unusual interface name.



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