[jira] [Commented] (SPARK-24295) Purge Structured streaming FileStreamSinkLog metadata compact file data.
[ https://issues.apache.org/jira/browse/SPARK-24295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16787502#comment-16787502 ] Alfredo Gimenez commented on SPARK-24295: - Currently old metadata files are already deleted after expiring (see [https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala#L222)]. The problem is that the compaction file includes the contents from the previous compaction file, so every compaction is at least as large as the last one and they grow linearly. I am tempted to say that old compaction files should be deleted or not included in the next compaction, but I don't know if that will negatively affect checkpointing or any other internals that may depend on the metadata logs. > Purge Structured streaming FileStreamSinkLog metadata compact file data. > > > Key: SPARK-24295 > URL: https://issues.apache.org/jira/browse/SPARK-24295 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 2.3.0 >Reporter: Iqbal Singh >Priority: Major > Attachments: spark_metadatalog_compaction_perfbug_repro.tar.gz > > > FileStreamSinkLog metadata logs are concatenated to a single compact file > after defined compact interval. > For long running jobs, compact file size can grow up to 10's of GB's, Causing > slowness while reading the data from FileStreamSinkLog dir as spark is > defaulting to the "__spark__metadata" dir for the read. > We need a functionality to purge the compact file size. > -- 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
[jira] [Commented] (SPARK-24295) Purge Structured streaming FileStreamSinkLog metadata compact file data.
[ https://issues.apache.org/jira/browse/SPARK-24295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16782510#comment-16782510 ] Alfredo Gimenez commented on SPARK-24295: - Our current workaround FWII: We've added a streaming query listener that, at every query progress event, writes out a manual checkpoint (from the QueryProgressEvent sourceOffset member that contains the last used source offsets). We gracefully stop the stream job every 6 hours, purge the _spark_metadata and spark checkpoints, and upon restart check for the existence of the manual checkpoint and use it if available. We do the stop/purge/restart via Airflow but it would be trivial to do this by looping around a stream awaitTermination with a provided timeout. A simple solution would be to just have an option to disable metadata file compaction that also allows old metadata files to be deleted after a delay. Currently it appears that all files stay around until compaction, upon which files older than the delay and not in the compaction are purged. > Purge Structured streaming FileStreamSinkLog metadata compact file data. > > > Key: SPARK-24295 > URL: https://issues.apache.org/jira/browse/SPARK-24295 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 2.3.0 >Reporter: Iqbal Singh >Priority: Major > Attachments: spark_metadatalog_compaction_perfbug_repro.tar.gz > > > FileStreamSinkLog metadata logs are concatenated to a single compact file > after defined compact interval. > For long running jobs, compact file size can grow up to 10's of GB's, Causing > slowness while reading the data from FileStreamSinkLog dir as spark is > defaulting to the "__spark__metadata" dir for the read. > We need a functionality to purge the compact file size. > -- 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
[jira] [Comment Edited] (SPARK-24295) Purge Structured streaming FileStreamSinkLog metadata compact file data.
[ https://issues.apache.org/jira/browse/SPARK-24295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16777197#comment-16777197 ] Alfredo Gimenez edited comment on SPARK-24295 at 2/25/19 6:57 PM: -- We've run into the exact same issue, I uploaded a minimal reproducible example showing the continuously growing metadata compaction files. This is especially an issue in streaming jobs that rely on checkpointing, as we cannot purge metadata files and restart–the checkpointing mechanism depends on the metadata. A current workaround we have is to manually grab the last checkpoint offsets, purge both checkpoints and metadata, and set the "startingOffsets" to the latest offsets that we grabbed. This is obviously not ideal, as it relies on the current serialized data structure for the checkpoints, which can change with spark versions. It also introduces the possibility of losing checkpoint data if a spark job fails before creating a new checkpoint file. [~kabhwan] taking a look at your PR now, thanks! Is there another reliable workaround for this setup? was (Author: alfredo-gimenez-bv): We've run into the exact same issue, I uploaded a minimal reproducible example showing the continuously growing metadata compaction files. This is especially an issue in streaming jobs that rely on checkpointing, as we cannot purge metadata files and restart–the checkpointing mechanism depends on the metadata. A current workaround we have is to manually grab the last checkpoint offsets, purge both checkpoints and metadata, and set the "startingOffsets" to the latest offsets that we grabbed. This is obviously not ideal, as it relies on the current serialized data structure for the checkpoints, which can change with spark versions. It also introduces the possibility of losing checkpoint data if a spark job fails before creating a new checkpoint file. [~kabhwan] can you point us to your PR? Is there another reliable workaround for this setup? > Purge Structured streaming FileStreamSinkLog metadata compact file data. > > > Key: SPARK-24295 > URL: https://issues.apache.org/jira/browse/SPARK-24295 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 2.3.0 >Reporter: Iqbal Singh >Priority: Major > Attachments: spark_metadatalog_compaction_perfbug_repro.tar.gz > > > FileStreamSinkLog metadata logs are concatenated to a single compact file > after defined compact interval. > For long running jobs, compact file size can grow up to 10's of GB's, Causing > slowness while reading the data from FileStreamSinkLog dir as spark is > defaulting to the "__spark__metadata" dir for the read. > We need a functionality to purge the compact file size. > -- 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
[jira] [Commented] (SPARK-24295) Purge Structured streaming FileStreamSinkLog metadata compact file data.
[ https://issues.apache.org/jira/browse/SPARK-24295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16777197#comment-16777197 ] Alfredo Gimenez commented on SPARK-24295: - We've run into the exact same issue, I uploaded a minimal reproducible example showing the continuously growing metadata compaction files. This is especially an issue in streaming jobs that rely on checkpointing, as we cannot purge metadata files and restart–the checkpointing mechanism depends on the metadata. A current workaround we have is to manually grab the last checkpoint offsets, purge both checkpoints and metadata, and set the "startingOffsets" to the latest offsets that we grabbed. This is obviously not ideal, as it relies on the current serialized data structure for the checkpoints, which can change with spark versions. It also introduces the possibility of losing checkpoint data if a spark job fails before creating a new checkpoint file. [~kabhwan] can you point us to your PR? Is there another reliable workaround for this setup? > Purge Structured streaming FileStreamSinkLog metadata compact file data. > > > Key: SPARK-24295 > URL: https://issues.apache.org/jira/browse/SPARK-24295 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 2.3.0 >Reporter: Iqbal Singh >Priority: Major > Attachments: spark_metadatalog_compaction_perfbug_repro.tar.gz > > > FileStreamSinkLog metadata logs are concatenated to a single compact file > after defined compact interval. > For long running jobs, compact file size can grow up to 10's of GB's, Causing > slowness while reading the data from FileStreamSinkLog dir as spark is > defaulting to the "__spark__metadata" dir for the read. > We need a functionality to purge the compact file size. > -- 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
[jira] [Updated] (SPARK-24295) Purge Structured streaming FileStreamSinkLog metadata compact file data.
[ https://issues.apache.org/jira/browse/SPARK-24295?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alfredo Gimenez updated SPARK-24295: Attachment: spark_metadatalog_compaction_perfbug_repro.tar.gz > Purge Structured streaming FileStreamSinkLog metadata compact file data. > > > Key: SPARK-24295 > URL: https://issues.apache.org/jira/browse/SPARK-24295 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 2.3.0 >Reporter: Iqbal Singh >Priority: Major > Attachments: spark_metadatalog_compaction_perfbug_repro.tar.gz > > > FileStreamSinkLog metadata logs are concatenated to a single compact file > after defined compact interval. > For long running jobs, compact file size can grow up to 10's of GB's, Causing > slowness while reading the data from FileStreamSinkLog dir as spark is > defaulting to the "__spark__metadata" dir for the read. > We need a functionality to purge the compact file size. > -- 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