You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Fabian Hueske (Jira)" <ji...@apache.org> on 2020/09/24 13:03:00 UTC

[jira] [Created] (FLINK-19398) Hive connector fails with IllegalAccessError if submitted as usercode

Fabian Hueske created FLINK-19398:
-------------------------------------

             Summary: Hive connector fails with IllegalAccessError if submitted as usercode
                 Key: FLINK-19398
                 URL: https://issues.apache.org/jira/browse/FLINK-19398
             Project: Flink
          Issue Type: Bug
          Components: Connectors / Hive
    Affects Versions: 1.11.2, 1.12.0
            Reporter: Fabian Hueske


Using Flink's Hive connector fails if the dependency is loaded with the user code classloader with the following exception.


{code:java}
java.lang.IllegalAccessError: tried to access method org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.<init>(Lorg/apache/flink/core/fs/Path;Lorg/apache/flink/streaming/api/functions/sink/filesystem/BucketAssigner;Lorg/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory;Lorg/apache/flink/streaming/api/functions/sink/filesystem/BucketWriter;Lorg/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy;ILorg/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig;)V from class org.apache.flink.streaming.api.functions.sink.filesystem.HadoopPathBasedBulkFormatBuilder
	at org.apache.flink.streaming.api.functions.sink.filesystem.HadoopPathBasedBulkFormatBuilder.createBuckets(HadoopPathBasedBulkFormatBuilder.java:127) ~[flink-connector-hive_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.table.filesystem.stream.StreamingFileWriter.initializeState(StreamingFileWriter.java:81) ~[flink-table-blink_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.initializeOperatorState(StreamOperatorStateHandler.java:106) ~[flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:258) ~[flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:290) ~[flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:479) ~[flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:92) ~[flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:475) ~[flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:528) ~[flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721) [flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) [flink-dist_2.12-1.11.2-stream2-SNAPSHOT.jar:1.11.2-stream2-SNAPSHOT]
{code}

The problem is the constructor of {{Buckets}} with default visibility which is called from {{HadoopPathBasedBulkFormatBuilder}} . This works as long as both classes are loaded with the same classloader but when they are loaded in different classloaders, the access fails.

{{Buckets}} is loaded with the system CL because it is part of flink-streaming-java. 

 

To solve this issue, we should change the visibility of the {{Buckets}} constructor to {{public}}.

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)