[ https://issues.apache.org/jira/browse/FLINK-36826?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Fabian Paul updated FLINK-36826: -------------------------------- Description: When trying to deploy the following deployment with the operator the job fails to start. {code:java} { "kind": "FlinkDeployment", "metadata": { "name": "basic-example2" }, "spec": { "image": "flink:1.20.0-scala_2.12-java17", "flinkVersion": "v1_20", "flinkConfiguration": { "taskmanager.numberOfTaskSlots": "1", "metrics.reporter.prom.factory.class": "org.apache.flink.metrics.prometheus.PrometheusReporterFactory", "metrics.reporter.prom.port": "9249-9250" }, "serviceAccount": "flink", "jobManager": { "resource": { "memory": "1048m", "cpu": 1 } }, "taskManager": { "resource": { "memory": "1048m", "cpu": 1 } }, "job": { "jarURI": "local:///opt/flink/examples/streaming/StateMachineExample.jar", "state": "running", "parallelism": 3, "upgradeMode": "stateless" } }, "status": null } {code} The job fails to start with {code:java} Caused by: java.lang.ExceptionInInitializerError 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 Caused by: java.lang.RuntimeException: Can not register process function transformation translator. at org.apache.flink.datastream.impl.ExecutionEnvironmentImpl.<clinit>(ExecutionEnvironmentImpl.java:98) ~[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 Caused 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} I did some initial analysis, and it looks like, by default, the operator does not set the env.java.opts.all correctly while running the same image in standalone mode with docker works without issues. Content of flink-conf in k8s pod {code:java} blob.server.port: 6124 kubernetes.jobmanager.annotations: flinkdeployment.flink.apache.org/generation:2 kubernetes.jobmanager.replicas: 1 execution.submit-failed-job-on-application-error: true jobmanager.rpc.address: basic-example2.flink metrics.reporter.prom.factory.class: org.apache.flink.metrics.prometheus.PrometheusReporterFactory kubernetes.service-account: flink kubernetes.cluster-id: basic-example2 kubernetes.taskmanager.cpu.amount: 1.0 $internal.application.program-args: parallelism.default: 3 kubernetes.namespace: flink taskmanager.numberOfTaskSlots: 2 kubernetes.rest-service.exposed.type: ClusterIP kubernetes.jobmanager.owner.reference: controller:false,name:basic-example2,uid:1a31be61-e254-44b2-99ee-211b8482e045,kind:FlinkDeployment,apiVersion:flink.apache.org/v1beta1,blockOwnerDeletion:true kubernetes.container.image.ref: flink:1.20.0-scala_2.12-java17 metrics.reporter.prom.port: 9249-9250 taskmanager.memory.process.size: 1048 mb kubernetes.internal.jobmanager.entrypoint.class: org.apache.flink.kubernetes.entrypoint.KubernetesApplicationClusterEntrypoint pipeline.name: basic-example2 kubernetes.pod-template-file.taskmanager: /tmp/flink_op_generated_podTemplate_7474583206560050522.yaml web.cancel.enable: false execution.target: kubernetes-application jobmanager.memory.process.size: 1048 mb execution.shutdown-on-application-finish: false taskmanager.rpc.port: 6122 kubernetes.jobmanager.cpu.amount: 1.0 internal.cluster.execution-mode: NORMAL $internal.pipeline.job-id: 1dd75be3536073ffa185acb2d1385dbd pipeline.jars: local:///opt/flink/examples/streaming/StateMachineExample.jar execution.checkpointing.externalized-checkpoint-retention: RETAIN_ON_CANCELLATION rest.profiling.enabled: true $internal.flink.version: v1_20 kubernetes.pod-template-file.jobmanager: /tmp/flink_op_generated_podTemplate_2207026818603138122.yaml {code} was: When trying to deploy the following deployment with the operator the job fails to start. {code:java} { "apiVersion": "cmf.confluent.io/v1alpha1", "kind": "FlinkApplication", "metadata": { "name": "basic-example2" }, "spec": { "image": "flink:1.20.0-scala_2.12-java17", "flinkVersion": "v1_20", "flinkConfiguration": { "taskmanager.numberOfTaskSlots": "1", "metrics.reporter.prom.factory.class": "org.apache.flink.metrics.prometheus.PrometheusReporterFactory", "metrics.reporter.prom.port": "9249-9250" }, "serviceAccount": "flink", "jobManager": { "resource": { "memory": "1048m", "cpu": 1 } }, "taskManager": { "resource": { "memory": "1048m", "cpu": 1 } }, "job": { "jarURI": "local:///opt/flink/examples/streaming/StateMachineExample.jar", "state": "running", "parallelism": 3, "upgradeMode": "stateless" } }, "status": null } {code} The job fails to start with {code:java} Caused by: java.lang.ExceptionInInitializerError 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 Caused by: java.lang.RuntimeException: Can not register process function transformation translator. at org.apache.flink.datastream.impl.ExecutionEnvironmentImpl.<clinit>(ExecutionEnvironmentImpl.java:98) ~[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 Caused 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} I did some initial analysis, and it looks like, by default, the operator does not set the env.java.opts.all correctly while running the same image in standalone mode with docker works without issues. Content of flink-conf in k8s pod {code:java} blob.server.port: 6124 kubernetes.jobmanager.annotations: flinkdeployment.flink.apache.org/generation:2 kubernetes.jobmanager.replicas: 1 execution.submit-failed-job-on-application-error: true jobmanager.rpc.address: basic-example2.flink metrics.reporter.prom.factory.class: org.apache.flink.metrics.prometheus.PrometheusReporterFactory kubernetes.service-account: flink kubernetes.cluster-id: basic-example2 kubernetes.taskmanager.cpu.amount: 1.0 $internal.application.program-args: parallelism.default: 3 kubernetes.namespace: flink taskmanager.numberOfTaskSlots: 2 kubernetes.rest-service.exposed.type: ClusterIP kubernetes.jobmanager.owner.reference: controller:false,name:basic-example2,uid:1a31be61-e254-44b2-99ee-211b8482e045,kind:FlinkDeployment,apiVersion:flink.apache.org/v1beta1,blockOwnerDeletion:true kubernetes.container.image.ref: flink:1.20.0-scala_2.12-java17 metrics.reporter.prom.port: 9249-9250 taskmanager.memory.process.size: 1048 mb kubernetes.internal.jobmanager.entrypoint.class: org.apache.flink.kubernetes.entrypoint.KubernetesApplicationClusterEntrypoint pipeline.name: basic-example2 kubernetes.pod-template-file.taskmanager: /tmp/flink_op_generated_podTemplate_7474583206560050522.yaml web.cancel.enable: false execution.target: kubernetes-application jobmanager.memory.process.size: 1048 mb execution.shutdown-on-application-finish: false taskmanager.rpc.port: 6122 kubernetes.jobmanager.cpu.amount: 1.0 internal.cluster.execution-mode: NORMAL $internal.pipeline.job-id: 1dd75be3536073ffa185acb2d1385dbd pipeline.jars: local:///opt/flink/examples/streaming/StateMachineExample.jar execution.checkpointing.externalized-checkpoint-retention: RETAIN_ON_CANCELLATION rest.profiling.enabled: true $internal.flink.version: v1_20 kubernetes.pod-template-file.jobmanager: /tmp/flink_op_generated_podTemplate_2207026818603138122.yaml {code} > Default configurations fail all to start Flink java 17 images > ------------------------------------------------------------- > > Key: FLINK-36826 > URL: https://issues.apache.org/jira/browse/FLINK-36826 > Project: Flink > Issue Type: Bug > Components: Kubernetes Operator > Affects Versions: kubernetes-operator-1.10.0 > Reporter: Fabian Paul > Priority: Major > > When trying to deploy the following deployment with the operator the job > fails to start. > {code:java} > { > "kind": "FlinkDeployment", > "metadata": { > "name": "basic-example2" > }, > "spec": { > "image": "flink:1.20.0-scala_2.12-java17", > "flinkVersion": "v1_20", > "flinkConfiguration": { > "taskmanager.numberOfTaskSlots": "1", > "metrics.reporter.prom.factory.class": > "org.apache.flink.metrics.prometheus.PrometheusReporterFactory", > "metrics.reporter.prom.port": "9249-9250" > }, > "serviceAccount": "flink", > "jobManager": { > "resource": { > "memory": "1048m", > "cpu": 1 > } > }, > "taskManager": { > "resource": { > "memory": "1048m", > "cpu": 1 > } > }, > "job": { > "jarURI": > "local:///opt/flink/examples/streaming/StateMachineExample.jar", > "state": "running", > "parallelism": 3, > "upgradeMode": "stateless" > } > }, > "status": null > } > {code} > The job fails to start with > {code:java} > Caused by: java.lang.ExceptionInInitializerError > 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 > Caused by: java.lang.RuntimeException: Can not register process function > transformation translator. > at > org.apache.flink.datastream.impl.ExecutionEnvironmentImpl.<clinit>(ExecutionEnvironmentImpl.java:98) > ~[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 > Caused 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} > I did some initial analysis, and it looks like, by default, the operator does > not set the > env.java.opts.all correctly while running the same image in standalone mode > with docker works without issues. > > Content of flink-conf in k8s pod > {code:java} > blob.server.port: 6124 > kubernetes.jobmanager.annotations: > flinkdeployment.flink.apache.org/generation:2 > kubernetes.jobmanager.replicas: 1 > execution.submit-failed-job-on-application-error: true > jobmanager.rpc.address: basic-example2.flink > metrics.reporter.prom.factory.class: > org.apache.flink.metrics.prometheus.PrometheusReporterFactory > kubernetes.service-account: flink > kubernetes.cluster-id: basic-example2 > kubernetes.taskmanager.cpu.amount: 1.0 > $internal.application.program-args: > parallelism.default: 3 > kubernetes.namespace: flink > taskmanager.numberOfTaskSlots: 2 > kubernetes.rest-service.exposed.type: ClusterIP > kubernetes.jobmanager.owner.reference: > controller:false,name:basic-example2,uid:1a31be61-e254-44b2-99ee-211b8482e045,kind:FlinkDeployment,apiVersion:flink.apache.org/v1beta1,blockOwnerDeletion:true > kubernetes.container.image.ref: flink:1.20.0-scala_2.12-java17 > metrics.reporter.prom.port: 9249-9250 > taskmanager.memory.process.size: 1048 mb > kubernetes.internal.jobmanager.entrypoint.class: > org.apache.flink.kubernetes.entrypoint.KubernetesApplicationClusterEntrypoint > pipeline.name: basic-example2 > kubernetes.pod-template-file.taskmanager: > /tmp/flink_op_generated_podTemplate_7474583206560050522.yaml > web.cancel.enable: false > execution.target: kubernetes-application > jobmanager.memory.process.size: 1048 mb > execution.shutdown-on-application-finish: false > taskmanager.rpc.port: 6122 > kubernetes.jobmanager.cpu.amount: 1.0 > internal.cluster.execution-mode: NORMAL > $internal.pipeline.job-id: 1dd75be3536073ffa185acb2d1385dbd > pipeline.jars: local:///opt/flink/examples/streaming/StateMachineExample.jar > execution.checkpointing.externalized-checkpoint-retention: > RETAIN_ON_CANCELLATION > rest.profiling.enabled: true > $internal.flink.version: v1_20 > kubernetes.pod-template-file.jobmanager: > /tmp/flink_op_generated_podTemplate_2207026818603138122.yaml {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)