Hi Xiao, I'm not quite convinced that the azure plugin ruined your workload, I would take a look at the dependency graph you've in the pom. Adding multiple deps can conflict in terms of class loader services (src/main/resources/META-INF/services).
As an example you've 2 such dependencies where org.apache.flink.core.fs.FileSystemFactory is in the jar. Hadoop core contains "flie" and the other one something different. Let's say you don't use service merge plugin in your maven project. Such case Hadoop core `file` entry will be just overwritten by the second one. Solution: Either avoid deps with conflicting services or add ServicesResourceTransformer to your maven project. G On Wed, Jun 26, 2024 at 10:16 AM Xiao Xu <ffxrqy...@gmail.com> wrote: > Hi, all > > I try to use Flink to write Azure Blob Storage which called ADLS, I put > the flink-azure-fs-hadoop jar in plugins directory and when I start my > write job it shows: > > Caused by: org.apache.hadoop.fs.UnsupportedFileSystemException: No > FileSystem for scheme "file" > at > org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:3443) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3466) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:496) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:316) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:393) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262) > ~[hadoop-azure-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.<init>(AbfsOutputStream.java:173) > ~[hadoop-azure-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580) > ~[hadoop-azure-3.3.4.5.1.5.3.jar:?] > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301) > ~[hadoop-azure-3.3.4.5.1.5.3.jar:?] > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1064) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1052) > ~[hadoop-common-3.3.4.5.1.5.3.jar:?] > > I search the issue looks like this: > https://stackoverflow.com/questions/77238642/apache-flink-azure-abfs-file-sink-error-streaming-unsupportedfilesystemexcep > > my env: > Flink: 1.18.1 > JDK: 1.8 > > Does anyone else have the same problem? >