[ https://issues.apache.org/jira/browse/FLINK-30081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17645077#comment-17645077 ]
Mingliang Liu commented on FLINK-30081: --------------------------------------- [~xtsong] Thanks for providing more context. I'd love to provide a patch by improving {{adjustForLocalExecution}} method. The pointer here is very helpful! [~chesnay] I think it is an option to just fail the MiniCluster. However, if we choose to fail the MiniCluster, it's inconsistent to just fail when the min and max value do not match. For example, if you modify the sample code above and set the min/max to the exactly same value, it will not fail or throw exceptions. In the [doc|https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/memory/mem_setup_tm/#local-execution], we have listed all supported memory settings for local execution. I'd assume they would check that before tuning the parameters for local mode if they intend to do that. However, in my daily job, I don't met users who intend to test memory setting for local mode. But some of them still see the problem because their job inherit the default configs that are tuned for production - either via config files or Spring Boot properties. If the inherited min/max settings differ, the local job fails; if the min/max are the same, the job just runs locally. That makes me think MiniCluster failing would be intrusive and I prefer an ERROR logging and just ignore as suggested above. Thoughts? Thanks! > 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 > > 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)