[ https://issues.apache.org/jira/browse/SPARK-28025?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16862908#comment-16862908 ]
Stavros Kontopoulos edited comment on SPARK-28025 at 6/13/19 10:51 AM: ----------------------------------------------------------------------- I just found out that the following config options would suffice to avoid creating crcs: --conf spark.hadoop.spark.sql.streaming.checkpointFileManagerClass=org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem So need to make a PR to disable crc emission if user wants to. Unfortunately the filesystem hierarchy in hadoop is a bit inconsistent. So the LocalFileSystem will use a a FilterFileSystem that has the flag setWriteChecksum but the DistributedFileSystem does not have it and it is controlled by property [https://github.com/apache/hadoop/blob/533138718cc05b78e0afe583d7a9bd30e8a48fdc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java#L620] `dfs.checksum.type`. Btw there is a nice summary on the topic in chapter 5 of the book: hadoop the definitive guide. So should we clean up things like in the old PR? I am not sure if the old PR would work with the LocalFs as it seems that the crc file will be renamed by default as the underlying system supports checksums by default. was (Author: skonto): I just found out that the following config options would suffice to avoid creating crcs: --conf spark.hadoop.spark.sql.streaming.checkpointFileManagerClass=org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem So need to make a PR to disable crc emission if user wants to in this case but could make one for the generic case if the filesystem supports the flag mentioned above. Will do that. Btw there is a nice summary on the topic in chapter 5 of the book: hadoop the definitive guide. So should we clean up things like in the old PR? I am not sure if the old PR would work with the LocalFs as it seems that the crc file will be renamed by default as the underlying system supports checksums by default. > HDFSBackedStateStoreProvider should not leak .crc files > -------------------------------------------------------- > > Key: SPARK-28025 > URL: https://issues.apache.org/jira/browse/SPARK-28025 > Project: Spark > Issue Type: Bug > Components: Structured Streaming > Affects Versions: 2.4.3 > Environment: Spark 2.4.3 > Kubernetes 1.11(?) (OpenShift) > StateStore storage on a mounted PVC. Viewed as a local filesystem by the > `FileContextBasedCheckpointFileManager` : > {noformat} > scala> glusterfm.isLocal > res17: Boolean = true{noformat} > Reporter: Gerard Maas > Priority: Major > > The HDFSBackedStateStoreProvider when using the default CheckpointFileManager > is leaving '.crc' files behind. There's a .crc file created for each > `atomicFile` operation of the CheckpointFileManager. > Over time, the number of files becomes very large. It makes the state store > file system constantly increase in size and, in our case, deteriorates the > file system performance. > Here's a sample of one of our spark storage volumes after 2 days of execution > (4 stateful streaming jobs, each on a different sub-dir): > # > {noformat} > Total files in PVC (used for checkpoints and state store) > $find . | wc -l > 431796 > # .crc files > $find . -name "*.crc" | wc -l > 418053{noformat} > With each .crc file taking one storage block, the used storage runs into the > GBs of data. > These jobs are running on Kubernetes. Our shared storage provider, GlusterFS, > shows serious performance deterioration with this large number of files: > {noformat} > DEBUG HDFSBackedStateStoreProvider: fetchFiles() took 29164ms{noformat} > -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org