Mingliang Liu created FLINK-30081:
-------------------------------------
Summary: Local executor can not accept different
jvm-overhead.min/max values
Key: FLINK-30081
URL: https://issues.apache.org/jira/browse/FLINK-30081
Project: Flink
Issue Type: Improvement
Components: Runtime / Configuration
Affects Versions: 1.16.0
Reporter: Mingliang Liu
In local executor, it's not possible to set different values for
{{taskmanager.memory.jvm-overhead.max}} and
{{{}taskmanager.memory.jvm-overhead.min{}}}. The same problem for
{{taskmanager.memory.network.max}} and {{taskmanager.memory.network.min}}.
Sample code to reproduce:
{code:java}
Configuration conf = new Configuration();
conf.setString(TaskManagerOptions.JVM_OVERHEAD_MAX.key(), "1GB");
conf.setString(TaskManagerOptions.JVM_OVERHEAD_MIN.key(), "2GB");
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.createLocalEnvironment(conf);
env.fromElements("Hello", "World")
.executeAndCollect()
.forEachRemaining(System.out::println);
{code}
The failing exception is something like:
{code}
Exception in thread "main" java.lang.IllegalArgumentException
at org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:122)
at
org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.calculateTotalProcessMemoryFromComponents(TaskExecutorResourceUtils.java:182)
at
org.apache.flink.runtime.taskexecutor.TaskExecutorMemoryConfiguration.create(TaskExecutorMemoryConfiguration.java:119)
{code}
I think the problem was that we expect the max and min to equal, but local
executor did not reset them correctly?
{code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)