Rob Young created FLINK-36646: --------------------------------- Summary: Support Java 17+ job images more conveniently in the operator Key: FLINK-36646 URL: https://issues.apache.org/jira/browse/FLINK-36646 Project: Flink Issue Type: Improvement Components: Kubernetes Operator Reporter: Rob Young
Currently if I execute the operator quickstart, but change the FlinkDeployment image to one of the official java17 based images like `flink:1.20.0-scala_2.12-java17`: {code:java} apiVersion: flink.apache.org/v1beta1 kind: FlinkDeployment metadata: name: basic-example spec: image: flink:1.20.0-scala_2.12-java17 flinkVersion: v1_20 flinkConfiguration: taskmanager.numberOfTaskSlots: "2" serviceAccount: flink jobManager: resource: memory: "2048m" cpu: 1 taskManager: resource: memory: "2048m" cpu: 1 job: jarURI: local:///opt/flink/examples/streaming/StateMachineExample.jar parallelism: 2 upgradeMode: stateless {code} then the application fails with exception, eventually caused by: {code:java} aused by: java.lang.reflect.InaccessibleObjectException: Unable to make field private final java.util.Map java.util.Collections$UnmodifiableMap.m accessible: module java.base does not "opens java.util" to unnamed module @21a947fe at java.lang.reflect.AccessibleObject.checkCanSetAccessible(Unknown Source) ~[?:?] at java.lang.reflect.AccessibleObject.checkCanSetAccessible(Unknown Source) ~[?:?] at java.lang.reflect.Field.checkCanSetAccessible(Unknown Source) ~[?:?] at java.lang.reflect.Field.setAccessible(Unknown Source) ~[?:?] at org.apache.flink.streaming.runtime.translators.DataStreamV2SinkTransformationTranslator.registerSinkTransformationTranslator(DataStreamV2SinkTransformationTranslator.java:104) ~[flink-dist-1.20.0.jar:1.20.0] at org.apache.flink.datastream.impl.ExecutionEnvironmentImpl.<clinit>(ExecutionEnvironmentImpl.java:96) ~[flink-dist-1.20.0.jar:1.20.0] at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:109) ~[flink-dist-1.20.0.jar:1.20.0] at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.runApplicationEntryPoint(ApplicationDispatcherBootstrap.java:301) ~[flink-dist-1.20.0.jar:1.20.0] at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.lambda$runApplicationAsync$2(ApplicationDispatcherBootstrap.java:254) ~[flink-dist-1.20.0.jar:1.20.0] at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) ~[?:?] at java.util.concurrent.FutureTask.run(Unknown Source) ~[?:?] at org.apache.flink.runtime.concurrent.pekko.ActorSystemScheduledExecutorAdapter$ScheduledFutureTask.run(ActorSystemScheduledExecutorAdapter.java:172) ~[?:?] at org.apache.flink.runtime.concurrent.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68) ~[flink-dist-1.20.0.jar:1.20.0] at org.apache.flink.runtime.concurrent.ClassLoadingUtils.lambda$withContextClassLoader$0(ClassLoadingUtils.java:41) ~[flink-dist-1.20.0.jar:1.20.0] ... 7 more {code} the user can fix this by setting: {code:java} env.java.opts.all: "--add-exports=java.base/sun.net.util=ALL-UNNAMED --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.text=ALL-UNNAMED --add-opens=java.base/java.time=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED"{code} in the FlinkDeployment flinkConfiguration, matching the [default configuration from the flink binary distribution|https://github.com/apache/flink/blob/b1fe7b4099497f02b4658df7c3de8e45b62b7e21/flink-dist/src/main/resources/config.yaml#L24] It would be cool if you could use Java 17+ based job images without having to set this up and the operator or flink shell scripts could set this up for you. >From flink 1.18 the config.sh script is setting >`-XX:+IgnoreUnrecognizedVMOptions` so it would appear safe to redundantly set >these JVM params for Flink 1_18+. Maybe if there is no user-supplied >`env.java.opts.all` the operator could set it, else append them to the >user-supplied opts. I'm wondering if this should be a core flink responsibility though, maybe `config.sh` could set these parameters, recent versions of config.sh are already sniffing for JVM 17+ so that it can set `-Djava.security.manager=allow`, maybe it would be appropriate for it to also set the add-opens up too? -- This message was sent by Atlassian Jira (v8.20.10#820010)