17
17
18
18
package org .apache .spark .sql .streaming
19
19
20
- import java .util .UUID
20
+ import java .util .{ ConcurrentModificationException , UUID }
21
21
import java .util .concurrent .TimeUnit
22
22
import javax .annotation .concurrent .GuardedBy
23
23
@@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
37
37
import org .apache .spark .sql .execution .streaming .state .StateStoreCoordinatorRef
38
38
import org .apache .spark .sql .internal .SQLConf
39
39
import org .apache .spark .sql .internal .StaticSQLConf .STREAMING_QUERY_LISTENERS
40
- import org .apache .spark .util .{Clock , SystemClock , Utils }
40
+ import org .apache .spark .util .{Clock , SystemClock , ThreadUtils , Utils }
41
41
42
42
/**
43
43
* A class to manage all the [[StreamingQuery ]] active in a `SparkSession`.
@@ -51,9 +51,10 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
51
51
StateStoreCoordinatorRef .forDriver(sparkSession.sparkContext.env)
52
52
private val listenerBus = new StreamingQueryListenerBus (sparkSession.sparkContext.listenerBus)
53
53
54
- @ GuardedBy (" activeQueriesLock " )
54
+ @ GuardedBy (" activeQueriesSharedLock " )
55
55
private val activeQueries = new mutable.HashMap [UUID , StreamingQuery ]
56
- private val activeQueriesLock = new Object
56
+ // A global lock to keep track of active streaming queries across Spark sessions
57
+ private val activeQueriesSharedLock = sparkSession.sharedState.activeQueriesLock
57
58
private val awaitTerminationLock = new Object
58
59
59
60
@ GuardedBy (" awaitTerminationLock" )
@@ -77,7 +78,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
77
78
*
78
79
* @since 2.0.0
79
80
*/
80
- def active : Array [StreamingQuery ] = activeQueriesLock .synchronized {
81
+ def active : Array [StreamingQuery ] = activeQueriesSharedLock .synchronized {
81
82
activeQueries.values.toArray
82
83
}
83
84
@@ -86,7 +87,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
86
87
*
87
88
* @since 2.1.0
88
89
*/
89
- def get (id : UUID ): StreamingQuery = activeQueriesLock .synchronized {
90
+ def get (id : UUID ): StreamingQuery = activeQueriesSharedLock .synchronized {
90
91
activeQueries.get(id).orNull
91
92
}
92
93
@@ -343,27 +344,61 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
343
344
trigger,
344
345
triggerClock)
345
346
346
- activeQueriesLock.synchronized {
347
+ // The following code block checks if a stream with the same name or id is running. Then it
348
+ // returns an Option of an already active stream to stop outside of the lock
349
+ // to avoid a deadlock.
350
+ val activeRunOpt = activeQueriesSharedLock.synchronized {
347
351
// Make sure no other query with same name is active
348
352
userSpecifiedName.foreach { name =>
349
353
if (activeQueries.values.exists(_.name == name)) {
350
- throw new IllegalArgumentException (
351
- s " Cannot start query with name $name as a query with that name is already active" )
354
+ throw new IllegalArgumentException (s " Cannot start query with name $name as a query " +
355
+ s " with that name is already active in this SparkSession " )
352
356
}
353
357
}
354
358
355
359
// Make sure no other query with same id is active across all sessions
356
- val activeOption =
357
- Option (sparkSession.sharedState.activeStreamingQueries.putIfAbsent(query.id, this ))
358
- if (activeOption.isDefined || activeQueries.values.exists(_.id == query.id)) {
359
- throw new IllegalStateException (
360
- s " Cannot start query with id ${query.id} as another query with same id is " +
361
- s " already active. Perhaps you are attempting to restart a query from checkpoint " +
362
- s " that is already active. " )
360
+ val activeOption = Option (sparkSession.sharedState.activeStreamingQueries.get(query.id))
361
+ .orElse(activeQueries.get(query.id)) // shouldn't be needed but paranoia ...
362
+
363
+ val shouldStopActiveRun =
364
+ sparkSession.sessionState.conf.getConf(SQLConf .STREAMING_STOP_ACTIVE_RUN_ON_RESTART )
365
+ if (activeOption.isDefined) {
366
+ if (shouldStopActiveRun) {
367
+ val oldQuery = activeOption.get
368
+ logWarning(s " Stopping existing streaming query [id= ${query.id}, " +
369
+ s " runId= ${oldQuery.runId}], as a new run is being started. " )
370
+ Some (oldQuery)
371
+ } else {
372
+ throw new IllegalStateException (
373
+ s " Cannot start query with id ${query.id} as another query with same id is " +
374
+ s " already active. Perhaps you are attempting to restart a query from checkpoint " +
375
+ s " that is already active. You may stop the old query by setting the SQL " +
376
+ " configuration: " +
377
+ s """ spark.conf.set(" ${SQLConf .STREAMING_STOP_ACTIVE_RUN_ON_RESTART .key}", true) """ +
378
+ " and retry." )
379
+ }
380
+ } else {
381
+ // nothing to stop so, no-op
382
+ None
363
383
}
384
+ }
364
385
386
+ // stop() will clear the queryId from activeStreamingQueries as well as activeQueries
387
+ activeRunOpt.foreach(_.stop())
388
+
389
+ activeQueriesSharedLock.synchronized {
390
+ // We still can have a race condition when two concurrent instances try to start the same
391
+ // stream, while a third one was already active and stopped above. In this case, we throw a
392
+ // ConcurrentModificationException.
393
+ val oldActiveQuery = sparkSession.sharedState.activeStreamingQueries.put(
394
+ query.id, query.streamingQuery) // we need to put the StreamExecution, not the wrapper
395
+ if (oldActiveQuery != null ) {
396
+ throw new ConcurrentModificationException (
397
+ " Another instance of this query was just started by a concurrent session." )
398
+ }
365
399
activeQueries.put(query.id, query)
366
400
}
401
+
367
402
try {
368
403
// When starting a query, it will call `StreamingQueryListener.onQueryStarted` synchronously.
369
404
// As it's provided by the user and can run arbitrary codes, we must not hold any lock here.
@@ -372,15 +407,15 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
372
407
query.streamingQuery.start()
373
408
} catch {
374
409
case e : Throwable =>
375
- unregisterTerminatedStream(query.id )
410
+ unregisterTerminatedStream(query)
376
411
throw e
377
412
}
378
413
query
379
414
}
380
415
381
416
/** Notify (by the StreamingQuery) that the query has been terminated */
382
417
private [sql] def notifyQueryTermination (terminatedQuery : StreamingQuery ): Unit = {
383
- unregisterTerminatedStream(terminatedQuery.id )
418
+ unregisterTerminatedStream(terminatedQuery)
384
419
awaitTerminationLock.synchronized {
385
420
if (lastTerminatedQuery == null || terminatedQuery.exception.nonEmpty) {
386
421
lastTerminatedQuery = terminatedQuery
@@ -390,11 +425,12 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
390
425
stateStoreCoordinator.deactivateInstances(terminatedQuery.runId)
391
426
}
392
427
393
- private def unregisterTerminatedStream (terminatedQueryId : UUID ): Unit = {
394
- activeQueriesLock.synchronized {
395
- // remove from shared state only if the streaming query manager also matches
396
- sparkSession.sharedState.activeStreamingQueries.remove(terminatedQueryId, this )
397
- activeQueries -= terminatedQueryId
428
+ private def unregisterTerminatedStream (terminatedQuery : StreamingQuery ): Unit = {
429
+ activeQueriesSharedLock.synchronized {
430
+ // remove from shared state only if the streaming execution also matches
431
+ sparkSession.sharedState.activeStreamingQueries.remove(
432
+ terminatedQuery.id, terminatedQuery)
433
+ activeQueries -= terminatedQuery.id
398
434
}
399
435
}
400
436
}
0 commit comments