gyfora commented on code in PR #762: URL: https://github.com/apache/flink-kubernetes-operator/pull/762#discussion_r1464969261
########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/ScalingExecutor.java: ########## @@ -131,6 +141,12 @@ public boolean scaleResource( getVertexParallelismOverrides( evaluatedMetrics.getVertexMetrics(), scalingSummaries)); + if (tmHeapMemoryOpt.isPresent()) { + Configuration configOverrides = autoScalerStateStore.getConfigOverrides(context); + configOverrides.set(TaskManagerOptions.TASK_HEAP_MEMORY, tmHeapMemoryOpt.get()); Review Comment: I wonder what will happen if this is too big and the user also defined other memory options like managed fraction etc. We may create a config that simply won't run / make sense ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/utils/MemoryTuningUtils.java: ########## @@ -0,0 +1,132 @@ +/* + * 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.utils; + +import org.apache.flink.autoscaler.JobAutoScalerContext; +import org.apache.flink.autoscaler.ScalingSummary; +import org.apache.flink.autoscaler.config.AutoScalerOptions; +import org.apache.flink.autoscaler.metrics.EvaluatedMetrics; +import org.apache.flink.autoscaler.metrics.EvaluatedScalingMetric; +import org.apache.flink.autoscaler.metrics.ScalingMetric; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Optional; + +/** Tunes the TaskManager memory. */ +public class MemoryTuningUtils { + + private static final Logger LOG = LoggerFactory.getLogger(MemoryTuningUtils.class); + + public static Optional<MemorySize> tuneTaskManagerHeapMemory( + JobAutoScalerContext<?> context, + EvaluatedMetrics evaluatedMetrics, + Map<JobVertexID, ScalingSummary> scalingSummaries) { + + var config = context.getConfiguration(); + if (!config.get(AutoScalerOptions.MEMORY_TUNING_ENABLED)) { + return Optional.empty(); + } + + var globalMetrics = evaluatedMetrics.getGlobalMetrics(); + double avgHeapSize = globalMetrics.get(ScalingMetric.HEAP_AVERAGE_SIZE).getAverage(); + + double numTaskSlotsUsed = globalMetrics.get(ScalingMetric.NUM_TASK_SLOTS_USED).getCurrent(); + int taskSlotsPerTm = config.get(TaskManagerOptions.NUM_TASK_SLOTS); + int currentNumTMs = (int) Math.ceil(numTaskSlotsUsed / taskSlotsPerTm); + + double usedTotalHeapSize = currentNumTMs * avgHeapSize; + LOG.info("Total used heap size: {}", new MemorySize((long) usedTotalHeapSize)); + usedTotalHeapSize *= computeDataChangeRate(evaluatedMetrics); + LOG.info("Resized total heap size: {}", new MemorySize((long) usedTotalHeapSize)); + + int numTaskSlotsAfterRescale = + ResourceCheckUtils.estimateNumTaskSlotsAfterRescale( + evaluatedMetrics, scalingSummaries, numTaskSlotsUsed); + int newNumTms = (int) Math.ceil(numTaskSlotsAfterRescale / (double) taskSlotsPerTm); + LOG.info( + "Estimating {} task slots in use after rescale, spread across {} TaskManagers", + numTaskSlotsAfterRescale, + newNumTms); + + MemorySize newHeapSize = new MemorySize((long) (usedTotalHeapSize / newNumTms)); + // TM container memory can never grow beyond the user-specified max + Optional<MemorySize> maxMemory = context.getTaskManagerMemoryFromSpec(); + if (maxMemory.isEmpty()) { + return Optional.empty(); + } + // Apply limits + newHeapSize = + new MemorySize( + Math.min( + maxMemory.get().getBytes(), + Math.max( + config.get(AutoScalerOptions.MEMORY_TUNING_MIN_HEAP) + .getBytes(), + newHeapSize.getBytes()))); Review Comment: Furthermore the user may have configured a specific heap size directly in the conf ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/utils/MemoryTuningUtils.java: ########## @@ -0,0 +1,132 @@ +/* + * 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.utils; + +import org.apache.flink.autoscaler.JobAutoScalerContext; +import org.apache.flink.autoscaler.ScalingSummary; +import org.apache.flink.autoscaler.config.AutoScalerOptions; +import org.apache.flink.autoscaler.metrics.EvaluatedMetrics; +import org.apache.flink.autoscaler.metrics.EvaluatedScalingMetric; +import org.apache.flink.autoscaler.metrics.ScalingMetric; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Optional; + +/** Tunes the TaskManager memory. */ +public class MemoryTuningUtils { + + private static final Logger LOG = LoggerFactory.getLogger(MemoryTuningUtils.class); + + public static Optional<MemorySize> tuneTaskManagerHeapMemory( + JobAutoScalerContext<?> context, + EvaluatedMetrics evaluatedMetrics, + Map<JobVertexID, ScalingSummary> scalingSummaries) { + + var config = context.getConfiguration(); + if (!config.get(AutoScalerOptions.MEMORY_TUNING_ENABLED)) { + return Optional.empty(); + } + + var globalMetrics = evaluatedMetrics.getGlobalMetrics(); + double avgHeapSize = globalMetrics.get(ScalingMetric.HEAP_AVERAGE_SIZE).getAverage(); + + double numTaskSlotsUsed = globalMetrics.get(ScalingMetric.NUM_TASK_SLOTS_USED).getCurrent(); + int taskSlotsPerTm = config.get(TaskManagerOptions.NUM_TASK_SLOTS); + int currentNumTMs = (int) Math.ceil(numTaskSlotsUsed / taskSlotsPerTm); + + double usedTotalHeapSize = currentNumTMs * avgHeapSize; + LOG.info("Total used heap size: {}", new MemorySize((long) usedTotalHeapSize)); + usedTotalHeapSize *= computeDataChangeRate(evaluatedMetrics); + LOG.info("Resized total heap size: {}", new MemorySize((long) usedTotalHeapSize)); + + int numTaskSlotsAfterRescale = + ResourceCheckUtils.estimateNumTaskSlotsAfterRescale( + evaluatedMetrics, scalingSummaries, numTaskSlotsUsed); + int newNumTms = (int) Math.ceil(numTaskSlotsAfterRescale / (double) taskSlotsPerTm); + LOG.info( + "Estimating {} task slots in use after rescale, spread across {} TaskManagers", + numTaskSlotsAfterRescale, + newNumTms); + + MemorySize newHeapSize = new MemorySize((long) (usedTotalHeapSize / newNumTms)); + // TM container memory can never grow beyond the user-specified max + Optional<MemorySize> maxMemory = context.getTaskManagerMemoryFromSpec(); + if (maxMemory.isEmpty()) { + return Optional.empty(); + } + // Apply limits + newHeapSize = + new MemorySize( + Math.min( + maxMemory.get().getBytes(), + Math.max( + config.get(AutoScalerOptions.MEMORY_TUNING_MIN_HEAP) + .getBytes(), + newHeapSize.getBytes()))); Review Comment: is it correct to compare here against `maxMemory`? `getTaskManagerMemoryFromSpec` returns the total process memory size, and only a small part of that may be heap. -- 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