Page properties | |||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
Document the state by adding a label to the FLIP page with one of "discussion", "accepted", "released", "rejected".
|
[This FLIP proposal is a joint work between Rui Fan and Samrat Deb , and did some consulting with Gyula Fora Maximilian Michels ]
Table of Contents
1. Motivation
|
Table of Contents
1. Motivation
The proposal to introduce The proposal to introduce autoscaling for Flink (FLIP-271) has garnered significant interest due to its potential to greatly enhance the usability of Flink. The primary objective is to enable users to effortlessly enable the autoscaler for their Flink jobs without the need for intricate parallelism configurations. However, the current implementation of the flink-autoscaler is tightly integrated coupled with Kubernetes and resides within the flink-kubernetes-operator repository.
...
Hence, this FLIP is centered around the crucial task of decoupling the autoscaler functionality from Kubernetes.
By achieving this decoupling, we aim to empower Flink users to leverage the benefits of autoscaling irrespective of their chosen deployment platform, whether it be Kubernetes or YARN.
2. Core Idea
- JobAutoScaler and JobAutoScalerImpl: These components will define the generic autoscaling strategy for Flink jobs and are essential for the implementation of the autoscaler module.
- Interfaces: The FLIP outlines the necessity of defining a few interfaces -
ScalingRealizer, AutoScalerEventHandler
andAutoScalerStateStore
.The ScalingRealizer
interface handles scaling action.- The
AutoScalerEventHandler
interface handles loggable events. - The
AutoScalerStateStore
interface is responsible for accessing and persisting the autoscaler's state.
- Dependencies: The autoscaler module should not rely on any Kubernetesbe agnostic to any specific deployment framework like Kubernetes, YARN etc. Currently, core autoscaling framework closely coupled with Kubernetes-related dependencies such as fabric8, flink-kubernetes-operator, or flink-kubernetes.
InsteadIdeally, it can should rely on Apache Flink project dependencies to gather metrics and make scaling decisions based onJobVertexID
,Configuration
,MetricGroup
, and other relevant classes. - Optional Goal: For flink-autoscaler module, we prefer stay it at flink-k8s-operator first(Advance discussion with Gyula Fora).
a. Since the autoscaler is not only supports compatible with the latest flink Flink version, so moving it to the Flink repository may not be a good choice moving it to flink repo. It needs to work across Flink versions this probably have to be a separate repothe most suitable option.
It would need to function across various Flink versions, potentially necessitating a separate repository/subproject.b. Remove Removing the autoscaler from operator repo will also bring a lot of operational overhead / release process etc which will be a negative from the operator’s perspective.the flink-kubernetes-operator repository would introduce significant operational overhead and complicate the release process,
which would be viewed negatively from the operator's perspective.Therefore, considering above two points preference is to retain core autoscaler framework/logic within c. So, we prefer stay it and flink-yarn-autoscaler as a submodule of the flink-kubernetes-operator as a submodule in the short term. If it's Should it prove to be stable in the future,
we can discuss move them. After it's stablerevisit the idea of relocating it. As it stabilizes, the release frequency will be reduceddecrease, and it will be small negative for this transition will have a minor impact on the operator.
Note:
Independent flink-kubernetes-operator-autoscaler
module is not necessary. Moving classes to flink-kubernetes-operator
will reduce complexity.
Why it isn flink-kubernetes-operator-autoscaler
as seperate module isn't necessary?
...
If flink-kubernetes-operator-autoscaler
as an independent module, it must depend on flink-kubernetes-operator
module.
...
flink-kubernetes-operator
cannot depend on
flink-kubernetes-operator-autoscaler
, so it's more difficult to load these classes than remove the flink-kubernetes-operator-autoscaler
module.
...
flink-kubernetes-operator-autoscaler
...
just defines KubernetesScalingRealizer,
KubernetesAutoScalerEventHandler
and KubernetesAutoScalerStateStore
.
...
There are not many of these classes, so moving them to flink-kubernetes-operator
will reduce complexity.
3. Public Interfaces
3.1 JobAutoScaler
The proposed interface retains similarity maintains a resemblance to the existing JobAutoScaler, although albeit with modified adjustments to its method parameters. Instead In lieu of using employing Kubernetes-related specific classes, the parameters will be replaced substituted with JobAutoScalerContext<KEY>
. The KEY is jobKey, if the KEY is the same, it is considered to be the same flink , rendering it framework-agnostic, suitable for various deployment frameworks beyond Kubernetes.
The 'KEY' in this context corresponds to the 'jobKey', and when two instances share the same 'KEY,' they are regarded as representing the same Flink job.
Code Block |
---|
/** * The generic Autoscaler instance. */ public interface* JobAutoScaler<KEY>@param { <KEY> The job booleankey. */ @Internal public interface JobAutoScaler<KEY, Context extends JobAutoScalerContext<KEY>> { /** Called as part of the reconciliation loop. */ void scale(JobAutoScalerContext<KEY>Context context) throws Exception; /** Called when the job is deleted. */ void cleanup(JobAutoScalerContext<KEY>KEY contextkey); } |
3.2 JobAutoScalerContext
The JobAutoScalerContext
encapsulates essential information required plays a pivotal role in consolidating crucial information necessary for scaling Flink jobs, including . It encompasses essential data such as the jobKey, jobId, configuration and , MetricGroup, etcand more.
Currently as of now, in the existing code or for Kubernetes jobs, the jobKey is defined as using io.javaoperatorsdk.operator.processing.event.ResourceID
.
However, there is a possibility to define , as seen in the existing code and for Kubernetes jobs. However, it's important to note that there may be potential future developments where the jobKey for Flink jobs running operating on YARN in the futurecould differ.
The entire JobAutoScalerContext
, comprising encompassing all relevant pertinent details, will be passed furnished to these implementations. This comprehensive context will be provided when the autoscaler invokes their respective callbacks.
...
Code Block |
---|
/** * The job autoscaler context, it includes all details related to the current job. * * @param <KEY> The job key. */ @Experimental @AllArgsConstructor @ToString public interfaceclass JobAutoScalerContext<KEY> { //** The identifier of each flink job. */ @Getter private final KEY getJobKey()jobKey; JobID getJobID(); Configuration getConfiguration(); MetricGroup getMetricGroup(); /** The jobId and jobStatus can be null when the job isn't started. */ @Nullable @Getter private final JobID jobID; RestClusterClient<String> getRestClusterClient() throws Exception@Nullable @Getter private final JobStatus jobStatus; } |
3.3 ScalingRealizer
The ScalingRealizer interface handles scaling action, and the ScalingRealizer#realize
will be called inside of the JobAutoscaler#scale
.
Code Block |
---|
public interface ScalingRealizer<KEY, @Getter private final Configuration configuration; @Getter private final Context extends JobAutoScalerContext<KEY>> {MetricGroup metricGroup; void realize(Context context, @ToString.Exclude private final SupplierWithException<RestClusterClient<String>, Exception> restClientSupplier; public RestClusterClient<String> getRestClusterClient() throws Exception { Map<String, String> parallelismOverrides return restClientSupplier.get(); } } |
3.
...
3 ScalingRealizer
The AutoScalerEventHandler
interface handles loggable events, and AutoScalerEventHandler#handleEvent
will be called by auto scaler when some loggable events need to be handled, such as: scaling error, report scaling result, etc.
The AutoScalerEventHandler
object is shared for all flink jobs, it doesn't have the job information, that's why the JobAutoScalerContext as the parameter of handleEvent.ScalingRealizer interface is responsible for managing scaling actions(upscale or downscale), and specifically, the ScalingRealizer#realize
method will be invoked from JobAutoscaler#scale
function.
Code Block |
---|
/** * Handler all loggable events during scalingThe Scaling Realizer is responsible for managing scaling actions. * * @param <KEY> */ public interface AutoScalerEventHandler<KEY,The job key. * @param <Context> Instance of JobAutoScalerContext. */ @Experimental public interface ScalingRealizer<KEY, Context extends JobAutoScalerContext<KEY>> { void handleEvent(Context context, /** Update job's parallelism to parallelismOverrides. */ void realize(Context context, Map<String, Type type, String reason, String message, @Nullable String messageKey); enum Type { Normal, Warning } } |
3.5 AutoScalerStateStore
...
String> parallelismOverrides);
} |
3.4 AutoScalerEventHandler
The AutoScalerEventHandler interface is responsible for managing loggable events, and specifically, the AutoScalerEventHandler#handleEvent
method will be invoked by the auto scaler when there's a need to handle such events. These events might include scaling errors, reporting scaling results, and more.
It's important to note that the AutoScalerEventHandler object is shared across all Flink jobs and doesn't possess specific job information. That's precisely why the JobAutoScalerContext is passed as a parameter to the handleEvent
method, allowing it to access the necessary job-related details when handling events.
Code Block |
---|
/**
* Handler all loggable events during scaling.
*
* @param <KEY> The job key.
* @param <Context> Instance of JobAutoScalerContext.
*/
@Experimental
public interface AutoScalerEventHandler<KEY, Context extends JobAutoScalerContext<KEY>> {
/**
* Handle the event.
*
* @param interval When interval is great than 0, events that repeat within the interval will be
* ignored.
*/
void handleEvent(
Context context,
Type type,
String reason,
String message,
@Nullable String messageKey,
@Nullable Duration interval);
/** The type of the events. */
enum Type {
Normal,
Warning
}
} |
3.5 AutoScalerStateStore
The AutoScalerStateStore serves the crucial role of persisting and providing access to state information during the scaling process.
In the existing code and for Kubernetes jobs, this state is stored in a ConfigMap
. Consequently, the KubernetesAutoScalerStateStore
is responsible for retrieving the ConfigMap before the scaling operation and preserving it after the scaling event.
However, for other types of jobs, such as those running on YARN or in standalone mode, the default behavior involves persisting scaling information in memory itself via introducing new implementation InMemoryAutoScalerStateStore
. It's important to note that, in the future, there is the possibility to pesist in RDBMS or any persistent storage. It can be new implementation such as JdbcAutoScalerStateStore
etc to ensure persistent storage of the state.
These method parameters of AutoScalerStateStore
to the specific class instead of String, such as: Map<JobVertexID, SortedMap<Instant, ScalingSummary>> scalingHistory
. So, all state store stores are responsible for the serialization, deserialization and state store.
Code Block |
---|
/**
* The state store is responsible for storing all state during scaling.
*
* @param <KEY> The job key.
* @param <Context> Instance of JobAutoScalerContext.
*/
@Experimental
public interface AutoScalerStateStore<KEY, Context extends JobAutoScalerContext<KEY>> {
void storeScalingHistory(
Context jobContext, Map<JobVertexID, SortedMap<Instant, ScalingSummary>> scalingHistory)
throws Exception;
Optional<Map<JobVertexID, SortedMap<Instant, ScalingSummary>>> getScalingHistory(
Context jobContext) throws Exception;
void removeScalingHistory(Context jobContext) throws Exception;
void storeEvaluatedMetrics(
|
Code Block |
---|
/** It will be used store state during scaling. */
public interface AutoScalerStateStore<KEY,
Context extends JobAutoScalerContext<KEY>> {
void storeScalingHistory(Context jobContext, String decision);
String getScalingHistory(Context jobContext);
void removeScalingHistory(Context jobContext);
void storeEvaluatedMetrics(Context jobContext, String metrics);
String getEvaluatedMetrics(Context jobContext);
void removeEvaluatedMetrics(Context jobContext);
void storeParallelismOverrides(Context jobContext, String parallelismOverrides);
String getParallelismOverrides(Context jobContext);
void removeParallelismOverrides(Context jobContext);
void removeInfoFromCache(Context jobContext);
// The flush is needed because we just save data in cache for all store methods, and flush these data to the physical storage after the flush is called to improve the performance
void flush(Context jobContext);
} |
3.6 The generic autoscaler
As discussed with Gyula Fora and Samrat Deb , we don't implement the yarn autoscaler at this FLIP, we can just implement a generic autoscaler based on rescale api(FLIP-291). The generic autoscaler doesn't know any job, and users can pass the JobAutoScalerContext to using the generic autoscaler. It can communicate with flink job through RestClusterClient.
4. Proposed Changes
4.1 Ensure new autoscaler module keeps the generic auto scaling strategy.
...
Using the RestClusterClient
instead of org.apache.flink.kubernetes.operator.service.FlinkService
- The FlinkService is related to kubernetes, so we shouldn't use it.The RestClusterClient is generic flink client, it supports all flink types, including: kubernetes, yarn, standalone.
The RestClusterClient<String> is included in JobAutoScalerContext.
4.2 KubernetesJobAutoScalerContext
Note: some code can be extracted into a AbstractJobAutoScalerContext, such as: jobKey, jobId, configuration, metric group and restClieentSupplier.
These logic should be generic for k8s, yarn and standalone.
Code Block |
---|
public class KubernetesJobAutoScalerContext implements JobAutoScalerContext<ResourceID> { private final ResourceID resourceID; private final JobID jobID; private final Configuration configuration; private final MetricGroup metricGroup; private final SupplierWithException<RestClusterClient<String>, Exception> restClientSupplier; private final AbstractFlinkResource<?, ?> resource; public KubernetesJobAutoScalerContext( ResourceID resourceID, JobID jobID, Configuration configuration, MetricGroupContext metricGroupjobContext, SortedMap<Instant, CollectedMetrics> evaluatedMetrics) SupplierWithException<RestClusterClient<String>, Exception> restClientSupplier, throws Exception; Optional<SortedMap<Instant, CollectedMetrics>> getEvaluatedMetrics(Context jobContext) AbstractFlinkResource<?, ?> resource) { this.resourceID =throws resourceIDException; void removeEvaluatedMetrics(Context jobContext) this.jobID = jobID;throws Exception; void storeParallelismOverrides(Context this.configuration = configuration;jobContext, Map<String, String> parallelismOverrides) this.metricGroup = metricGroup; throws Exception; this.restClientSupplier = restClientSupplier; Optional<Map<String, String>> getParallelismOverrides(Context jobContext) throws Exception; this.resource = resource; } @Overridevoid removeParallelismOverrides(Context jobContext) throws Exception; /** public ResourceID getJobKey() { return resourceID; } @Override public JobID getJobID() {* Flushing is needed because we just save data in cache for all store methods. For less write * operations, returnwe jobID; flush the cached } data to the @Override physical storage only publicafter Configurationall getConfiguration()operations {have * return configuration;been performed. } @Override*/ public MetricGroupvoid getMetricGroupflush(Context jobContext) { return metricGroupthrows Exception; } /** Clean @Override up all information publicrelated RestClusterClient<String>to getRestClusterClient()the throwscurrent Exceptionjob. {*/ void return restClientSupplier.get(removeInfoFromCache(KEY jobKey); } public AbstractFlinkResource<?, ?> getResource() { return resource; } } |
4.3 KubernetesScalingRealizer
Code Block |
---|
public class KubernetesScalingRealizer
implements ScalingRealizer<ResourceID, KubernetesJobAutoScalerContext> {
@Override
public void realize(
KubernetesJobAutoScalerContext context,
Map<String, String> parallelismOverrides) {
spec.getFlinkConfiguration().put(
PipelineOptions.PARALLELISM_OVERRIDES.key(),
ConfigurationUtils.convertValue(parallelismOverrides, String.class));
context.getResource().getStatus().setImmediateReconciliationNeeded(true);
}
} |
4.4 KubernetesAutoScalerEventHandler
} |
4. Proposed Changes
4.1 Ensure new autoscaler module keeps the generic auto scaling strategy.
It includes JobAutoScalerImpl
, ScalingMetricCollector
, AutoScalerInfo,
ScalingExecutor
etc.
kubernetes related dependencies should be removed from these classes and use JobAutoScalerContext
, ScalingRealizer
, AutoScalerEventHandler
and AutoScalerStateStore
instead.
Using the RestClusterClient
instead of org.apache.flink.kubernetes.operator.service.FlinkService
- The FlinkService is related to kubernetes, so we shouldn't use it.The RestClusterClient is generic flink client, it supports all flink types, including: kubernetes, yarn, standalone.
The RestClusterClient<String> is included in JobAutoScalerContext.
4.2 KubernetesJobAutoScalerContext
Note: some code can be extracted into a AbstractJobAutoScalerContext, such as: jobKey, jobId, configuration, metric group and restClieentSupplier.
These logic should be generic for k8s, yarn and standalone.
Code Block |
---|
/** An implementation of JobAutoscalerContext for Kubernetes. */
public class KubernetesJobAutoScalerContext extends JobAutoScalerContext<ResourceID> {
private final AbstractFlinkResource<?, ?> resource;
private final KubernetesClient kubernetesClient;
public KubernetesJobAutoScalerContext(
@Nullable JobID jobID |
Code Block |
public class KubernetesAutoScalerEventHandler implements AutoScalerEventHandler<ResourceID, KubernetesJobAutoScalerContext> { private final EventRecorder eventRecorder; public KubernetesAutoScalerEventHandler(EventRecorder eventRecorder) { this.eventRecorder = eventRecorder; } @Override public void handleEvent(KubernetesJobAutoScalerContext context, @Nullable JobStatus jobStatus, Configuration Type typeconfiguration, MetricGroup metricGroup, SupplierWithException<RestClusterClient<String>, Exception> String reasonrestClientSupplier, AbstractFlinkResource<?, ?> resource, KubernetesClient StringkubernetesClient) message,{ super( ResourceID.fromResource(resource), @Nullable String messageKey) { eventRecorder.triggerEvent( jobID, context.getResource()jobStatus, EventRecorder.Type.valueOf(type.name()), configuration, reasonmetricGroup, EventRecorder.Component.Operator, restClientSupplier); this.resource = resource; this.kubernetesClient = kubernetesClient; message,} public AbstractFlinkResource<?, ?> getResource() { return messageKey)resource; } } |
4.5 KubernetesAutoScalerStateStore
For current code or kubernetes job, the state is persisted to ConfigMap. So the KubernetesAutoScalerStateStore needs to fetch ConfigMap before scaling, and persist the ConfigMap after scaling.
public KubernetesClient getKubernetesClient() {
return kubernetesClient;
}
} |
4.3 KubernetesScalingRealizer
Code Block |
---|
/** The Kubernetes implementation for applying parallelism overrides. */
public class KubernetesScalingRealizer
|
Code Block |
/** The kubernetes auto scaler state store, it's based on the config map. */ public class KubernetesAutoScalerStateStore implements AutoScalerStateStore<ResourceIDScalingRealizer<ResourceID, KubernetesJobAutoScalerContext> { private @Override static final Logger LOGpublic =void LoggerFactory.getLogger(KubernetesAutoScalerStateStore.class); realize( private static final String LABEL_COMPONENT_AUTOSCALER = "autoscaler"; private static final String COLLECTED_METRICS_KEY = "collectedMetrics"; private static final String SCALING_HISTORY_KEY = "scalingHistory"; private static final String PARALLELISM_OVERRIDES_KEY = "parallelismOverrides"; private final KubernetesClient kubernetesClient; private final ConcurrentHashMap<ResourceID, ConfigMap> cache = new ConcurrentHashMap<>(); public KubernetesAutoScalerStateStore(KubernetesClient kubernetesClient) { KubernetesJobAutoScalerContext context, Map<String, String> parallelismOverrides) { context.getResource() .getSpec() .getFlinkConfiguration() this.kubernetesClient = kubernetesClient;.put( } // COLLECTED_METRICS_KEY and PARALLELISM_OVERRIDES_KEY is similar to SCALING_HISTORY_KEY @Override public void storeScalingHistory(KubernetesJobAutoScalerContext jobContext, String decision) { PipelineOptions.PARALLELISM_OVERRIDES.key(), getState(jobContext).put(SCALING_HISTORY_KEY, decision); } @Override public String getScalingHistory(KubernetesJobAutoScalerContext jobContext) { getState(jobContext).get(SCALING_HISTORY_KEYConfigurationUtils.convertValue(parallelismOverrides, String.class)); } } |
4.4 KubernetesAutoScalerEventHandler
Code Block |
---|
/** An event handler which posts events to the Kubernetes events API. */ public class KubernetesAutoScalerEventHandler @Override public void removeScalingHistory(KubernetesJobAutoScalerContext jobContext) { getState(jobContext).remove(SCALING_HISTORY_KEY); } implements AutoScalerEventHandler<ResourceID, KubernetesJobAutoScalerContext> { private Map<String, String> getState(KubernetesJobAutoScalerContext jobContext final EventRecorder eventRecorder; public KubernetesAutoScalerEventHandler(EventRecorder eventRecorder) { return cache.computeIfAbsent(jobContext.getJobKey(), jobKey -> getOrCreateConfigMap(jobContext)).getData(); this.eventRecorder = eventRecorder; } @Override public void flushhandleEvent(KubernetesJobAutoScalerContext jobContext) { ConfigMap configMap = cache.get(jobContext.getJobKey());KubernetesJobAutoScalerContext context, Preconditions.checkState(configMap != null, "The configMap shouldn't be null."); Type type, tryString {reason, cache.put(jobContext.getJobKey(), kubernetesClient.resource(configMap).update()); String message, } catch (Exception@Nullable e)String {messageKey, LOG.error( @Nullable Duration interval) { if (interval == null) { "Error while updating autoscaler info configmap, invalidating to clear the cache", eventRecorder.triggerEvent( e); context.getResource(), removeInfoFromCache(jobContext); EventRecorder.Type.valueOf(type.name()), throw e; }reason, } @Override public void removeInfoFromCache(KubernetesJobAutoScalerContext jobContext) { cache.remove(jobContext.getJobKey());message, } private ConfigMap getOrCreateConfigMap(KubernetesJobAutoScalerContext jobContext) { AbstractFlinkResource<?, ?> cr = jobContext.getResource(); EventRecorder.Component.Operator, var meta = createCmObjectMeta(ResourceID.fromResource(cr)); return getScalingInfoConfigMap(meta).orElseGet(() -> createConfigMap(crmessageKey, meta)); } private ConfigMap createConfigMap(HasMetadata cr, ObjectMeta meta) { LOGcontext.info("Creating scaling info config map"); getKubernetesClient()); } else { var cm = new ConfigMap(); cmeventRecorder.setMetadatatriggerEventByInterval(meta); cm.addOwnerReference(cr); cm.setData(new HashMap<>context.getResource());, return kubernetesClient.resource(cm).create(); EventRecorder.Type.valueOf(type.name()), } private ObjectMeta createCmObjectMeta(ResourceID uid) { var objectMeta =reason, new ObjectMeta(); objectMeta.setName("autoscaler-" + uid.getName()); uid.getNamespace().ifPresent(objectMeta::setNamespace); EventRecorder.Component.Operator, objectMeta.setLabels( Map.of(message, Constants.LABEL_COMPONENT_KEYmessageKey, LABEL_COMPONENT_AUTOSCALERcontext.getKubernetesClient(), Constants.LABEL_APP_KEY,interval); } uid.getName())); return objectMeta; } private Optional<ConfigMap> getScalingInfoConfigMap(ObjectMeta objectMeta) { return Optional.ofNullable( kubernetesClient .configMaps() .inNamespace(objectMeta.getNamespace()) .withName(objectMeta.getName()) .get()); } } |
Compatibility, Deprecation, and Migration Plan
It must be compatible with current kubernetes operator.
Test Plan
UT & IT & Manually verify that the autoscaler is working as expected.
...
}
} |
4.5 KubernetesAutoScalerStateStore
For current code or kubernetes job, the state is persisted to ConfigMap. So the KubernetesAutoScalerStateStore needs to fetch ConfigMap before scaling, and persist the ConfigMap after scaling.
ConfigMapStore is responsible for put/get/remove seralizedStat.
KubernetesAutoScalerStateStore is responsible for serialize state from raw type to String, and deserialize state from String to raw type, it will access state from ConfigMapStore.
5. Standalone AutoScaler
We will implement the StandaloneAutoScaler at this FLIP, it's the generic autoscaler.
5.1 Separate process, JobListFetcher and control loop
The StandaloneAutoScaler can run as a separate process, the StandaloneAutoscalerEntrypoint is the main class of StandaloneAutoScaler.
The StandaloneAutoScaler is not responsible for job management, so there is no job list. In addition to the interfaces mentioned above, we introduced the JobListFetcher interface for StandaloneAutoScaler. The JobListFetcher will provide the job list and the jobContext of all jobs.
Code Block |
---|
/** The JobListFetcher will fetch all job list and the jobContext of all jobs. */
public interface JobListFetcher<KEY> {
List<JobAutoScalerContext<KEY>> fetch(MetricGroup metricGroup);
} |
Control loop
StandaloneAutoscalerEntrypoint call the JobListFetcher to fetch job list, and call the JobAutoscaler#scale for each job peroidically.
We can define the control-loop-interval option for StandaloneAutoscalerEntrypoint.
Note: The kubernetes-autoscaler doesn't need the StandaloneAutoScaler and JobListFetcher, it has the job management and control loop.
5.2 The implemetation of JobListFetcher
For flink standalone cluster, we can implement the StandaloneJobListFetcher. User provide the flink standalone cluster address, and StandaloneJobListFetcher will fetch job list via flink rest api.
For yarn cluster, we can implement the YarnJobListFetcher. User provide the yarn address, and YarnJobListFetcher will fetch job list via yarn rest api and flink rest api.
5.3 The implemetation of ScalingRealizer, AutoScalerEventHandler and AutoScalerStateStore
- We will implement the ScalingApiRealizer, it based on the rescale api of FLIP-291.
- The generic EventHandler based on the logger.
- The generic StateStore based on the Heap. This means that the state information is stored in memory and can be lost if the autoscaler restarts.
- We will implement the JdbcAutoscalerStateStore as well.
6. Compatibility, Deprecation, and Migration Plan
It must be compatible with current kubernetes operator.
7. Test Plan
UT & IT & Manually verify that the autoscaler is working as expected.
Ensure 100% functionality and test coverage of Kubernetes implementation.
8. Rejected Alternatives
Rejected a couple of interface designs, the the whole solution is fine.
yarn-autoscaler
For first version of yarn-autoscaler, we just support rescale job via the rescale api, and don't support re-deploy a new yarn application. Following the some reasons:
If we wanna support the entire yarn deployment, it means we need to implement the yarn job management in the yarn-autoscaler.
Not only rescale, but also start stop keepalive, etc.
- We need to maintains the job jar, job information, etc.
These parts are too heavy, so we just support the rescale api in the first version.
Note: Each company have a flink platform to manage their flink jobs that running on yarn. The flink platform has jobName, job address, job jar, etc. And the flink platform has the ability of start stop keepalive, so these platforms are easy to implement the ScalingRealizer in their cases after this FLIP.