Skip to content

Commit

Permalink
[SPARK-19779][SS] Delete needless tmp file after restart structured s…
Browse files Browse the repository at this point in the history
…treaming job

## What changes were proposed in this pull request?

[SPARK-19779](https://issues.apache.org/jira/browse/SPARK-19779)

The PR (#17012) can to fix restart a Structured Streaming application using hdfs as fileSystem, but also exist a problem that a tmp file of delta file is still reserved in hdfs. And Structured Streaming don't delete the tmp file generated when restart streaming job in future.

## How was this patch tested?
 unit tests

Author: guifeng <guifengleaf@gmail.com>

Closes #17124 from gf53520/SPARK-19779.
  • Loading branch information
gf53520 authored and zsxwing committed Mar 3, 2017
1 parent f37bb14 commit e24f21b
Show file tree
Hide file tree
Showing 2 changed files with 10 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -283,7 +283,9 @@ private[state] class HDFSBackedStateStoreProvider(
// semantically correct because Structured Streaming requires rerunning a batch should
// generate the same output. (SPARK-19677)
// scalastyle:on
if (!fs.exists(finalDeltaFile) && !fs.rename(tempDeltaFile, finalDeltaFile)) {
if (fs.exists(finalDeltaFile)) {
fs.delete(tempDeltaFile, true)
} else if (!fs.rename(tempDeltaFile, finalDeltaFile)) {
throw new IOException(s"Failed to rename $tempDeltaFile to $finalDeltaFile")
}
loadedMaps.put(newVersion, map)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,11 @@ package org.apache.spark.sql.execution.streaming.state
import java.io.{File, IOException}
import java.net.URI

import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.util.Random

import org.apache.commons.io.FileUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem}
import org.scalatest.{BeforeAndAfter, PrivateMethodTester}
Expand Down Expand Up @@ -293,6 +295,11 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth
val provider = newStoreProvider(hadoopConf = conf)
provider.getStore(0).commit()
provider.getStore(0).commit()

// Verify we don't leak temp files
val tempFiles = FileUtils.listFiles(new File(provider.id.checkpointLocation),
null, true).asScala.filter(_.getName.startsWith("temp-"))
assert(tempFiles.isEmpty)
}

test("corrupted file handling") {
Expand Down

0 comments on commit e24f21b

Please sign in to comment.