-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-29568][SS] Stop existing running streams when a new stream is launched #26225
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
1d4167f
e30ec9a
7b6b17c
ff14e95
30892ba
d8d4e8f
dd20574
d999fb7
2216fe2
9fbf56a
3cea936
bff9162
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,7 @@ | |
|
|
||
| package org.apache.spark.sql.streaming | ||
|
|
||
| import java.util.UUID | ||
| import java.util.{ConcurrentModificationException, UUID} | ||
| import java.util.concurrent.TimeUnit | ||
| import javax.annotation.concurrent.GuardedBy | ||
|
|
||
|
|
@@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution | |
| import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS | ||
| import org.apache.spark.util.{Clock, SystemClock, Utils} | ||
| import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} | ||
|
|
||
| /** | ||
| * A class to manage all the [[StreamingQuery]] active in a `SparkSession`. | ||
|
|
@@ -51,9 +51,10 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo | |
| StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) | ||
| private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus) | ||
|
|
||
| @GuardedBy("activeQueriesLock") | ||
| @GuardedBy("activeQueriesSharedLock") | ||
| private val activeQueries = new mutable.HashMap[UUID, StreamingQuery] | ||
| private val activeQueriesLock = new Object | ||
| // A global lock to keep track of active streaming queries across Spark sessions | ||
| private val activeQueriesSharedLock = sparkSession.sharedState.activeQueriesLock | ||
| private val awaitTerminationLock = new Object | ||
|
|
||
| @GuardedBy("awaitTerminationLock") | ||
|
|
@@ -77,7 +78,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo | |
| * | ||
| * @since 2.0.0 | ||
| */ | ||
| def active: Array[StreamingQuery] = activeQueriesLock.synchronized { | ||
| def active: Array[StreamingQuery] = activeQueriesSharedLock.synchronized { | ||
| activeQueries.values.toArray | ||
| } | ||
|
|
||
|
|
@@ -86,7 +87,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo | |
| * | ||
| * @since 2.1.0 | ||
| */ | ||
| def get(id: UUID): StreamingQuery = activeQueriesLock.synchronized { | ||
| def get(id: UUID): StreamingQuery = activeQueriesSharedLock.synchronized { | ||
| activeQueries.get(id).orNull | ||
| } | ||
|
|
||
|
|
@@ -343,27 +344,61 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo | |
| trigger, | ||
| triggerClock) | ||
|
|
||
| activeQueriesLock.synchronized { | ||
| // The following code block checks if a stream with the same name or id is running. Then it | ||
| // returns an Option of an already active stream to stop outside of the lock | ||
| // to avoid a deadlock. | ||
| val activeRunOpt = activeQueriesSharedLock.synchronized { | ||
| // Make sure no other query with same name is active | ||
| userSpecifiedName.foreach { name => | ||
| if (activeQueries.values.exists(_.name == name)) { | ||
| throw new IllegalArgumentException( | ||
| s"Cannot start query with name $name as a query with that name is already active") | ||
| throw new IllegalArgumentException(s"Cannot start query with name $name as a query " + | ||
| s"with that name is already active in this SparkSession") | ||
| } | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. super nit: probably update this to "is already active in this SparkSession." to be clearer |
||
|
|
||
| // Make sure no other query with same id is active across all sessions | ||
| val activeOption = | ||
| Option(sparkSession.sharedState.activeStreamingQueries.putIfAbsent(query.id, this)) | ||
| if (activeOption.isDefined || activeQueries.values.exists(_.id == query.id)) { | ||
| throw new IllegalStateException( | ||
| s"Cannot start query with id ${query.id} as another query with same id is " + | ||
| s"already active. Perhaps you are attempting to restart a query from checkpoint " + | ||
| s"that is already active.") | ||
| val activeOption = Option(sparkSession.sharedState.activeStreamingQueries.get(query.id)) | ||
| .orElse(activeQueries.get(query.id)) // shouldn't be needed but paranoia ... | ||
|
|
||
| val shouldStopActiveRun = | ||
| sparkSession.sessionState.conf.getConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART) | ||
| if (activeOption.isDefined) { | ||
| if (shouldStopActiveRun) { | ||
| val oldQuery = activeOption.get | ||
| logWarning(s"Stopping existing streaming query [id=${query.id}, " + | ||
| s"runId=${oldQuery.runId}], as a new run is being started.") | ||
| Some(oldQuery) | ||
| } else { | ||
| throw new IllegalStateException( | ||
| s"Cannot start query with id ${query.id} as another query with same id is " + | ||
| s"already active. Perhaps you are attempting to restart a query from checkpoint " + | ||
| s"that is already active. You may stop the old query by setting the SQL " + | ||
| "configuration: " + | ||
| s"""spark.conf.set("${SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key}", true) """ + | ||
| "and retry.") | ||
| } | ||
| } else { | ||
brkyvz marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| // nothing to stop so, no-op | ||
| None | ||
| } | ||
| } | ||
|
|
||
| // stop() will clear the queryId from activeStreamingQueries as well as activeQueries | ||
| activeRunOpt.foreach(_.stop()) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: Please document here that stop() will automatically clear the |
||
|
|
||
| activeQueriesSharedLock.synchronized { | ||
| // We still can have a race condition when two concurrent instances try to start the same | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: This comment is true only if the active run was stopped. So qualify the comment accordingly. |
||
| // stream, while a third one was already active and stopped above. In this case, we throw a | ||
| // ConcurrentModificationException. | ||
| val oldActiveQuery = sparkSession.sharedState.activeStreamingQueries.put( | ||
| query.id, query.streamingQuery) // we need to put the StreamExecution, not the wrapper | ||
| if (oldActiveQuery != null) { | ||
| throw new ConcurrentModificationException( | ||
| "Another instance of this query was just started by a concurrent session.") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is not the correct error message when In other words, this can stay as the same message as it was in Spark 2.4,.... may be improved by adding the run id. |
||
| } | ||
| activeQueries.put(query.id, query) | ||
| } | ||
|
|
||
| try { | ||
| // When starting a query, it will call `StreamingQueryListener.onQueryStarted` synchronously. | ||
| // 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 | |
| query.streamingQuery.start() | ||
| } catch { | ||
| case e: Throwable => | ||
| unregisterTerminatedStream(query.id) | ||
| unregisterTerminatedStream(query) | ||
| throw e | ||
| } | ||
| query | ||
| } | ||
|
|
||
| /** Notify (by the StreamingQuery) that the query has been terminated */ | ||
| private[sql] def notifyQueryTermination(terminatedQuery: StreamingQuery): Unit = { | ||
| unregisterTerminatedStream(terminatedQuery.id) | ||
| unregisterTerminatedStream(terminatedQuery) | ||
| awaitTerminationLock.synchronized { | ||
| if (lastTerminatedQuery == null || terminatedQuery.exception.nonEmpty) { | ||
| lastTerminatedQuery = terminatedQuery | ||
|
|
@@ -390,11 +425,12 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo | |
| stateStoreCoordinator.deactivateInstances(terminatedQuery.runId) | ||
| } | ||
|
|
||
| private def unregisterTerminatedStream(terminatedQueryId: UUID): Unit = { | ||
| activeQueriesLock.synchronized { | ||
| // remove from shared state only if the streaming query manager also matches | ||
| sparkSession.sharedState.activeStreamingQueries.remove(terminatedQueryId, this) | ||
| activeQueries -= terminatedQueryId | ||
| private def unregisterTerminatedStream(terminatedQuery: StreamingQuery): Unit = { | ||
| activeQueriesSharedLock.synchronized { | ||
| // remove from shared state only if the streaming execution also matches | ||
| sparkSession.sharedState.activeStreamingQueries.remove( | ||
| terminatedQuery.id, terminatedQuery) | ||
| activeQueries -= terminatedQuery.id | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shall we have
falseby default to avoid the behavior changes?cc @gatorsmile
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great question. Here's my argument why we should change it:
I would argue that 3 is more common than 2, and including 1, this is where we can change behavior and mention in release notes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great question. Here's my argument why we should change it:
I would argue that 3 is more common than 2, and including 1, this is where we can change behavior and mention in release notes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 for the release notes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: I think the docs can be better. here are confusing parts.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 one the name now. I like it.