Yang Wang created FLINK-16115:
---------------------------------

             Summary: Aliyun oss filesystem could not work with plugin mechanism
                 Key: FLINK-16115
                 URL: https://issues.apache.org/jira/browse/FLINK-16115
             Project: Flink
          Issue Type: Bug
          Components: Connectors / FileSystem
    Affects Versions: 1.10.0
            Reporter: Yang Wang


>From release-1.9, Flink suggest users to load all filesystem with plugin, 
>including oss. However, it could not work for oss filesystem. The root cause 
>is it does not shade the {{org.apache.flink.runtime.fs.hdfs}} and 
>{{org.apache.flink.runtime.util}}. So they will always be loaded by system 
>classloader and throw the following exceptions.

 
{code:java}
2020-02-17 17:28:47,247 ERROR 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Could not start 
cluster entrypoint StandaloneSessionClusterEntrypoint.
org.apache.flink.runtime.entrypoint.ClusterEntrypointException: Failed to 
initialize the cluster entrypoint StandaloneSessionClusterEntrypoint.
        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:187)
        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:518)
        at 
org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint.main(StandaloneSessionClusterEntrypoint.java:64)
Caused by: java.lang.NoSuchMethodError: 
org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.<init>(Lorg/apache/flink/fs/shaded/hadoop3/org/apache/hadoop/fs/FileSystem;)V
        at 
org.apache.flink.fs.osshadoop.OSSFileSystemFactory.create(OSSFileSystemFactory.java:85)
        at 
org.apache.flink.core.fs.PluginFileSystemFactory.create(PluginFileSystemFactory.java:61)
        at 
org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:441)
        at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:362)
        at org.apache.flink.core.fs.Path.getFileSystem(Path.java:298)
        at 
org.apache.flink.runtime.blob.BlobUtils.createFileSystemBlobStore(BlobUtils.java:100)
        at 
org.apache.flink.runtime.blob.BlobUtils.createBlobStoreFromConfig(BlobUtils.java:89)
        at 
org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.createHighAvailabilityServices(HighAvailabilityServicesUtils.java:125)
        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.createHaServices(ClusterEntrypoint.java:305)
        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.initializeServices(ClusterEntrypoint.java:263)
        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:207)
        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$0(ClusterEntrypoint.java:169)
        at 
org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:168)
        ... 2 more
{code}



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

Reply via email to