Skip to content

Draft PR for plugin based approach for custom evaluator for scaling metric evaluation #953

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Draft
wants to merge 3 commits into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,17 @@

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.autoscaler.config.AutoScalerOptions;
import org.apache.flink.autoscaler.event.AutoScalerEventHandler;
import org.apache.flink.autoscaler.exceptions.NotReadyException;
import org.apache.flink.autoscaler.metrics.AutoscalerFlinkMetrics;
import org.apache.flink.autoscaler.metrics.CustomEvaluator;
import org.apache.flink.autoscaler.metrics.EvaluatedMetrics;
import org.apache.flink.autoscaler.realizer.ScalingRealizer;
import org.apache.flink.autoscaler.state.AutoScalerStateStore;
import org.apache.flink.autoscaler.tuning.ConfigChanges;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.PipelineOptions;
import org.apache.flink.util.Preconditions;

Expand All @@ -36,11 +39,14 @@
import java.time.Clock;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;

import static org.apache.flink.autoscaler.config.AutoScalerOptions.AUTOSCALER_ENABLED;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.CUSTOM_EVALUATOR_NAME;
import static org.apache.flink.autoscaler.metrics.AutoscalerFlinkMetrics.initRecommendedParallelism;
import static org.apache.flink.autoscaler.metrics.AutoscalerFlinkMetrics.resetRecommendedParallelism;
import static org.apache.flink.autoscaler.metrics.CustomEvaluatorOptions.CUSTOM_EVALUATOR_CLASS;
import static org.apache.flink.autoscaler.metrics.ScalingHistoryUtils.getTrimmedScalingHistory;
import static org.apache.flink.autoscaler.metrics.ScalingHistoryUtils.getTrimmedScalingTracking;

Expand All @@ -58,6 +64,7 @@ public class JobAutoScalerImpl<KEY, Context extends JobAutoScalerContext<KEY>>
private final AutoScalerEventHandler<KEY, Context> eventHandler;
private final ScalingRealizer<KEY, Context> scalingRealizer;
private final AutoScalerStateStore<KEY, Context> stateStore;
private final Map<String, CustomEvaluator> customEvaluators;

private Clock clock = Clock.systemDefaultZone();

Expand All @@ -73,13 +80,15 @@ public JobAutoScalerImpl(
ScalingExecutor<KEY, Context> scalingExecutor,
AutoScalerEventHandler<KEY, Context> eventHandler,
ScalingRealizer<KEY, Context> scalingRealizer,
AutoScalerStateStore<KEY, Context> stateStore) {
AutoScalerStateStore<KEY, Context> stateStore,
Map<String, CustomEvaluator> customEvaluators) {
this.metricsCollector = metricsCollector;
this.evaluator = evaluator;
this.scalingExecutor = scalingExecutor;
this.eventHandler = eventHandler;
this.scalingRealizer = scalingRealizer;
this.stateStore = stateStore;
this.customEvaluators = customEvaluators;
}

