1996fanrui commented on code in PR #762:
URL:
https://github.com/apache/flink-kubernetes-operator/pull/762#discussion_r1477167226
##########
flink-autoscaler/src/main/java/org/apache/flink/autoscaler/state/AutoScalerStateStore.java:
##########
@@ -69,6 +70,13 @@ void storeParallelismOverrides(Context jobContext,
Map<String, String> paralleli
void removeParallelismOverrides(Context jobContext) throws Exception;
+ void storeConfigOverrides(Context jobContext, Configuration
configOverrides) throws Exception;
Review Comment:
Currently, this PR only stores some configs that applying on the regular
config. So all of them are `configOverrides`.
What I mean is that we may store some other configurations in the future,
some of them may not apply to jobs. These configurations are only used during
the scaling. The naming of configOverrides may not be appropriate for the
storage of these configurations. What do you think?
Note: I don't have a strong opinion about this comment, we can rename it in
the future if there are changes.
##########
flink-autoscaler/src/main/java/org/apache/flink/autoscaler/JobAutoScalerContext.java:
##########
@@ -56,15 +58,21 @@ public class JobAutoScalerContext<KEY> {
@Getter private final MetricGroup metricGroup;
- /** Task manager CPU as a fraction (if available). */
- @Getter private final double taskManagerCpu;
-
- /** Task manager memory size (if available). */
- @Getter @Nullable private final MemorySize taskManagerMemory;
-
@ToString.Exclude
private final SupplierWithException<RestClusterClient<String>, Exception>
restClientSupplier;
+ /** Retrieve the currently configured TaskManager CPU for this context. */
+ public Optional<Double> getTaskManagerCpu() {
+ // Not supported by default
+ return Optional.empty();
+ }
+
+ /** Retrieve the currently tuned TaskManager memory for this context. */
Review Comment:
```suggestion
/** Retrieve the currently configured TaskManager memory for this
context. */
```
IIUC, it's configured TaskManager memory instead of tuned, right?
##########
flink-autoscaler/src/main/java/org/apache/flink/autoscaler/utils/MemoryTuningUtils.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.config.AutoScalerOptions;
+import org.apache.flink.autoscaler.metrics.EvaluatedMetrics;
+import org.apache.flink.autoscaler.metrics.ScalingMetric;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import
org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverheadOptions;
+import org.apache.flink.runtime.util.config.memory.ProcessMemoryOptions;
+import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils;
+import
org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemory;
+import
org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemoryUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+import static org.apache.commons.lang3.math.Fraction.getFraction;
+
+/** Tunes the TaskManager memory. */
+public class MemoryTuningUtils {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(MemoryTuningUtils.class);
+ public static final ProcessMemoryUtils<TaskExecutorFlinkMemory>
FLINK_MEMORY_UTILS =
+ new ProcessMemoryUtils<>(getMemoryOptions(), new
TaskExecutorFlinkMemoryUtils());
+
+ /**
+ * Emits a Configuration which contains overrides for the current
configuration. We are not
+ * modifying the config directly, but we are emitting a new configuration
which contains any
+ * overrides. This config is persisted separately and applied by the
autoscaler. That way we can
+ * clear any applied overrides if auto-tuning is disabled.
+ */
+ public static Configuration tuneTaskManagerHeapMemory(
+ JobAutoScalerContext<?> context, EvaluatedMetrics
evaluatedMetrics) {
+
+ // Please note that this config is the original configuration created
from the user spec.
+ // It does not contain any already applied overrides.
+ var config = new UnmodifiableConfiguration(context.getConfiguration());
Review Comment:
I tried to debug it, I found it's not the original configuration.
I run the `autoscaling.yaml`, the original TM total memory is 2GB. After
tuning, it's changed to about 1.85 GB. And the taskHeap is reduced to 512 MB.
I found the config and memSpecs is changed.
<img width="1754" alt="image"
src="https://github.com/apache/flink-kubernetes-operator/assets/38427477/cb275663-9e30-4ed1-a3b4-9a8e3cf03182">
##########
flink-autoscaler/src/main/java/org/apache/flink/autoscaler/utils/MemoryTuningUtils.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.config.AutoScalerOptions;
+import org.apache.flink.autoscaler.metrics.EvaluatedMetrics;
+import org.apache.flink.autoscaler.metrics.ScalingMetric;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import
org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverheadOptions;
+import org.apache.flink.runtime.util.config.memory.ProcessMemoryOptions;
+import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils;
+import
org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemory;
+import
org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemoryUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+import static org.apache.commons.lang3.math.Fraction.getFraction;
+
+/** Tunes the TaskManager memory. */
+public class MemoryTuningUtils {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(MemoryTuningUtils.class);
+ public static final ProcessMemoryUtils<TaskExecutorFlinkMemory>
FLINK_MEMORY_UTILS =
+ new ProcessMemoryUtils<>(getMemoryOptions(), new
TaskExecutorFlinkMemoryUtils());
+
+ /**
+ * Emits a Configuration which contains overrides for the current
configuration. We are not
+ * modifying the config directly, but we are emitting a new configuration
which contains any
+ * overrides. This config is persisted separately and applied by the
autoscaler. That way we can
+ * clear any applied overrides if auto-tuning is disabled.
+ */
+ public static Configuration tuneTaskManagerHeapMemory(
+ JobAutoScalerContext<?> context, EvaluatedMetrics
evaluatedMetrics) {
+
+ // Please note that this config is the original configuration created
from the user spec.
+ // It does not contain any already applied overrides.
+ var config = new UnmodifiableConfiguration(context.getConfiguration());
+ if
(!context.getConfiguration().get(AutoScalerOptions.MEMORY_TUNING_ENABLED)) {
+ return config;
+ }
+
+ var globalMetrics = evaluatedMetrics.getGlobalMetrics();
+ MemorySize avgHeapSize =
+ new MemorySize(
+ (long)
globalMetrics.get(ScalingMetric.HEAP_AVERAGE_SIZE).getAverage());
+ LOG.info("Average TM used heap size: {}", avgHeapSize);
+
+ // Gather original memory configuration from the user spec
+ var memSpecs = FLINK_MEMORY_UTILS.memoryProcessSpecFromConfig(config);
+ var maxHeapSize = memSpecs.getFlinkMemory().getJvmHeapMemorySize();
+ LOG.info("Current configured heap size: {}", maxHeapSize);
+
+ // Apply min/max heap size limits
+ MemorySize newHeapSize =
+ new MemorySize(
+ Math.min(
+ // Upper limit is the original max heap size
in the spec
+ maxHeapSize.getBytes(),
+ Math.max(
+ // Lower limit is the minimum
configured heap size
+
config.get(AutoScalerOptions.MEMORY_TUNING_MIN_HEAP)
+ .getBytes(),
+ avgHeapSize.getBytes())));
+ LOG.info("New TM heap memory {}", newHeapSize.toHumanReadableString());
+
+ // Diff can be negative (memory shrinks) or positive (memory grows)
+ final long heapDiffBytes = newHeapSize.getBytes() -
maxHeapSize.getBytes();
+
+ final MemorySize totalMemory = adjustTotalTmMemory(context,
heapDiffBytes);
+ if (totalMemory.equals(MemorySize.ZERO)) {
+ return config;
+ }
+
+ // Prepare the tuning config for new configuration values
+ var tuningConfig = new Configuration();
+ // Update total memory according to new heap size
+ // Adjust the total container memory and the JVM heap size accordingly.
+ tuningConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, totalMemory);
+ // Framework and Task heap memory configs add up together yield the
max heap memory.
+ // To simplify the calculation, set the framework heap memory to zero.
+ tuningConfig.set(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY,
MemorySize.ZERO);
+ tuningConfig.set(TaskManagerOptions.TASK_HEAP_MEMORY, newHeapSize);
+
+ // All memory options which can be configured via fractions need to be
set to their
+ // absolute values or, if there is no absolute setting, the fractions
need to be
+ // re-calculated.
+ MemorySize managedMemory = memSpecs.getFlinkMemory().getManaged();
+ if (shouldTransferHeapToManagedMemory(config, heapDiffBytes)) {
+ // If RocksDB is configured, give back the heap memory as managed
memory to RocksDB
+ MemorySize newManagedMemory =
+ new MemorySize(managedMemory.getBytes() +
Math.abs(heapDiffBytes));
+ LOG.info(
+ "Increasing managed memory size from {} to {}",
+ managedMemory,
+ newManagedMemory);
+ tuningConfig.set(TaskManagerOptions.MANAGED_MEMORY_SIZE,
newManagedMemory);
+ } else {
+ tuningConfig.set(TaskManagerOptions.MANAGED_MEMORY_SIZE,
managedMemory);
+ }
+
+ tuningConfig.set(
+ TaskManagerOptions.NETWORK_MEMORY_FRACTION,
+ getFraction(
+ memSpecs.getFlinkMemory().getNetwork(),
+ new MemorySize(
+ memSpecs.getTotalFlinkMemorySize().getBytes()
+ heapDiffBytes)));
+ tuningConfig.set(
+ TaskManagerOptions.JVM_OVERHEAD_FRACTION,
+ getFraction(memSpecs.getJvmOverheadSize(), totalMemory));
+
+ return tuningConfig;
+ }
+
+ private static boolean shouldTransferHeapToManagedMemory(
Review Comment:
As I discussed in the design doc, would you mind introducing a option to
control it?
--
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]