[ https://issues.apache.org/jira/browse/FLINK-34733?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
ASF GitHub Bot updated FLINK-34733: ----------------------------------- Labels: pull-request-available (was: ) > OSS Filesystem throws ClassNotFoundException > -------------------------------------------- > > Key: FLINK-34733 > URL: https://issues.apache.org/jira/browse/FLINK-34733 > Project: Flink > Issue Type: Bug > Components: FileSystems > Affects Versions: 1.17.2 > Environment: Flink >= v1.17 > Reporter: zhuoshaojian > Priority: Major > Labels: pull-request-available > > The ClassNotFoundException was caused by this commit > [[https://github.com/apache/flink/commit/52a2b98bb5af842633df0c051b5da95d437a6b2f]], > which removed the relocation configuration from > pom.xml.[FLINK-31612|https://issues.apache.org/jira/browse/FLINK-31612]. > But in plugin flink-oss-fs-hadoop, the shaded prefix was hardcoded > [https://github.com/apache/flink/blob/c0027e5777f9d77970fdb99bcc158d65ea48d514/flink-filesystems/flink-oss-fs-hadoop/src/main/java/org/apache/flink/fs/osshadoop/OSSFileSystemFactory.java#L50] > This resulted in the exception: > ``` > Caused by: java.lang.ClassNotFoundException: > org.apache.flink.fs.osshadoop.shaded.com.aliyun.oss.common.auth.EnvironmentVariableCredentialsProvider > at java.net.URLClassLoader.findClass(Unknown Source) ~[?:?] > at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?] > at > org.apache.flink.core.classloading.ComponentClassLoader.loadClassFromComponentOnly(ComponentClassLoader.java:150) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.core.classloading.ComponentClassLoader.loadClassFromOwnerFirst(ComponentClassLoader.java:172) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.core.classloading.ComponentClassLoader.loadClass(ComponentClassLoader.java:107) > ~[flink-dist-1.17.2.jar:1.17.2] > at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?] > at java.lang.Class.forName0(Native Method) ~[?:?] > at java.lang.Class.forName(Unknown Source) ~[?:?] > at > org.apache.hadoop.fs.aliyun.oss.AliyunOSSUtils.getCredentialsProvider(AliyunOSSUtils.java:118) > ~[?:?] > at > org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore.initialize(AliyunOSSFileSystemStore.java:155) > ~[?:?] > at > org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem.initialize(AliyunOSSFileSystem.java:349) > ~[?:?] > at > org.apache.flink.fs.osshadoop.OSSFileSystemFactory.create(OSSFileSystemFactory.java:103) > ~[?:?] > at > org.apache.flink.core.fs.PluginFileSystemFactory.create(PluginFileSystemFactory.java:62) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:508) > ~[flink-dist-1.17.2.jar:1.17.2] > at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:409) > ~[flink-dist-1.17.2.jar:1.17.2] > at org.apache.flink.core.fs.Path.getFileSystem(Path.java:274) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.blob.BlobUtils.createFileSystemBlobStore(BlobUtils.java:99) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.blob.BlobUtils.createBlobStoreFromConfig(BlobUtils.java:86) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.kubernetes.highavailability.KubernetesHaServicesFactory.createHAServices(KubernetesHaServicesFactory.java:41) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.createCustomHAServices(HighAvailabilityServicesUtils.java:296) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.createHighAvailabilityServices(HighAvailabilityServicesUtils.java:139) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.entrypoint.ClusterEntrypoint.createHaServices(ClusterEntrypoint.java:442) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.entrypoint.ClusterEntrypoint.initializeServices(ClusterEntrypoint.java:391) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:282) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$1(ClusterEntrypoint.java:232) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:28) > ~[flink-dist-1.17.2.jar:1.17.2] > at > org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:229) > ~[flink-dist-1.17.2.jar:1.17.2] > ... 2 more > ``` -- This message was sent by Atlassian Jira (v8.20.10#820010)