You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Stavros Kontopoulos (JIRA)" <ji...@apache.org> on 2019/06/12 18:18:00 UTC

[jira] [Comment Edited] (SPARK-28025) HDFSBackedStateStoreProvider should not leak .crc files

    [ https://issues.apache.org/jira/browse/SPARK-28025?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16862339#comment-16862339 ] 

Stavros Kontopoulos edited comment on SPARK-28025 at 6/12/19 6:17 PM:
----------------------------------------------------------------------

There is a workaround by setting `--conf spark.hadoop.spark.sql.streaming.checkpointFileManagerClass=org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager` when using local fs

and modifying the FileSystemBasedCheckpointFileManager to run `fs.setWriteChecksum(false)` after fs is created.

Reason is the FileContextBasedCheckpointFileManager will use ChecksumFS ([https://github.com/apache/hadoop/blob/73746c5da76d5e39df131534a1ec35dfc5d2529b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java]) under the hoods which will ignore "

CreateOpts.checksumParam(ChecksumOpt.createDisabled())" passed. These settings will only avoid creating checksums for the checksums themshelves.

FileSystemBasedCheckpointFileManager uses [https://github.com/apache/hadoop/blob/73746c5da76d5e39df131534a1ec35dfc5d2529b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java] which allows to avoid checksums when creating files.

Note that the crc is created when the tmp file is created not during rename or mv.

I will create a PR shortly.


was (Author: skonto):
There is a workaround by setting `--conf spark.hadoop.spark.sql.streaming.checkpointFileManagerClass=org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager` when using local fs

and modifying the FileSystemBasedCheckpointFileManager to run `fs.setWriteChecksum(false)` after fs is created.

Reason is the FileContextBasedCheckpointFileManager will use ChecksumFS ([https://github.com/apache/hadoop/blob/73746c5da76d5e39df131534a1ec35dfc5d2529b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java]) under the hoods which will ignore "

CreateOpts.checksumParam(ChecksumOpt.createDisabled())" passed. These settings will only avoid creating checksums for the checksums themshelves.

FileSystemBasedCheckpointFileManager uses [https://github.com/apache/hadoop/blob/73746c5da76d5e39df131534a1ec35dfc5d2529b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java] which allows to avoid checksums when creating files.

Note that the crc is created when the tmp file is created not during rename or mv.

> 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-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org