@Override
Expand Down Expand Up @@ -203,8 +212,15 @@ private void runScalingLogic(Context ctx, AutoscalerFlinkMetrics autoscalerMetri

// Scaling tracking data contains previous restart times that are taken into account
var restartTime = scalingTracking.getMaxRestartTimeOrDefault(ctx.getConfiguration());

var customEvaluatorWithConfig = getCustomEvaluatorIfRequired(ctx.getConfiguration());

var evaluatedMetrics =
evaluator.evaluate(ctx.getConfiguration(), collectedMetrics, restartTime);
evaluator.evaluate(
ctx.getConfiguration(),
collectedMetrics,
restartTime,
customEvaluatorWithConfig);
LOG.debug("Evaluated metrics: {}", evaluatedMetrics);
lastEvaluatedMetrics.put(ctx.getJobKey(), evaluatedMetrics);

Expand Down Expand Up @@ -259,4 +275,17 @@ void setClock(Clock clock) {
this.metricsCollector.setClock(clock);
this.scalingExecutor.setClock(clock);
}

@VisibleForTesting
protected Tuple2<CustomEvaluator, Configuration> getCustomEvaluatorIfRequired(
Configuration conf) {
var customEvaluatorName = conf.get(CUSTOM_EVALUATOR_NAME);
var customEvaluatorConfig = AutoScalerOptions.forCustomEvaluator(conf, customEvaluatorName);
CustomEvaluator evaluator =
Optional.ofNullable(customEvaluatorConfig.get(CUSTOM_EVALUATOR_CLASS))
.map(this.customEvaluators::get)
.orElse(null);

return evaluator != null ? new Tuple2<>(evaluator, customEvaluatorConfig) : null;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,16 +18,19 @@
package org.apache.flink.autoscaler;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.autoscaler.config.AutoScalerOptions;
import org.apache.flink.autoscaler.metrics.CollectedMetricHistory;
import org.apache.flink.autoscaler.metrics.CollectedMetrics;
import org.apache.flink.autoscaler.metrics.CustomEvaluator;
import org.apache.flink.autoscaler.metrics.EvaluatedMetrics;
import org.apache.flink.autoscaler.metrics.EvaluatedScalingMetric;
import org.apache.flink.autoscaler.metrics.MetricAggregator;
import org.apache.flink.autoscaler.metrics.ScalingMetric;
import org.apache.flink.autoscaler.topology.JobTopology;
import org.apache.flink.autoscaler.utils.AutoScalerUtils;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.UnmodifiableConfiguration;
import org.apache.flink.runtime.jobgraph.JobVertexID;

import org.slf4j.Logger;
Expand All @@ -38,6 +41,7 @@

import java.time.Duration;
import java.time.Instant;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
Expand Down Expand Up @@ -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<CustomEvaluator, Configuration> customEvaluatorWithConfig) {
LOG.debug("Restart time used in metrics evaluation: {}", restartTime);
var scalingOutput = new HashMap<JobVertexID, Map<ScalingMetric, EvaluatedScalingMetric>>();
var metricsHistory = collectedMetrics.getMetricHistory();
var topology = collectedMetrics.getJobTopology();

boolean processingBacklog = isProcessingBacklog(topology, metricsHistory, conf);

var customEvaluationSession =
Optional.ofNullable(customEvaluatorWithConfig)
.map(
info ->
Tuple2.of(
info.f0,
new CustomEvaluator.Context(
new UnmodifiableConfiguration(conf),
Collections.unmodifiableSortedMap(
metricsHistory),
Collections.unmodifiableMap(scalingOutput),
topology,
processingBacklog,
restartTime,
info.f1)))
.orElse(null);

for (var vertex : topology.getVerticesInTopologicalOrder()) {
scalingOutput.put(
vertex,
Expand All @@ -90,7 +114,8 @@ public EvaluatedMetrics evaluate(
topology,
vertex,
processingBacklog,
restartTime));
restartTime,
customEvaluationSession));
}

