-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-29352][SQL][SS] Track active streaming queries in the SparkSession.sharedState #26018
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 6 commits
08be288
b95474a
b5eb762
a91f017
0ddb7e5
b808d33
d275ec6
37b43ec
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 |
|---|---|---|
|
|
@@ -352,8 +352,10 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo | |
| } | ||
| } | ||
|
|
||
| // Make sure no other query with same id is active | ||
| if (activeQueries.values.exists(_.id == query.id)) { | ||
| // 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 " + | ||
|
|
@@ -370,19 +372,15 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo | |
| query.streamingQuery.start() | ||
| } catch { | ||
| case e: Throwable => | ||
| activeQueriesLock.synchronized { | ||
| activeQueries -= query.id | ||
| } | ||
| unregisterTerminatedStream(query.id) | ||
| throw e | ||
| } | ||
| query | ||
| } | ||
|
|
||
| /** Notify (by the StreamingQuery) that the query has been terminated */ | ||
| private[sql] def notifyQueryTermination(terminatedQuery: StreamingQuery): Unit = { | ||
| activeQueriesLock.synchronized { | ||
| activeQueries -= terminatedQuery.id | ||
| } | ||
| unregisterTerminatedStream(terminatedQuery.id) | ||
| awaitTerminationLock.synchronized { | ||
| if (lastTerminatedQuery == null || terminatedQuery.exception.nonEmpty) { | ||
| lastTerminatedQuery = terminatedQuery | ||
|
|
@@ -391,4 +389,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 | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
|
|
||
| package org.apache.spark.sql.streaming | ||
|
|
||
| import java.io.File | ||
| import java.util.concurrent.CountDownLatch | ||
|
|
||
| import scala.concurrent.Future | ||
|
|
@@ -28,7 +29,7 @@ import org.scalatest.time.Span | |
| import org.scalatest.time.SpanSugar._ | ||
|
|
||
| import org.apache.spark.SparkException | ||
| import org.apache.spark.sql.Dataset | ||
| import org.apache.spark.sql.{Dataset, Encoders} | ||
| import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.streaming.util.BlockingSource | ||
|
|
@@ -242,6 +243,83 @@ class StreamingQueryManagerSuite extends StreamTest { | |
| } | ||
| } | ||
|
|
||
| testQuietly("can't start a streaming query with the same name in the same session") { | ||
| val ds1 = makeDataset._2 | ||
| val ds2 = makeDataset._2 | ||
| val queryName = "abc" | ||
|
|
||
| val query1 = ds1.writeStream.format("noop").queryName(queryName).start() | ||
| try { | ||
| val e = intercept[IllegalArgumentException] { | ||
| ds2.writeStream.format("noop").queryName(queryName).start() | ||
| } | ||
| assert(e.getMessage.contains("query with that name is already active")) | ||
| } finally { | ||
| query1.stop() | ||
| } | ||
| } | ||
|
|
||
| testQuietly("can start a streaming query with the same name in a different session") { | ||
| val session2 = spark.cloneSession() | ||
|
|
||
| val ds1 = MemoryStream(Encoders.INT, spark.sqlContext).toDS() | ||
| val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() | ||
| val queryName = "abc" | ||
|
|
||
| val query1 = ds1.writeStream.format("noop").queryName(queryName).start() | ||
| val query2 = ds2.writeStream.format("noop").queryName(queryName).start() | ||
|
|
||
| query1.stop() | ||
| query2.stop() | ||
| } | ||
|
|
||
| testQuietly("can't start multiple instances of the same streaming query in the same session") { | ||
|
Member
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. The test case names seems to be switched, @brkyvz . |
||
| withTempDir { dir => | ||
| val session2 = spark.cloneSession() | ||
|
|
||
| val ms1 = MemoryStream(Encoders.INT, spark.sqlContext) | ||
| val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() | ||
| val chkLocation = new File(dir, "_checkpoint").getCanonicalPath | ||
| val dataLocation = new File(dir, "data").getCanonicalPath | ||
|
|
||
| val query1 = ms1.toDS().writeStream.format("parquet") | ||
| .option("checkpointLocation", chkLocation).start(dataLocation) | ||
| ms1.addData(1, 2, 3) | ||
| try { | ||
| val e = intercept[IllegalStateException] { | ||
| ds2.writeStream.format("parquet") | ||
| .option("checkpointLocation", chkLocation).start(dataLocation) | ||
| } | ||
| assert(e.getMessage.contains("same id")) | ||
| } finally { | ||
| query1.stop() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| testQuietly( | ||
|
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. Just a sake of understanding, this patch is intended to prevent starting multiple instances of the same streaming query in the different sessions (while it was allowed to do this and it would probably incur some problem), right?
Contributor
Author
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. yes. This is the specific test which would have previously failed.
Member
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. Technically, this test case doesn't fail without this patch because this is for single session test case. The above test case (line 276) fails without this patch correctly. |
||
| "can't start multiple instances of the same streaming query in the different sessions") { | ||
|
Member
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. According to the test body,
Contributor
Author
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. good catch. I named them wrong :) |
||
| withTempDir { dir => | ||
| val (ms1, ds1) = makeDataset | ||
| val (ms2, ds2) = makeDataset | ||
| val chkLocation = new File(dir, "_checkpoint").getCanonicalPath | ||
| val dataLocation = new File(dir, "data").getCanonicalPath | ||
|
|
||
| val query1 = ds1.writeStream.format("parquet") | ||
| .option("checkpointLocation", chkLocation).start(dataLocation) | ||
| ms1.addData(1, 2, 3) | ||
| try { | ||
| val e = intercept[IllegalStateException] { | ||
| ds2.writeStream.format("parquet") | ||
| .option("checkpointLocation", chkLocation).start(dataLocation) | ||
| } | ||
| assert(e.getMessage.contains("same id")) | ||
| } finally { | ||
| query1.stop() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** Run a body of code by defining a query on each dataset */ | ||
| private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[StreamingQuery] => Unit): Unit = { | ||
| failAfter(streamingTimeout) { | ||
|
|
||
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.
Just curious: are you seeing the actual case where activeOption is None but activeQueries contain such query? I'm not seeing the case, though I don't think adding this would hurt.