[ 
https://issues.apache.org/jira/browse/FLINK-30081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17720687#comment-17720687
 ] 

Mingliang Liu commented on FLINK-30081:
---------------------------------------

I checked again and found we have already been logging the unused memory 
configuration options at WARN level. However there is a small bug in the 
logging message. We also adjust memory configuration options for local 
execution, but for JVM overhead min/max, the logic is just for missing 
configuration options. I filed [PR 
#22547|https://github.com/apache/flink/pull/22547] to improve logging message 
and reset those unused options so local execution will not fail. This 
essentially takes [~xtsong]'s suggestion. I added a unit test and the same 
snippet in the Jira summary now also pass.

> 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
>            Priority: Major
>              Labels: pull-request-available
>
> 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_MIN.key(), "1GB");
> conf.setString(TaskManagerOptions.JVM_OVERHEAD_MAX.key(), "2GB");
> StreamExecutionEnvironment.createLocalEnvironment(conf)
>     .fromElements("Hello", "World")
>     .executeAndCollect()
>     .forEachRemaining(System.out::println); {code}
> The failing exception is something like:
> {code:java}
> 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?



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to