Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

Commit 9dc5fa5

Browse files
zsxwingtdas
authored andcommitted
[SPARK-18796][SS] StreamingQueryManager should not block when starting a query
## What changes were proposed in this pull request? Major change in this PR: - Add `pendingQueryNames` and `pendingQueryIds` to track that are going to start but not yet put into `activeQueries` so that we don't need to hold a lock when starting a query. Minor changes: - Fix a potential NPE when the user sets `checkpointLocation` using SQLConf but doesn't specify a query name. - Add missing docs in `StreamingQueryListener` ## How was this patch tested? Jenkins Author: Shixiong Zhu <[email protected]> Closes apache#16220 from zsxwing/SPARK-18796. (cherry picked from commit 417e45c) Signed-off-by: Tathagata Das <[email protected]>
1 parent 1aeb7f4 commit 9dc5fa5

File tree

4 files changed

+158
-58
lines changed

4 files changed

+158
-58
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -223,7 +223,8 @@ class StreamExecution(
223223
sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics)
224224
}
225225

226-
postEvent(new QueryStartedEvent(id, runId, name)) // Assumption: Does not throw exception.
226+
// `postEvent` does not throw non fatal exception.
227+
postEvent(new QueryStartedEvent(id, runId, name))
227228

228229
// Unblock starting thread
229230
startLatch.countDown()
@@ -286,7 +287,7 @@ class StreamExecution(
286287
e,
287288
committedOffsets.toOffsetSeq(sources, offsetSeqMetadata).toString,
288289
availableOffsets.toOffsetSeq(sources, offsetSeqMetadata).toString)
289-
logError(s"Query $name terminated with error", e)
290+
logError(s"Query $prettyIdString terminated with error", e)
290291
updateStatusMessage(s"Terminated with exception: ${e.getMessage}")
291292
// Rethrow the fatal errors to allow the user using `Thread.UncaughtExceptionHandler` to
292293
// handle them

sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -83,6 +83,9 @@ object StreamingQueryListener {
8383
/**
8484
* :: Experimental ::
8585
* Event representing the start of a query
86+
* @param id An unique query id that persists across restarts. See `StreamingQuery.id()`.
87+
* @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`.
88+
* @param name User-specified name of the query, null if not specified.
8689
* @since 2.1.0
8790
*/
8891
@Experimental
@@ -94,6 +97,7 @@ object StreamingQueryListener {
9497
/**
9598
* :: Experimental ::
9699
* Event representing any progress updates in a query.
100+
* @param progress The query progress updates.
97101
* @since 2.1.0
98102
*/
99103
@Experimental
@@ -103,7 +107,8 @@ object StreamingQueryListener {
103107
* :: Experimental ::
104108
* Event representing that termination of a query.
105109
*
106-
* @param id The query id.
110+
* @param id An unique query id that persists across restarts. See `StreamingQuery.id()`.
111+
* @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`.
107112
* @param exception The exception message of the query if the query was terminated
108113
* with an exception. Otherwise, it will be `None`.
109114
* @since 2.1.0

sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala

Lines changed: 93 additions & 55 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
package org.apache.spark.sql.streaming
1919

2020
import java.util.UUID
21-
import java.util.concurrent.atomic.AtomicLong
21+
import javax.annotation.concurrent.GuardedBy
2222

2323
import scala.collection.mutable
2424

@@ -44,10 +44,13 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) {
4444
private[sql] val stateStoreCoordinator =
4545
StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env)
4646
private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus)
47+
48+
@GuardedBy("activeQueriesLock")
4749
private val activeQueries = new mutable.HashMap[UUID, StreamingQuery]
4850
private val activeQueriesLock = new Object
4951
private val awaitTerminationLock = new Object
5052

53+
@GuardedBy("awaitTerminationLock")
5154
private var lastTerminatedQuery: StreamingQuery = null
5255

5356
/**
@@ -181,8 +184,65 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) {
181184
listenerBus.post(event)
182185
}
183186

187+
private def createQuery(
188+
userSpecifiedName: Option[String],
189+
userSpecifiedCheckpointLocation: Option[String],
190+
df: DataFrame,
191+
sink: Sink,
192+
outputMode: OutputMode,
193+
useTempCheckpointLocation: Boolean,
194+
recoverFromCheckpointLocation: Boolean,
195+
trigger: Trigger,
196+
triggerClock: Clock): StreamExecution = {
197+
val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified =>
198+
new Path(userSpecified).toUri.toString
199+
}.orElse {
200+
df.sparkSession.sessionState.conf.checkpointLocation.map { location =>
201+
new Path(location, userSpecifiedName.getOrElse(UUID.randomUUID().toString)).toUri.toString
202+
}
203+
}.getOrElse {
204+
if (useTempCheckpointLocation) {
205+
Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath
206+
} else {
207+
throw new AnalysisException(
208+
"checkpointLocation must be specified either " +
209+
"""through option("checkpointLocation", ...) or """ +
210+
s"""SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...)""")
211+
}
212+
}
213+
214+
// If offsets have already been created, we trying to resume a query.
215+
if (!recoverFromCheckpointLocation) {
216+
val checkpointPath = new Path(checkpointLocation, "offsets")
217+
val fs = checkpointPath.getFileSystem(df.sparkSession.sessionState.newHadoopConf())
218+
if (fs.exists(checkpointPath)) {
219+
throw new AnalysisException(
220+
s"This query does not support recovering from checkpoint location. " +
221+
s"Delete $checkpointPath to start over.")
222+
}
223+
}
224+
225+
val analyzedPlan = df.queryExecution.analyzed
226+
df.queryExecution.assertAnalyzed()
227+
228+
if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) {
229+
UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode)
230+
}
231+
232+
new StreamExecution(
233+
sparkSession,
234+
userSpecifiedName.orNull,
235+
checkpointLocation,
236+
analyzedPlan,
237+
sink,
238+
trigger,
239+
triggerClock,
240+
outputMode)
241+
}
242+
184243
/**
185244
* Start a [[StreamingQuery]].
245+
*
186246
* @param userSpecifiedName Query name optionally specified by the user.
187247
* @param userSpecifiedCheckpointLocation Checkpoint location optionally specified by the user.
188248
* @param df Streaming DataFrame.
@@ -206,72 +266,50 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) {
206266
recoverFromCheckpointLocation: Boolean = true,
207267
trigger: Trigger = ProcessingTime(0),
208268
triggerClock: Clock = new SystemClock()): StreamingQuery = {
209-
activeQueriesLock.synchronized {
210-
val name = userSpecifiedName match {
211-
case Some(n) =>
212-
if (activeQueries.values.exists(_.name == userSpecifiedName.get)) {
213-
throw new IllegalArgumentException(
214-
s"Cannot start query with name $n as a query with that name is already active")
215-
}
216-
n
217-
case None => null
218-
}
219-
val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified =>
220-
new Path(userSpecified).toUri.toString
221-
}.orElse {
222-
df.sparkSession.sessionState.conf.checkpointLocation.map { location =>
223-
new Path(location, name).toUri.toString
224-
}
225-
}.getOrElse {
226-
if (useTempCheckpointLocation) {
227-
Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath
228-
} else {
229-
throw new AnalysisException(
230-
"checkpointLocation must be specified either " +
231-
"""through option("checkpointLocation", ...) or """ +
232-
s"""SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...)""")
233-
}
234-
}
269+
val query = createQuery(
270+
userSpecifiedName,
271+
userSpecifiedCheckpointLocation,
272+
df,
273+
sink,
274+
outputMode,
275+
useTempCheckpointLocation,
276+
recoverFromCheckpointLocation,
277+
trigger,
278+
triggerClock)
235279

236-
// If offsets have already been created, we trying to resume a query.
237-
if (!recoverFromCheckpointLocation) {
238-
val checkpointPath = new Path(checkpointLocation, "offsets")
239-
val fs = checkpointPath.getFileSystem(df.sparkSession.sessionState.newHadoopConf())
240-
if (fs.exists(checkpointPath)) {
241-
throw new AnalysisException(
242-
s"This query does not support recovering from checkpoint location. " +
243-
s"Delete $checkpointPath to start over.")
280+
activeQueriesLock.synchronized {
281+
// Make sure no other query with same name is active
282+
userSpecifiedName.foreach { name =>
283+
if (activeQueries.values.exists(_.name == name)) {
284+
throw new IllegalArgumentException(
285+
s"Cannot start query with name $name as a query with that name is already active")
244286
}
245287
}
246288

247-
val analyzedPlan = df.queryExecution.analyzed
248-
df.queryExecution.assertAnalyzed()
249-
250-
if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) {
251-
UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode)
252-
}
253-
254-
val query = new StreamExecution(
255-
sparkSession,
256-
name,
257-
checkpointLocation,
258-
analyzedPlan,
259-
sink,
260-
trigger,
261-
triggerClock,
262-
outputMode)
263-
289+
// Make sure no other query with same id is active
264290
if (activeQueries.values.exists(_.id == query.id)) {
265291
throw new IllegalStateException(
266292
s"Cannot start query with id ${query.id} as another query with same id is " +
267-
s"already active. Perhaps you are attempting to restart a query from checkpoint" +
293+
s"already active. Perhaps you are attempting to restart a query from checkpoint " +
268294
s"that is already active.")
269295
}
270296

271-
query.start()
272297
activeQueries.put(query.id, query)
273-
query
274298
}
299+
try {
300+
// When starting a query, it will call `StreamingQueryListener.onQueryStarted` synchronously.
301+
// As it's provided by the user and can run arbitrary codes, we must not hold any lock here.
302+
// Otherwise, it's easy to cause dead-lock, or block too long if the user codes take a long
303+
// time to finish.
304+
query.start()
305+
} catch {
306+
case e: Throwable =>
307+
activeQueriesLock.synchronized {
308+
activeQueries -= query.id
309+
}
310+
throw e
311+
}
312+
query
275313
}
276314

277315
/** Notify (by the StreamingQuery) that the query has been terminated */

sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala

Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import org.scalatest.BeforeAndAfter
2727

2828
import org.apache.spark.sql._
2929
import org.apache.spark.sql.execution.streaming._
30+
import org.apache.spark.sql.internal.SQLConf
3031
import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider}
3132
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, StreamingQuery, StreamTest}
3233
import org.apache.spark.sql.types._
@@ -575,4 +576,59 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter {
575576
sq.stop()
576577
}
577578
}
579+
580+
test("user specified checkpointLocation precedes SQLConf") {
581+
import testImplicits._
582+
withTempDir { checkpointPath =>
583+
withTempPath { userCheckpointPath =>
584+
assert(!userCheckpointPath.exists(), s"$userCheckpointPath should not exist")
585+
withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) {
586+
val queryName = "test_query"
587+
val ds = MemoryStream[Int].toDS
588+
ds.writeStream
589+
.format("memory")
590+
.queryName(queryName)
591+
.option("checkpointLocation", userCheckpointPath.getAbsolutePath)
592+
.start()
593+
.stop()
594+
assert(checkpointPath.listFiles().isEmpty,
595+
"SQLConf path is used even if user specified checkpointLoc: " +
596+
s"${checkpointPath.listFiles()} is not empty")
597+
assert(userCheckpointPath.exists(),
598+
s"The user specified checkpointLoc (userCheckpointPath) is not created")
599+
}
600+
}
601+
}
602+
}
603+
604+
test("use SQLConf checkpoint dir when checkpointLocation is not specified") {
605+
import testImplicits._
606+
withTempDir { checkpointPath =>
607+
withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) {
608+
val queryName = "test_query"
609+
val ds = MemoryStream[Int].toDS
610+
ds.writeStream.format("memory").queryName(queryName).start().stop()
611+
// Should use query name to create a folder in `checkpointPath`
612+
val queryCheckpointDir = new File(checkpointPath, queryName)
613+
assert(queryCheckpointDir.exists(), s"$queryCheckpointDir doesn't exist")
614+
assert(
615+
checkpointPath.listFiles().size === 1,
616+
s"${checkpointPath.listFiles().toList} has 0 or more than 1 files ")
617+
}
618+
}
619+
}
620+
621+
test("use SQLConf checkpoint dir when checkpointLocation is not specified without query name") {
622+
import testImplicits._
623+
withTempDir { checkpointPath =>
624+
withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) {
625+
val ds = MemoryStream[Int].toDS
626+
ds.writeStream.format("console").start().stop()
627+
// Should create a random folder in `checkpointPath`
628+
assert(
629+
checkpointPath.listFiles().size === 1,
630+
s"${checkpointPath.listFiles().toList} has 0 or more than 1 files ")
631+
}
632+
}
633+
}
578634
}

0 commit comments

Comments
 (0)