Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.sql.execution.streaming

import java.io.IOException
import java.util.UUID
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.concurrent.locks.ReentrantLock
Expand All @@ -41,16 +42,20 @@ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils}
* Unlike a standard query, a streaming query executes repeatedly each time new data arrives at any
* [[Source]] present in the query plan. Whenever new data arrives, a [[QueryExecution]] is created
* and the results are committed transactionally to the given [[Sink]].
*
* @param deleteCheckpointOnStop whether to delete the checkpoint if the query is stopped without
* errors
*/
class StreamExecution(
override val sparkSession: SparkSession,
override val name: String,
checkpointRoot: String,
val checkpointRoot: String,
analyzedPlan: LogicalPlan,
val sink: Sink,
val trigger: Trigger,
val triggerClock: Clock,
val outputMode: OutputMode)
val outputMode: OutputMode,
deleteCheckpointOnStop: Boolean)
extends StreamingQuery with ProgressReporter with Logging {

import org.apache.spark.sql.streaming.StreamingQueryListener._
Expand Down Expand Up @@ -213,6 +218,7 @@ class StreamExecution(
* has been posted to all the listeners.
*/
def start(): Unit = {
logInfo(s"Starting $prettyIdString. Use $checkpointRoot to store the query checkpoint.")
microBatchThread.setDaemon(true)
microBatchThread.start()
startLatch.await() // Wait until thread started and QueryStart event has been posted
Expand Down Expand Up @@ -323,6 +329,20 @@ class StreamExecution(
sparkSession.streams.notifyQueryTermination(StreamExecution.this)
postEvent(
new QueryTerminatedEvent(id, runId, exception.map(_.cause).map(Utils.exceptionString)))

// Delete the temp checkpoint only when the query didn't fail
if (deleteCheckpointOnStop && exception.isEmpty) {
val checkpointPath = new Path(checkpointRoot)
try {
val fs = checkpointPath.getFileSystem(sparkSession.sessionState.newHadoopConf())
fs.delete(checkpointPath, true)
} catch {
case NonFatal(e) =>
// Deleting temp checkpoint folder is best effort, don't throw non fatal exceptions
// when we cannot delete them.
logWarning(s"Cannot delete $checkpointPath", e)
}
}
} finally {
terminationLatch.countDown()
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) {
recoverFromCheckpointLocation: Boolean,
trigger: Trigger,
triggerClock: Clock): StreamingQueryWrapper = {
var deleteCheckpointOnStop = false
val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified =>
new Path(userSpecified).toUri.toString
}.orElse {
Expand All @@ -203,6 +204,8 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) {
}
}.getOrElse {
if (useTempCheckpointLocation) {
// Delete the temp checkpoint when a query is being stopped without errors.
deleteCheckpointOnStop = true
Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath
} else {
throw new AnalysisException(
Expand Down Expand Up @@ -244,7 +247,8 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) {
sink,
trigger,
triggerClock,
outputMode))
outputMode,
deleteCheckpointOnStop))
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -670,4 +670,30 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter with Pr
}
}
}

test("temp checkpoint dir should be deleted if a query is stopped without errors") {
import testImplicits._
val query = MemoryStream[Int].toDS.writeStream.format("console").start()
val checkpointDir = new Path(
query.asInstanceOf[StreamingQueryWrapper].streamingQuery.checkpointRoot)
val fs = checkpointDir.getFileSystem(spark.sessionState.newHadoopConf())
assert(fs.exists(checkpointDir))
query.stop()
assert(!fs.exists(checkpointDir))
}

testQuietly("temp checkpoint dir should not be deleted if a query is stopped with an error") {
import testImplicits._
val input = MemoryStream[Int]
val query = input.toDS.map(_ / 0).writeStream.format("console").start()
val checkpointDir = new Path(
query.asInstanceOf[StreamingQueryWrapper].streamingQuery.checkpointRoot)
val fs = checkpointDir.getFileSystem(spark.sessionState.newHadoopConf())
assert(fs.exists(checkpointDir))
input.addData(1)
intercept[StreamingQueryException] {
query.awaitTermination()
}
assert(fs.exists(checkpointDir))
}
}