Skip to content
Snippets Groups Projects
Commit d85bb10c authored by Shixiong Zhu's avatar Shixiong Zhu
Browse files

[SPARK-16116][SQL] ConsoleSink should not require checkpointLocation

## What changes were proposed in this pull request?

When the user uses `ConsoleSink`, we should use a temp location if `checkpointLocation` is not specified.

## How was this patch tested?

The added unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13817 from zsxwing/console-checkpoint.
parent b5a99766
No related branches found
No related tags found
No related merge requests found
......@@ -272,6 +272,12 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
useTempCheckpointLocation = true,
trigger = trigger)
} else {
val (useTempCheckpointLocation, recoverFromCheckpointLocation) =
if (source == "console") {
(true, false)
} else {
(false, true)
}
val dataSource =
DataSource(
df.sparkSession,
......@@ -284,6 +290,8 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
df,
dataSource.createSink(outputMode),
outputMode,
useTempCheckpointLocation = useTempCheckpointLocation,
recoverFromCheckpointLocation = recoverFromCheckpointLocation,
trigger = trigger)
}
}
......
......@@ -457,4 +457,14 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter {
}
}
}
test("ConsoleSink should not require checkpointLocation") {
LastOptions.clear()
val df = spark.readStream
.format("org.apache.spark.sql.streaming.test")
.load()
val sq = df.writeStream.format("console").start()
sq.stop()
}
}
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment