gyfora commented on code in PR #1099: URL: https://github.com/apache/flink-kubernetes-operator/pull/1099#discussion_r3425465811
########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/metrics/FlinkAutoscalerEvaluator.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.autoscaler.metrics; + +import org.apache.flink.autoscaler.topology.JobTopology; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import java.time.Duration; +import java.time.Instant; +import java.util.Map; +import java.util.SortedMap; + +/** + * A pluggable plugin that allows users to provide custom scaling-metric evaluation logic on top of + * the metrics evaluated internally by the autoscaler. Implementations are invoked once per job + * vertex during each evaluation cycle, and the metrics they return are merged on top of the + * internally evaluated metrics, allowing users to override or augment specific {@link + * ScalingMetric} values. + * + * <p>Only one custom metric evaluator per pipeline is supported for now. If multiple instances are + * configured, the autoscaler logs a warning and falls back to the first entry, ignoring the rest. + * Registering multiple implementations via {@code META-INF/services} is fine as they form a + * registry that different jobs can select from by class FQN, but a single job cannot chain or + * compose more than one evaluator. + * + * <p>This was introduced as part of <a + * href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-514%3A+Custom+Evaluator+plugin+for+Flink+Autoscaler">FLIP-514: + * Custom Evaluator plugin for Flink Autoscaler</a> and is complementary to <a + * href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-575%3A+Scaling+Executor+Plugin+SPI+for+Flink+Autoscaler">FLIP-575: + * Scaling Executor Plugin SPI for Flink Autoscaler</a> which provides extensibility at the scaling + * decision execution layer. + * + * <p>Implementations are discovered via Java's {@link java.util.ServiceLoader} mechanism. To + * register a custom metric evaluator, add the fully qualified class name of the implementation to + * {@code META-INF/services/org.apache.flink.autoscaler.metrics.FlinkAutoscalerEvaluator}. + */ +public interface FlinkAutoscalerEvaluator { + + /** + * Evaluates scaling metrics for a given job vertex based on the internally evaluated metrics + * and context. + * + * @param vertex The {@link JobVertexID} identifying the vertex whose metrics are being + * evaluated. + * @param evaluatedMetrics An un-modifiable view of current vertex internally evaluated metrics. + * @param evaluationContext The evaluation context providing job-related configurations and + * historical metrics. + * @return A map of evaluated scaling metrics for the vertex which would get merged with + * internally evaluated metrics. + * @throws UnsupportedOperationException if an attempt is made to modify the {@code + * evaluatedMetrics}, {@code Context.jobConf}, {@code Context.metricsHistory}, {@code + * Context.evaluatedVertexMetrics}. + */ + Map<ScalingMetric, EvaluatedScalingMetric> evaluateVertexMetrics( + JobVertexID vertex, + Map<ScalingMetric, EvaluatedScalingMetric> evaluatedMetrics, + Context evaluationContext); + + /** + * Context providing relevant job and metric information to assist in custom metric evaluation. + */ + class Context { Review Comment: Use `@Value` to reduce boilerplate ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/ScalingMetricEvaluator.java: ########## @@ -72,14 +76,34 @@ public class ScalingMetricEvaluator { private static final Logger LOG = LoggerFactory.getLogger(ScalingMetricEvaluator.class); public EvaluatedMetrics evaluate( - Configuration conf, CollectedMetricHistory collectedMetrics, Duration restartTime) { + Configuration conf, + CollectedMetricHistory collectedMetrics, + Duration restartTime, + @Nullable Tuple2<FlinkAutoscalerEvaluator, Configuration> customEvaluatorWithConfig) { Review Comment: As I said in another comment, maybe we can do this once during initialization? Unless we expect this to change and be reconfigured at runtime ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/ScalingMetricEvaluator.java: ########## @@ -588,4 +635,76 @@ protected static double computeEdgeDataRate( to); return getRate(ScalingMetric.NUM_RECORDS_OUT, from, metricsHistory); } + + /** + * Executes the provided custom metric evaluator for the given job vertex. Calls {@link + * FlinkAutoscalerEvaluator#evaluateVertexMetrics} to evaluate scaling metrics. + * + * @param vertex The job vertex being evaluated. + * @param evaluatedMetrics Current evaluated metrics. + * @param customEvaluationSession A tuple containing the custom metric evaluator and evaluation + * context. + * @return A map of scaling metrics, with its corresponding evaluated scaling metric. + */ + @VisibleForTesting + protected static Map<ScalingMetric, EvaluatedScalingMetric> runCustomEvaluator( + JobVertexID vertex, + Map<ScalingMetric, EvaluatedScalingMetric> evaluatedMetrics, + Tuple2<FlinkAutoscalerEvaluator, FlinkAutoscalerEvaluator.Context> + customEvaluationSession) { + try { + return customEvaluationSession.f0.evaluateVertexMetrics( + vertex, evaluatedMetrics, customEvaluationSession.f1); + } catch (UnsupportedOperationException e) { + LOG.warn( + "Custom metric evaluator {} tried accessing an un-modifiable view.", + customEvaluationSession.f0.getClass(), + e); + } catch (Exception e) { + LOG.warn( + "Custom metric evaluator {} threw an exception.", + customEvaluationSession.f0.getClass(), + e); + } + + return Collections.emptyMap(); + } + + /** + * Merges the incoming evaluated metrics into actual evaluated metrics. + * + * @param actual The target evaluated metrics map to merge into. + * @param incoming The incoming map containing new evaluated metrics map to be merged + * (nullable). + */ + @VisibleForTesting + protected static void mergeEvaluatedMetricsMaps( + Map<ScalingMetric, EvaluatedScalingMetric> actual, + @Nullable Map<ScalingMetric, EvaluatedScalingMetric> incoming) { + Optional.ofNullable(incoming) + .ifPresent( + customEvaluatedMetric -> + customEvaluatedMetric.forEach( + (scalingMetric, evaluatedScalingMetric) -> + actual.merge( + scalingMetric, + evaluatedScalingMetric, + ScalingMetricEvaluator + ::mergeEvaluatedScalingMetric))); + } + + /** + * Merges two {@link EvaluatedScalingMetric} instances. + * + * @param actual The existing evaluated scaling metric. + * @param incoming The incoming evaluated scaling metric. + * @return A new {@link EvaluatedScalingMetric} instance with merged values. + */ Review Comment: unnecessary ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/JobAutoScalerImpl.java: ########## @@ -259,4 +276,55 @@ void setClock(Clock clock) { this.metricsCollector.setClock(clock); this.scalingExecutor.setClock(clock); } + + @VisibleForTesting + protected Tuple2<FlinkAutoscalerEvaluator, Configuration> getCustomEvaluatorIfRequired( Review Comment: I think this method should go to a utility class if possible. We should aim to keep this class minimal with the core algorithm flow ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/ScalingMetricEvaluator.java: ########## @@ -588,4 +635,76 @@ protected static double computeEdgeDataRate( to); return getRate(ScalingMetric.NUM_RECORDS_OUT, from, metricsHistory); } + + /** + * Executes the provided custom metric evaluator for the given job vertex. Calls {@link + * FlinkAutoscalerEvaluator#evaluateVertexMetrics} to evaluate scaling metrics. + * + * @param vertex The job vertex being evaluated. + * @param evaluatedMetrics Current evaluated metrics. + * @param customEvaluationSession A tuple containing the custom metric evaluator and evaluation + * context. + * @return A map of scaling metrics, with its corresponding evaluated scaling metric. + */ + @VisibleForTesting + protected static Map<ScalingMetric, EvaluatedScalingMetric> runCustomEvaluator( + JobVertexID vertex, + Map<ScalingMetric, EvaluatedScalingMetric> evaluatedMetrics, + Tuple2<FlinkAutoscalerEvaluator, FlinkAutoscalerEvaluator.Context> + customEvaluationSession) { + try { + return customEvaluationSession.f0.evaluateVertexMetrics( + vertex, evaluatedMetrics, customEvaluationSession.f1); + } catch (UnsupportedOperationException e) { + LOG.warn( + "Custom metric evaluator {} tried accessing an un-modifiable view.", + customEvaluationSession.f0.getClass(), + e); Review Comment: This feels unnecessary ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/metrics/FlinkAutoscalerEvaluator.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.autoscaler.metrics; + +import org.apache.flink.autoscaler.topology.JobTopology; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import java.time.Duration; +import java.time.Instant; +import java.util.Map; +import java.util.SortedMap; + +/** + * A pluggable plugin that allows users to provide custom scaling-metric evaluation logic on top of + * the metrics evaluated internally by the autoscaler. Implementations are invoked once per job + * vertex during each evaluation cycle, and the metrics they return are merged on top of the + * internally evaluated metrics, allowing users to override or augment specific {@link + * ScalingMetric} values. + * + * <p>Only one custom metric evaluator per pipeline is supported for now. If multiple instances are + * configured, the autoscaler logs a warning and falls back to the first entry, ignoring the rest. + * Registering multiple implementations via {@code META-INF/services} is fine as they form a + * registry that different jobs can select from by class FQN, but a single job cannot chain or + * compose more than one evaluator. + * + * <p>This was introduced as part of <a + * href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-514%3A+Custom+Evaluator+plugin+for+Flink+Autoscaler">FLIP-514: + * Custom Evaluator plugin for Flink Autoscaler</a> and is complementary to <a + * href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-575%3A+Scaling+Executor+Plugin+SPI+for+Flink+Autoscaler">FLIP-575: + * Scaling Executor Plugin SPI for Flink Autoscaler</a> which provides extensibility at the scaling + * decision execution layer. + * + * <p>Implementations are discovered via Java's {@link java.util.ServiceLoader} mechanism. To + * register a custom metric evaluator, add the fully qualified class name of the implementation to + * {@code META-INF/services/org.apache.flink.autoscaler.metrics.FlinkAutoscalerEvaluator}. + */ +public interface FlinkAutoscalerEvaluator { + + /** + * Evaluates scaling metrics for a given job vertex based on the internally evaluated metrics + * and context. + * + * @param vertex The {@link JobVertexID} identifying the vertex whose metrics are being + * evaluated. + * @param evaluatedMetrics An un-modifiable view of current vertex internally evaluated metrics. + * @param evaluationContext The evaluation context providing job-related configurations and + * historical metrics. + * @return A map of evaluated scaling metrics for the vertex which would get merged with + * internally evaluated metrics. + * @throws UnsupportedOperationException if an attempt is made to modify the {@code + * evaluatedMetrics}, {@code Context.jobConf}, {@code Context.metricsHistory}, {@code + * Context.evaluatedVertexMetrics}. + */ + Map<ScalingMetric, EvaluatedScalingMetric> evaluateVertexMetrics( + JobVertexID vertex, + Map<ScalingMetric, EvaluatedScalingMetric> evaluatedMetrics, + Context evaluationContext); + + /** + * Context providing relevant job and metric information to assist in custom metric evaluation. + */ + class Context { + private final Configuration jobConf; Review Comment: I think it's a bit strange to have both jobConf and customEvaluatorConf. I think we should have a single `config` field that merges these two. Take the evaluator conf and apply the jobConf on top of it. This would allow CRs to control evaluator behaviour easily. This is the pattern followed in other places in general ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/JobAutoScalerImpl.java: ########## @@ -259,4 +276,55 @@ void setClock(Clock clock) { this.metricsCollector.setClock(clock); this.scalingExecutor.setClock(clock); } + + @VisibleForTesting + protected Tuple2<FlinkAutoscalerEvaluator, Configuration> getCustomEvaluatorIfRequired( Review Comment: Maybe this can happen inside the ScalingMetricEvaluator constructor/initialization so we don't have to update the evaluate method signature ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/ScalingMetricEvaluator.java: ########## @@ -588,4 +635,76 @@ protected static double computeEdgeDataRate( to); return getRate(ScalingMetric.NUM_RECORDS_OUT, from, metricsHistory); } + + /** + * Executes the provided custom metric evaluator for the given job vertex. Calls {@link + * FlinkAutoscalerEvaluator#evaluateVertexMetrics} to evaluate scaling metrics. + * + * @param vertex The job vertex being evaluated. + * @param evaluatedMetrics Current evaluated metrics. + * @param customEvaluationSession A tuple containing the custom metric evaluator and evaluation + * context. + * @return A map of scaling metrics, with its corresponding evaluated scaling metric. + */ + @VisibleForTesting + protected static Map<ScalingMetric, EvaluatedScalingMetric> runCustomEvaluator( + JobVertexID vertex, + Map<ScalingMetric, EvaluatedScalingMetric> evaluatedMetrics, + Tuple2<FlinkAutoscalerEvaluator, FlinkAutoscalerEvaluator.Context> + customEvaluationSession) { + try { + return customEvaluationSession.f0.evaluateVertexMetrics( + vertex, evaluatedMetrics, customEvaluationSession.f1); + } catch (UnsupportedOperationException e) { + LOG.warn( + "Custom metric evaluator {} tried accessing an un-modifiable view.", + customEvaluationSession.f0.getClass(), + e); + } catch (Exception e) { + LOG.warn( + "Custom metric evaluator {} threw an exception.", + customEvaluationSession.f0.getClass(), + e); + } + + return Collections.emptyMap(); + } + + /** + * Merges the incoming evaluated metrics into actual evaluated metrics. + * + * @param actual The target evaluated metrics map to merge into. + * @param incoming The incoming map containing new evaluated metrics map to be merged + * (nullable). + */ Review Comment: unnecessary -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