var globalMetrics = evaluateGlobalMetrics(metricsHistory);
Expand Down Expand Up @@ -132,7 +157,8 @@ private Map<ScalingMetric, EvaluatedScalingMetric> evaluateMetrics(
JobTopology topology,
JobVertexID vertex,
boolean processingBacklog,
Duration restartTime) {
Duration restartTime,
@Nullable Tuple2<CustomEvaluator, CustomEvaluator.Context> customEvaluationSession) {

var latestVertexMetrics =
metricsHistory.get(metricsHistory.lastKey()).getVertexMetrics().get(vertex);
Expand All @@ -142,6 +168,7 @@ private Map<ScalingMetric, EvaluatedScalingMetric> evaluateMetrics(
double inputRateAvg = getRate(ScalingMetric.NUM_RECORDS_IN, vertex, metricsHistory);

var evaluatedMetrics = new HashMap<ScalingMetric, EvaluatedScalingMetric>();

computeTargetDataRate(
topology,
vertex,
Expand Down Expand Up @@ -175,6 +202,24 @@ private Map<ScalingMetric, EvaluatedScalingMetric> evaluateMetrics(
EvaluatedScalingMetric.of(vertexInfo.getNumSourcePartitions()));

computeProcessingRateThresholds(evaluatedMetrics, conf, processingBacklog, restartTime);

Optional.ofNullable(customEvaluationSession)
.map(
session ->
runCustomEvaluator(
vertex,
Collections.unmodifiableMap(evaluatedMetrics),
session))
.filter(customEvaluatedMetrics -> !customEvaluatedMetrics.isEmpty())
.ifPresent(
customEvaluatedMetrics -> {
LOG.info(
"Merging custom evaluated metrics for vertex {}: {}",
vertex,
customEvaluatedMetrics);
mergeEvaluatedMetricsMaps(evaluatedMetrics, customEvaluatedMetrics);
});

return evaluatedMetrics;
}

Expand Down Expand Up @@ -585,4 +630,51 @@ protected static double computeEdgeDataRate(
to);
return getRate(ScalingMetric.NUM_RECORDS_OUT, from, metricsHistory);
}

@VisibleForTesting
protected static Map<ScalingMetric, EvaluatedScalingMetric> runCustomEvaluator(
JobVertexID vertex,
Map<ScalingMetric, EvaluatedScalingMetric> evaluatedMetrics,
Tuple2<CustomEvaluator, CustomEvaluator.Context> customEvaluationSession) {
try {
return customEvaluationSession.f0.evaluateVertexMetrics(
vertex, evaluatedMetrics, customEvaluationSession.f1);
} catch (UnsupportedOperationException e) {
LOG.warn(
"Custom evaluator {} tried accessing an un-modifiable view.",
customEvaluationSession.f0.getClass(),
e);
} catch (Exception e) {
LOG.warn(
"Custom evaluator {} threw an exception.",
customEvaluationSession.f0.getClass(),
e);
}

return Collections.emptyMap();
}

@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)));
}

@VisibleForTesting
protected static EvaluatedScalingMetric mergeEvaluatedScalingMetric(
EvaluatedScalingMetric actual, EvaluatedScalingMetric incoming) {
return new EvaluatedScalingMetric(
!Double.isNaN(incoming.getCurrent()) ? incoming.getCurrent() : actual.getCurrent(),
!Double.isNaN(incoming.getAverage()) ? incoming.getAverage() : actual.getAverage());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import org.apache.flink.autoscaler.metrics.MetricAggregator;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DelegatingConfiguration;
import org.apache.flink.configuration.MemorySize;

import java.time.Duration;
Expand All @@ -31,6 +33,7 @@ public class AutoScalerOptions {

public static final String OLD_K8S_OP_CONF_PREFIX = "kubernetes.operator.";
public static final String AUTOSCALER_CONF_PREFIX = "job.autoscaler.";
public static final String CUSTOM_EVALUATOR_CONF_PREFIX = "metrics.custom-evaluator.";

private static String oldOperatorConfigKey(String key) {
return OLD_K8S_OP_CONF_PREFIX + AUTOSCALER_CONF_PREFIX + key;
Expand Down Expand Up @@ -382,4 +385,19 @@ private static ConfigOptions.OptionBuilder autoScalerConfig(String key) {
"scaling.key-group.partitions.adjust.mode"))
.withDescription(
"How to adjust the parallelism of Source vertex or upstream shuffle is keyBy");

public static final ConfigOption<String> CUSTOM_EVALUATOR_NAME =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about deriving the name from the CustomEvaluator interface? We can load all the names when we load the custom evaluator implementations.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks so much for your inputs! That Sounds perfect! Will make the update!

autoScalerConfig(CUSTOM_EVALUATOR_CONF_PREFIX + "name")
.stringType()
.defaultValue(null)
.withFallbackKeys(oldOperatorConfigKey(CUSTOM_EVALUATOR_CONF_PREFIX + "name"))
.withDescription("Name of the custom evaluator to be used.");

public static Configuration forCustomEvaluator(
Configuration configuration, String customEvaluatorName) {
// add support for fallBackKey with DelegatingConfiguration.
return new DelegatingConfiguration(
configuration,
AUTOSCALER_CONF_PREFIX + CUSTOM_EVALUATOR_CONF_PREFIX + customEvaluatorName + ".");
}
}
Loading