Yusu Gao created FLINK-36810:
--------------------------------

             Summary: Reflective access warning not addressed causing Flink 
failing to process job creation
                 Key: FLINK-36810
                 URL: https://issues.apache.org/jira/browse/FLINK-36810
             Project: Flink
          Issue Type: Bug
          Components: API / DataStream
    Affects Versions: 1.20.0
            Reporter: Yusu Gao


While we are testing migration our Flink cluster from 1.11 on Java 11 runtime 
to 1.20 on Java 17 runtime, we noticed a job creation error.
{code:java}
"errors":["org.apache.flink.runtime.rest.handler.RestHandlerException: Could 
not execute application.
    at 
org.apache.flink.runtime.webmonitor.handlers.JarRunHandler.lambda$handleRequest$1(JarRunHandler.java:114)
    at 
java.base/java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:934)
    at 
java.base/java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:911)
    at 
java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
    at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1773)
    at java.base/java.lang.Thread.run(Thread.java:840)
Caused by: java.util.concurrent.CompletionException: 
java.lang.NoClassDefFoundError: Could not initialize class 
org.apache.flink.datastream.impl.ExecutionContextEnvironment
    at 
java.base/java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:315)
    at 
java.base/java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:320)
    at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1770)
    ... 1 more
Caused by: java.lang.NoClassDefFoundError: Could not initialize class 
org.apache.flink.datastream.impl.ExecutionContextEnvironment
    at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:109)
    at 
org.apache.flink.client.deployment.application.DetachedApplicationRunner.tryExecuteJobs(DetachedApplicationRunner.java:84)
    at 
org.apache.flink.client.deployment.application.DetachedApplicationRunner.run(DetachedApplicationRunner.java:70)
    at 
org.apache.flink.runtime.webmonitor.handlers.JarRunHandler.lambda$handleRequest$0(JarRunHandler.java:108)
    at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
    ... 1 more
Caused by: java.lang.ExceptionInInitializerError: Exception 
java.lang.ExceptionInInitializerError [in thread \"flink-jar-runner-thread-1\"]
    ... 6 more {code}
Reverting runtime back to Java 11 with {*}--illegal-access=warn{*}, we found a 
warning for Flink code
{code:java}
WARNING: Illegal reflective access by 
org.apache.flink.streaming.runtime.translators.DataStreamV2SinkTransformationTranslator
 (file:/path/to/flink/flink-1.20.0/lib/flink-dist-1.20.0.jar) to field 
java.util.Collections$UnmodifiableMap.m {code}
Which points to 
[DataStreamV2SinkTransformationTranslator|https://github.com/apache/flink/blob/release-1.20.0/flink-datastream/src/main/java/org/apache/flink/streaming/runtime/translators/DataStreamV2SinkTransformationTranslator.java#L98]
 which is used exactly in the [class 
|https://github.com/apache/flink/blob/master/flink-datastream/src/main/java/org/apache/flink/datastream/impl/ExecutionEnvironmentImpl.java#L95]throwing
 exception.

And adding argument to the JVM_ARGS env var fixed issue
{code:java}
export JVM_ARGS="$JVM_ARGS --add-opens=java.base/java.util=ALL-UNNAMED"{code}
This is a workaround but not necessarily the best solution nor the safest. 
Please help provide insights or better solution for such issue. 



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

Reply via email to