[jira] [Commented] (FLINK-32241) UnsupportedFileSystemException when using the ABFS Hadoop driver for checkpointing in Flink 1.17
[ https://issues.apache.org/jira/browse/FLINK-32241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17802077#comment-17802077 ] Martijn Visser commented on FLINK-32241: [~luoyuxia] Have you looked more into this topic? > UnsupportedFileSystemException when using the ABFS Hadoop driver for > checkpointing in Flink 1.17 > > > Key: FLINK-32241 > URL: https://issues.apache.org/jira/browse/FLINK-32241 > Project: Flink > Issue Type: Bug > Components: FileSystems, Runtime / Checkpointing >Affects Versions: 1.17.0, 1.17.1 > Environment: Flink 1.17.1 > Hadoop 3.3.4 > Flink Operator 1.4.0 > Kubernetes 1.24 >Reporter: Anton Ippolitov >Priority: Minor > > https://issues.apache.org/jira/browse/HADOOP-18707 introduced a new > functionality in the ABFS Hadoop client which buffers data on local disk by > default. > It looks like this breaks with Flink 1.17 in a scenario where: > * ABFS is used for checkpointing > * JobManager HA is enabled > * First JobManager leader dies and a stand-by JobManager takes over > I can reliably reproduce this with Flink 1.17.1 running on Kubernetes by > simply killing the JM leader pod. Once the stand-by JobManager takes over, > all checkpoints consistently fail with the following error: > {noformat} > org.apache.flink.runtime.checkpoint.CheckpointException: Failure to finalize > checkpoint. at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.finalizeCheckpoint(CheckpointCoordinator.java:1424) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.completePendingCheckpoint(CheckpointCoordinator.java:1310) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveAcknowledgeMessage(CheckpointCoordinator.java:1202) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$acknowledgeCheckpoint$1(ExecutionGraphHandler.java:89) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$processCheckpointCoordinatorMessage$3(ExecutionGraphHandler.java:119) > at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.base/java.lang.Thread.run(Thread.java:829) Caused by: > org.apache.hadoop.fs.UnsupportedFileSystemException: No FileSystem for scheme > "file" at > org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:3443) at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3466) at > org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174) at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574) at > org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521) at > org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540) at > org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:496) at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:316) > at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:393) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301) > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1064) at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:154) > at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:37) > at > org.apache.flink.core.fs.PluginFileSystemFactory$ClassLoaderFixingFileSystem.create(PluginFileSystemFactory.java:170) > at > org.apache.flink.runtime.state.filesystem.FSDataOutputStreamWrapper.(FSDataOutputStreamWrapper.java:42) > at > org.apache.flink.runtime.state.filesystem.FsCheckpointMetadataOutputStream.getOutputStreamWrapper(FsCheckpointMetadataOutputStream.java:179) > at > org.apache.flink.runtime.state.filesystem.FsCheckpointM
[jira] [Commented] (FLINK-32241) UnsupportedFileSystemException when using the ABFS Hadoop driver for checkpointing in Flink 1.17
[ https://issues.apache.org/jira/browse/FLINK-32241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17741662#comment-17741662 ] Alexis Sarda-Espinosa commented on FLINK-32241: --- [I am seeing this same issue without standby JMs|https://lists.apache.org/thread/24228yww7bz5kskx5zkjt1fwkvh47dqs], at some point the JM pod restarts and then all checkpoints and savepoints fail even if I restart all JM and TM pods manually. [~_anton] where did you set {{fs.azure.data.blocks.buffer}}? Does Flink forward it if I place that in {{flink-conf.yaml}}? > UnsupportedFileSystemException when using the ABFS Hadoop driver for > checkpointing in Flink 1.17 > > > Key: FLINK-32241 > URL: https://issues.apache.org/jira/browse/FLINK-32241 > Project: Flink > Issue Type: Bug > Components: FileSystems, Runtime / Checkpointing >Affects Versions: 1.17.0, 1.17.1 > Environment: Flink 1.17.1 > Hadoop 3.3.4 > Flink Operator 1.4.0 > Kubernetes 1.24 >Reporter: Anton Ippolitov >Priority: Minor > > https://issues.apache.org/jira/browse/HADOOP-18707 introduced a new > functionality in the ABFS Hadoop client which buffers data on local disk by > default. > It looks like this breaks with Flink 1.17 in a scenario where: > * ABFS is used for checkpointing > * JobManager HA is enabled > * First JobManager leader dies and a stand-by JobManager takes over > I can reliably reproduce this with Flink 1.17.1 running on Kubernetes by > simply killing the JM leader pod. Once the stand-by JobManager takes over, > all checkpoints consistently fail with the following error: > {noformat} > org.apache.flink.runtime.checkpoint.CheckpointException: Failure to finalize > checkpoint. at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.finalizeCheckpoint(CheckpointCoordinator.java:1424) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.completePendingCheckpoint(CheckpointCoordinator.java:1310) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveAcknowledgeMessage(CheckpointCoordinator.java:1202) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$acknowledgeCheckpoint$1(ExecutionGraphHandler.java:89) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$processCheckpointCoordinatorMessage$3(ExecutionGraphHandler.java:119) > at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.base/java.lang.Thread.run(Thread.java:829) Caused by: > org.apache.hadoop.fs.UnsupportedFileSystemException: No FileSystem for scheme > "file" at > org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:3443) at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3466) at > org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174) at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574) at > org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521) at > org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540) at > org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:496) at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:316) > at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:393) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301) > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1064) at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:154) > at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:37) > at > org.apache.flink.core.fs.PluginFileSystemFactory$ClassLoaderFixingFileSystem.create(PluginFileSystemFactory.jav
[jira] [Commented] (FLINK-32241) UnsupportedFileSystemException when using the ABFS Hadoop driver for checkpointing in Flink 1.17
[ https://issues.apache.org/jira/browse/FLINK-32241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17729314#comment-17729314 ] luoyuxia commented on FLINK-32241: -- I think the root cause maynot similar to FLINK-30635. I suspect it may be caused by the different class loader. >From the excpetion and hadoop code, we can know the result of >`ServiceLoader.load(FileSystem.class)` is empty although there should be >LocalFileSystem since hadoop-common is on classpaths. Since it's emty, it >can't find the FileSystem for scheme "file", so it throw >`org.apache.hadoop.fs.UnsupportedFileSystemException: No FileSystem for scheme >"file" at `. I'm trying to understand why `ServiceLoader.load(FileSystem.class)` is empty. I'm not very sure, but from the stack strace and flink plugin mechanism, the class `FileSystem` is loaded by `PluginClassLoader`, and `ServiceLoader.load(FileSystem.class)` will try to use ` Thread.currentThread().getContextClassLoader()` which maybe appclassloader to load the `FileSystem.class`. Since they are different classloaders, the result is empty. If that's the case, I think you may can set `plugin.classloader.parent-first-patterns.additional` = `org.apache.hadoop.` to enforce FileSystem.class to be loaded by appclassloader. > UnsupportedFileSystemException when using the ABFS Hadoop driver for > checkpointing in Flink 1.17 > > > Key: FLINK-32241 > URL: https://issues.apache.org/jira/browse/FLINK-32241 > Project: Flink > Issue Type: Bug > Components: FileSystems, Runtime / Checkpointing >Affects Versions: 1.17.0, 1.17.1 > Environment: Flink 1.17.1 > Hadoop 3.3.4 > Flink Operator 1.4.0 > Kubernetes 1.24 >Reporter: Anton Ippolitov >Priority: Minor > > https://issues.apache.org/jira/browse/HADOOP-18707 introduced a new > functionality in the ABFS Hadoop client which buffers data on local disk by > default. > It looks like this breaks with Flink 1.17 in a scenario where: > * ABFS is used for checkpointing > * JobManager HA is enabled > * First JobManager leader dies and a stand-by JobManager takes over > I can reliably reproduce this with Flink 1.17.1 running on Kubernetes by > simply killing the JM leader pod. Once the stand-by JobManager takes over, > all checkpoints consistently fail with the following error: > {noformat} > org.apache.flink.runtime.checkpoint.CheckpointException: Failure to finalize > checkpoint. at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.finalizeCheckpoint(CheckpointCoordinator.java:1424) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.completePendingCheckpoint(CheckpointCoordinator.java:1310) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveAcknowledgeMessage(CheckpointCoordinator.java:1202) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$acknowledgeCheckpoint$1(ExecutionGraphHandler.java:89) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$processCheckpointCoordinatorMessage$3(ExecutionGraphHandler.java:119) > at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.base/java.lang.Thread.run(Thread.java:829) Caused by: > org.apache.hadoop.fs.UnsupportedFileSystemException: No FileSystem for scheme > "file" at > org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:3443) at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3466) at > org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174) at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574) at > org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521) at > org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540) at > org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:496) at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:316) > at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:393) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173) >
[jira] [Commented] (FLINK-32241) UnsupportedFileSystemException when using the ABFS Hadoop driver for checkpointing in Flink 1.17
[ https://issues.apache.org/jira/browse/FLINK-32241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17729240#comment-17729240 ] Hangxiang Yu commented on FLINK-32241: -- +1 it's similar to FLINK-30635. setting {{fs.azure.data.blocks.buffer}} to {{array}} makes abfs use memory to cache instead of local disk so that it doesn't need localfilesystem. it may also works if setting _fs.file.impl_ to _org.apache.hadoop.fs.LocalFileSystem.class.getName()_ explicitly (It may be a shaded path). > UnsupportedFileSystemException when using the ABFS Hadoop driver for > checkpointing in Flink 1.17 > > > Key: FLINK-32241 > URL: https://issues.apache.org/jira/browse/FLINK-32241 > Project: Flink > Issue Type: Bug > Components: FileSystems, Runtime / Checkpointing >Affects Versions: 1.17.0, 1.17.1 > Environment: Flink 1.17.1 > Hadoop 3.3.4 > Flink Operator 1.4.0 > Kubernetes 1.24 >Reporter: Anton Ippolitov >Priority: Minor > > https://issues.apache.org/jira/browse/HADOOP-18707 introduced a new > functionality in the ABFS Hadoop client which buffers data on local disk by > default. > It looks like this breaks with Flink 1.17 in a scenario where: > * ABFS is used for checkpointing > * JobManager HA is enabled > * First JobManager leader dies and a stand-by JobManager takes over > I can reliably reproduce this with Flink 1.17.1 running on Kubernetes by > simply killing the JM leader pod. Once the stand-by JobManager takes over, > all checkpoints consistently fail with the following error: > {noformat} > org.apache.flink.runtime.checkpoint.CheckpointException: Failure to finalize > checkpoint. at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.finalizeCheckpoint(CheckpointCoordinator.java:1424) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.completePendingCheckpoint(CheckpointCoordinator.java:1310) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveAcknowledgeMessage(CheckpointCoordinator.java:1202) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$acknowledgeCheckpoint$1(ExecutionGraphHandler.java:89) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$processCheckpointCoordinatorMessage$3(ExecutionGraphHandler.java:119) > at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.base/java.lang.Thread.run(Thread.java:829) Caused by: > org.apache.hadoop.fs.UnsupportedFileSystemException: No FileSystem for scheme > "file" at > org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:3443) at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3466) at > org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174) at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574) at > org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521) at > org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540) at > org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:496) at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:316) > at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:393) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301) > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1064) at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:154) > at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:37) > at > org.apache.flink.core.fs.PluginFileSystemFactory$ClassLoaderFixingFileSystem.create(PluginFileSystemFactory.java:170) > at > org.apache.flink.runtime.state.filesystem.FSDataOu
[jira] [Commented] (FLINK-32241) UnsupportedFileSystemException when using the ABFS Hadoop driver for checkpointing in Flink 1.17
[ https://issues.apache.org/jira/browse/FLINK-32241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17728537#comment-17728537 ] luoyuxia commented on FLINK-32241: -- Seems the root cause is similar to FLINK-30635? > UnsupportedFileSystemException when using the ABFS Hadoop driver for > checkpointing in Flink 1.17 > > > Key: FLINK-32241 > URL: https://issues.apache.org/jira/browse/FLINK-32241 > Project: Flink > Issue Type: Bug > Components: Runtime / Checkpointing >Affects Versions: 1.17.0, 1.17.1 > Environment: Flink 1.17.1 > Hadoop 3.3.4 > Flink Operator 1.4.0 > Kubernetes 1.24 >Reporter: Anton Ippolitov >Priority: Minor > > https://issues.apache.org/jira/browse/HADOOP-18707 introduced a new > functionality in the ABFS Hadoop client which buffers data on local disk by > default. > It looks like this breaks with Flink 1.17 in a scenario where: > * ABFS is used for checkpointing > * JobManager HA is enabled > * First JobManager leader dies and a stand-by JobManager takes over > I can reliably reproduce this with Flink 1.17.1 running on Kubernetes by > simply killing the JM leader pod. Once the stand-by JobManager takes over, > all checkpoints consistently fail with the following error: > {noformat} > org.apache.flink.runtime.checkpoint.CheckpointException: Failure to finalize > checkpoint. at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.finalizeCheckpoint(CheckpointCoordinator.java:1424) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.completePendingCheckpoint(CheckpointCoordinator.java:1310) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveAcknowledgeMessage(CheckpointCoordinator.java:1202) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$acknowledgeCheckpoint$1(ExecutionGraphHandler.java:89) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$processCheckpointCoordinatorMessage$3(ExecutionGraphHandler.java:119) > at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.base/java.lang.Thread.run(Thread.java:829) Caused by: > org.apache.hadoop.fs.UnsupportedFileSystemException: No FileSystem for scheme > "file" at > org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:3443) at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3466) at > org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174) at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574) at > org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521) at > org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540) at > org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:496) at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:316) > at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:393) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301) > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1064) at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:154) > at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:37) > at > org.apache.flink.core.fs.PluginFileSystemFactory$ClassLoaderFixingFileSystem.create(PluginFileSystemFactory.java:170) > at > org.apache.flink.runtime.state.filesystem.FSDataOutputStreamWrapper.(FSDataOutputStreamWrapper.java:42) > at > org.apache.flink.runtime.state.filesystem.FsCheckpointMetadataOutputStream.getOutputStreamWrapper(FsCheckpointMetadataOutputStream.java:179) > at > org.apache.flink.runtime.state.filesystem.FsCheckpointMetadataOutputStream.(FsChec
[jira] [Commented] (FLINK-32241) UnsupportedFileSystemException when using the ABFS Hadoop driver for checkpointing in Flink 1.17
[ https://issues.apache.org/jira/browse/FLINK-32241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17728393#comment-17728393 ] Steve Loughran commented on FLINK-32241: this is odd as the buffer to disk code is derived from what's been in the s3a connector for years and it never blew up. on a host without the abfs change, can you grab the cloudstore module and run its storediag command to get a view of the world, including probes of temp dirs ... look for the output under "Output Buffering" to see https://github.com/steveloughran/cloudstore > UnsupportedFileSystemException when using the ABFS Hadoop driver for > checkpointing in Flink 1.17 > > > Key: FLINK-32241 > URL: https://issues.apache.org/jira/browse/FLINK-32241 > Project: Flink > Issue Type: Bug > Components: Runtime / Checkpointing >Affects Versions: 1.17.0, 1.17.1 > Environment: Flink 1.17.1 > Hadoop 3.3.4 > Flink Operator 1.4.0 > Kubernetes 1.24 >Reporter: Anton Ippolitov >Priority: Minor > > https://issues.apache.org/jira/browse/HADOOP-18707 introduced a new > functionality in the ABFS Hadoop client which buffers data on local disk by > default. > It looks like this breaks with Flink 1.17 in a scenario where: > * ABFS is used for checkpointing > * JobManager HA is enabled > * First JobManager leader dies and a stand-by JobManager takes over > I can reliably reproduce this with Flink 1.17.1 running on Kubernetes by > simply killing the JM leader pod. Once the stand-by JobManager takes over, > all checkpoints consistently fail with the following error: > {noformat} > org.apache.flink.runtime.checkpoint.CheckpointException: Failure to finalize > checkpoint. at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.finalizeCheckpoint(CheckpointCoordinator.java:1424) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.completePendingCheckpoint(CheckpointCoordinator.java:1310) > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveAcknowledgeMessage(CheckpointCoordinator.java:1202) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$acknowledgeCheckpoint$1(ExecutionGraphHandler.java:89) > at > org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$processCheckpointCoordinatorMessage$3(ExecutionGraphHandler.java:119) > at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.base/java.lang.Thread.run(Thread.java:829) Caused by: > org.apache.hadoop.fs.UnsupportedFileSystemException: No FileSystem for scheme > "file" at > org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:3443) at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3466) at > org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174) at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574) at > org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521) at > org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540) at > org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:496) at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:316) > at > org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:393) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165) > at > org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980) > at > org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262) > at > org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580) > at > org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301) > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175) at > org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1064) at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:154) > at > org.apache.flink.fs.azure.common.hadoop.HadoopFileSystem.create(HadoopFileSystem.java:37) > at > org.apache.flink.core.fs.PluginFileSystemFactory$ClassLoaderFixingFileSystem.create(PluginFileSystemFactory.java:170) > at