Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-19398

Hive connector fails with IllegalAccessError if submitted as usercode

    XMLWordPrintableJSON

    Details

      Description

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

      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]
      

      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.

       

        Attachments

          Issue Links

            Activity

              People

              • Assignee:
                gaoyunhaii Yun Gao
                Reporter:
                fhueske Fabian Hueske
              • Votes:
                0 Vote for this issue
                Watchers:
                9 Start watching this issue

                Dates

                • Created:
                  Updated:
                  Resolved: