Skip to content
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

[SPARK-29352][SQL][SS] Track active streaming queries in the SparkSession.sharedState #26018

Closed
wants to merge 8 commits into from

Conversation

@brkyvz
Copy link
Contributor

brkyvz commented Oct 4, 2019

What changes were proposed in this pull request?

This moves the tracking of active queries from a per SparkSession state, to the shared SparkSession for better safety in isolated Spark Session environments.

Why are the changes needed?

We have checks to prevent the restarting of the same stream on the same spark session, but we can actually make that better in multi-tenant environments by actually putting that state in the SharedState instead of SessionState. This would allow a more comprehensive check for multi-tenant clusters.

Does this PR introduce any user-facing change?

No

How was this patch tested?

Added tests to StreamingQueryManagerSuite

@brkyvz

This comment has been minimized.

Copy link
Contributor Author

brkyvz commented Oct 4, 2019

@brkyvz brkyvz changed the title Track active streaming queries in the SparkSession.sharedState [SPARK-29352][SQL][SS] Track active streaming queries in the SparkSession.sharedState Oct 4, 2019
@SparkQA

This comment has been minimized.

Copy link

SparkQA commented Oct 4, 2019

Test build #111760 has finished for PR 26018 at commit 08be288.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
@SparkQA

This comment has been minimized.

Copy link

SparkQA commented Oct 4, 2019

Test build #111783 has started for PR 26018 at commit b95474a.

brkyvz added 3 commits Oct 4, 2019
… into sharedStreamingQueryManager
@dongjoon-hyun dongjoon-hyun added the SQL label Oct 4, 2019
}
}

testQuietly(

This comment has been minimized.

Copy link
@HeartSaVioR

HeartSaVioR Oct 5, 2019

Contributor

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?

This comment has been minimized.

Copy link
@brkyvz

brkyvz Oct 7, 2019

Author Contributor

yes. This is the specific test which would have previously failed.

This comment has been minimized.

Copy link
@dongjoon-hyun

dongjoon-hyun Oct 15, 2019

Member

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.

Copy link
Contributor

HeartSaVioR left a comment

LGTM. It would keep isolation of active queries between SparkSession and only add restriction of running same streaming query concurrently across multiple SparkSessions.

@SparkQA

This comment has been minimized.

Copy link

SparkQA commented Oct 5, 2019

Test build #111791 has finished for PR 26018 at commit 0ddb7e5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
@SparkQA

This comment has been minimized.

Copy link

SparkQA commented Oct 5, 2019

Test build #111790 has finished for PR 26018 at commit a91f017.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
Copy link
Member

zsxwing left a comment

LGTM

@@ -355,7 +355,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
// 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) {
if (activeOption.isDefined || activeQueries.values.exists(_.id == query.id) {

This comment has been minimized.

Copy link
@HeartSaVioR

HeartSaVioR Oct 11, 2019

Contributor

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.

@dongjoon-hyun

This comment has been minimized.

Copy link
Member

dongjoon-hyun commented Oct 12, 2019

Retest this please.

@SparkQA

This comment has been minimized.

Copy link

SparkQA commented Oct 12, 2019

Test build #111984 has finished for PR 26018 at commit b808d33.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
// 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) {

This comment has been minimized.

Copy link
@dongjoon-hyun

dongjoon-hyun Oct 13, 2019

Member

It seems that one ) is missed after query.id). Style check and compilation failure.

[error] /home/jenkins/workspace/SparkPullRequestBuilder@2/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala: Expected token RPAREN but got Token(DOT,.,14050,.)
[error] Total time: 20 s, completed Oct 12, 2019 4:11:49 PM
@SparkQA

This comment has been minimized.

Copy link

SparkQA commented Oct 13, 2019

Test build #111996 has finished for PR 26018 at commit d275ec6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
@dongjoon-hyun

This comment has been minimized.

Copy link
Member

dongjoon-hyun commented Oct 15, 2019

Retest this please.

query2.stop()
}

testQuietly("can't start multiple instances of the same streaming query in the same session") {

This comment has been minimized.

Copy link
@dongjoon-hyun

dongjoon-hyun Oct 15, 2019

Member

The test case names seems to be switched, @brkyvz .
in the same session -> in the different sessions?

}

testQuietly(
"can't start multiple instances of the same streaming query in the different sessions") {

This comment has been minimized.

Copy link
@dongjoon-hyun

dongjoon-hyun Oct 15, 2019

Member

According to the test body, in the different sessions -> in the same session?

This comment has been minimized.

Copy link
@brkyvz

brkyvz Oct 22, 2019

Author Contributor

good catch. I named them wrong :)

Copy link
Member

dongjoon-hyun left a comment

+1, LGTM (except the switched test case names). Thank you for adding more missed test cases, @brkyvz .

@SparkQA

This comment has been minimized.

Copy link

SparkQA commented Oct 15, 2019

Test build #112083 has finished for PR 26018 at commit d275ec6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
@SparkQA

This comment has been minimized.

Copy link

SparkQA commented Oct 22, 2019

Test build #112455 has finished for PR 26018 at commit 37b43ec.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
Copy link
Contributor

HeartSaVioR left a comment

LGTM again.

@brkyvz

This comment has been minimized.

Copy link
Contributor Author

brkyvz commented Oct 23, 2019

Merging to master! Thanks all :)

@asfgit asfgit closed this in cbe6ead Oct 23, 2019
ulysses-you pushed a commit to ulysses-you/spark that referenced this pull request Oct 31, 2019
…sion.sharedState

### What changes were proposed in this pull request?

This moves the tracking of active queries from a per SparkSession state, to the shared SparkSession for better safety in isolated Spark Session environments.

### Why are the changes needed?

We have checks to prevent the restarting of the same stream on the same spark session, but we can actually make that better in multi-tenant environments by actually putting that state in the SharedState instead of SessionState. This would allow a more comprehensive check for multi-tenant clusters.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Added tests to StreamingQueryManagerSuite

Closes apache#26018 from brkyvz/sharedStreamingQueryManager.

Lead-authored-by: Burak Yavuz <burak@databricks.com>
Co-authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
ulysses-you pushed a commit to ulysses-you/spark that referenced this pull request Oct 31, 2019
…sion.sharedState

### What changes were proposed in this pull request?

This moves the tracking of active queries from a per SparkSession state, to the shared SparkSession for better safety in isolated Spark Session environments.

### Why are the changes needed?

We have checks to prevent the restarting of the same stream on the same spark session, but we can actually make that better in multi-tenant environments by actually putting that state in the SharedState instead of SessionState. This would allow a more comprehensive check for multi-tenant clusters.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Added tests to StreamingQueryManagerSuite

Closes apache#26018 from brkyvz/sharedStreamingQueryManager.

Lead-authored-by: Burak Yavuz <burak@databricks.com>
Co-authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
5 participants
You can’t perform that action at this time.