Skip to content

Conversation

@brkyvz
Copy link
Contributor

@brkyvz brkyvz commented Oct 23, 2019

What changes were proposed in this pull request?

This PR adds a SQL Conf: spark.sql.streaming.stopActiveRunOnRestart. When this conf is true (by default it is), an already running stream will be stopped, if a new copy gets launched on the same checkpoint location.

Why are the changes needed?

In multi-tenant environments where you have multiple SparkSessions, you can accidentally start multiple copies of the same stream (i.e. streams using the same checkpoint location). This will cause all new instantiations of the new stream to fail. However, sometimes you may want to turn off the old stream, as the old stream may have turned into a zombie (you no longer have access to the query handle or SparkSession).

It would be nice to have a SQL flag that allows the stopping of the old stream for such zombie cases.

Does this PR introduce any user-facing change?

Yes. Now by default, if you launch a new copy of an already running stream on a multi-tenant cluster, the existing stream will be stopped.

How was this patch tested?

Unit tests in StreamingQueryManagerSuite

@SparkQA
Copy link

SparkQA commented Oct 23, 2019

Test build #112529 has finished for PR 26225 at commit 1d4167f.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

val queryManager = activeOption.getOrElse(this)
logInfo(s"Stopping existing streaming query [id=${query.id}], as a new run is being " +
"started.")
queryManager.get(query.id).stop()
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the existing stream is a "zombie", can it happen that it does not respond to stop() and then this will block forever?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great question. I can add some safeguards against this, but in most cases we mean that the stream is a "zombie", because we lost all references to it, not because it is uninterruptable.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this should be fine. If stop returns, the query should already be stopped, because stop waits until the streaming thread dies.

Copy link
Member

@zsxwing zsxwing Nov 5, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, this has a deadlock. We are waiting for stopping query inside a lock which the query needs to remove itself from the active queries.

@SparkQA
Copy link

SparkQA commented Oct 23, 2019

Test build #112530 has finished for PR 26225 at commit e30ec9a.

  • This patch fails from timeout after a configured wait of 400m.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Oct 24, 2019

Test build #112590 has finished for PR 26225 at commit e30ec9a.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

sparkSession.sessionState.conf.getConf(SQLConf.STOP_RUNNING_DUPLICATE_STREAM)
if (streamAlreadyActive && turnOffOldStream) {
val queryManager = activeOption.getOrElse(this)
logInfo(s"Stopping existing streaming query [id=${query.id}], as a new run is being " +
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: whether giving a warning is better?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree, make this a warning, and add the previous runId and new runId to make it easier to debug.

@dongjoon-hyun
Copy link
Member

Retest this please.

"older stream's SparkSession may not be possible, and the stream may have turned into a " +
"zombie stream. When this flag is true, we will stop the old stream to start the new one.")
.booleanConf
.createWithDefault(true)
Copy link
Member

@dongjoon-hyun dongjoon-hyun Oct 24, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we have false by default to avoid the behavior changes?
cc @gatorsmile

Copy link
Contributor Author

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:

  1. This change is going into Spark 3.0, a release where we can actually break existing behavior (unless it is critical behavior which people depend on)
  2. The existing behavior was that any new start of a stream would fail, because an existing stream was already running. This is programming error on the user's part.
  3. However, there are legitimate cases, where a user would like to restart a new instance of the stream (because they upgrade the code for instance), but they have no way of stopping the existing stream, because it turns into a zombie.

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.

Copy link
Contributor Author

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:

  1. This change is going into Spark 3.0, a release where we can actually break existing behavior (unless it is critical behavior which people depend on)
  2. The existing behavior was that any new start of a stream would fail, because an existing stream was already running. This is programming error on the user's part.
  3. However, there are legitimate cases, where a user would like to restart a new instance of the stream (because they upgrade the code for instance), but they have no way of stopping the existing stream, because it turns into a zombie.

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.

Copy link
Member

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.

Copy link
Contributor

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.

  • it seems that this will work only when the stream is restarted in a different session. but is it s
  • the term stream is confusing here. does it refer to a streaming query, a query run? We should try to be clear by same starting a "streaming query" instead of a "stream" in the explanation, and depending on what is consistent with other confs.

Copy link
Contributor

@tdas tdas Nov 13, 2019

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.

.checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
.createWithDefault(2)

val STOP_RUNNING_DUPLICATE_STREAM = buildConf("spark.sql.streaming.stopExistingDuplicateStream")
Copy link
Member

@dongjoon-hyun dongjoon-hyun Oct 24, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

stopExistingDuplicateStream -> stopExistingDuplicatedStream?

@SparkQA
Copy link

SparkQA commented Oct 25, 2019

Test build #112621 has finished for PR 26225 at commit e30ec9a.

  • This patch fails from timeout after a configured wait of 400m.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gaborgsomogyi
Copy link
Contributor

retest this please

@SparkQA
Copy link

SparkQA commented Oct 29, 2019

Test build #112847 has finished for PR 26225 at commit e30ec9a.

  • This patch fails from timeout after a configured wait of 400m.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

Seems like it affects testing time considerably.

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Oct 30, 2019

Test build #112874 has finished for PR 26225 at commit e30ec9a.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@uncleGen
Copy link
Contributor

retest this please

@gaborgsomogyi
Copy link
Contributor

Seems like it affects testing time considerably.

+1 on this

@SparkQA
Copy link

SparkQA commented Oct 30, 2019

Test build #112900 has finished for PR 26225 at commit e30ec9a.

  • This patch fails from timeout after a configured wait of 400m.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@zsxwing zsxwing left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

One minor comment about the error message. Otherwise looks good to me.

"Cannot start query with id ${query.id} as another query with same id is " +
"already active. Perhaps you are attempting to restart a query from checkpoint " +
"that is already active. You may stop the old query by setting the SQL " +
s"""configuration: spark.conf.set("${SQLConf.STOP_RUNNING_DUPLICATE_STREAM}", true).""")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SQLConf.STOP_RUNNING_DUPLICATE_STREAM => SQLConf.STOP_RUNNING_DUPLICATE_STREAM.key
And

You may stop the old query by setting the SQL ... and retry.

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 7, 2019

There was a deadlock causing tests to fail. cc @zsxwing @tdas @dongjoon-hyun addressed your comments. Can you ptal?

@SparkQA
Copy link

SparkQA commented Nov 7, 2019

Test build #113349 has finished for PR 26225 at commit ff14e95.

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

@SparkQA
Copy link

SparkQA commented Nov 7, 2019

Test build #113351 has finished for PR 26225 at commit d8d4e8f.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 7, 2019

Test build #113394 has finished for PR 26225 at commit d999fb7.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 7, 2019

Test build #113404 has finished for PR 26225 at commit 2216fe2.

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

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 8, 2019

retest this please

@SparkQA
Copy link

SparkQA commented Nov 8, 2019

Test build #113468 has finished for PR 26225 at commit 2216fe2.

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

@dongjoon-hyun
Copy link
Member

Thank you for updating, @brkyvz !

.checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
.createWithDefault(2)

val STOP_RUNNING_DUPLICATE_STREAM = buildConf("spark.sql.streaming.stopExistingDuplicatedStream")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

STOP_RUNNING_DUPLICATE_STREAM -> STOP_RUNNING_DUPLICATED_STREAM?

// 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 activeDuplicateQuery = activeQueriesLock.synchronized {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

activeDuplicateQuery -> activeDuplicatedQuery?

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 10, 2019 via email

@dongjoon-hyun
Copy link
Member

Oops. I overlooked the above comment. Thanks, @brkyvz !

Copy link
Contributor

@tdas tdas left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Almost LGTM. I am mostly grumbling about the names.

throw new IllegalArgumentException(
s"Cannot start query with name $name as a query with that name is already active")
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The 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

// 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 activeDuplicateQuery = activeQueriesLock.synchronized {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe rename this to activeQuerySharedLock to indicate this is shared across sessions.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

or directly use sharedState.activeQueryLock

val activeOption = Option(sparkSession.sharedState.activeStreamingQueries.get(query.id))
.orElse(activeQueries.get(query.id))

val turnOffOldStream =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TuneOff?? make it same as the conf.

"older stream's SparkSession may not be possible, and the stream may have turned into a " +
"zombie stream. When this flag is true, we will stop the old stream to start the new one.")
.booleanConf
.createWithDefault(true)
Copy link
Contributor

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.

  • it seems that this will work only when the stream is restarted in a different session. but is it s
  • the term stream is confusing here. does it refer to a streaming query, a query run? We should try to be clear by same starting a "streaming query" instead of a "stream" in the explanation, and depending on what is consistent with other confs.

.checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
.createWithDefault(2)

val STOP_RUNNING_DUPLICATE_STREAM = buildConf("spark.sql.streaming.stopExistingDuplicatedStream")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

make the conf CAPS name consistent with the actual string.

Option(sparkSession.sharedState.activeStreamingQueries.putIfAbsent(query.id, this))
if (activeOption.isDefined || activeQueries.values.exists(_.id == query.id)) {
val activeOption = Option(sparkSession.sharedState.activeStreamingQueries.get(query.id))
.orElse(activeQueries.get(query.id))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we need to check both? can it be in the shared state and but not in the local one?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

paranoia

}
assert(e.getMessage.contains("same id"))
} finally {
query1.stop()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

stop all active streams

assert(!query1.isActive,
"First query should have stopped before starting the second query")
} finally {
query2.stop()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

stop all active streams

assert(!query1.isActive,
"First query should have stopped before starting the second query")
} finally {
query2.stop()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

stop all active streams

@SparkQA
Copy link

SparkQA commented Nov 12, 2019

Test build #113650 has finished for PR 26225 at commit 9fbf56a.

  • This patch fails Spark unit tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

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.")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is not the correct error message when stopActiveRunOnRestart is false.
If the active run was stopped, then this error message is correct.
If the active run was not stopped, then this error will be thrown and therefore should simply say that there is an active run (run id ...).

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.

activeRunOpt.foreach(_.stop())

activeQueriesSharedLock.synchronized {
// We still can have a race condition when two concurrent instances try to start the same
Copy link
Contributor

@tdas tdas Nov 13, 2019

Choose a reason for hiding this comment

The 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.

}
}

activeRunOpt.foreach(_.stop())
Copy link
Contributor

@tdas tdas Nov 13, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Please document here that stop() will automatically clear the activeStreamingQueries. Without this implicit easy-to-miss information, it is hard to reason about this code.

@tdas
Copy link
Contributor

tdas commented Nov 13, 2019

The implementation looks good, but please fix the error message before merging.

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 13, 2019 via email

@tdas
Copy link
Contributor

tdas commented Nov 13, 2019

Aah. I was confused. LGTM then.

@tdas
Copy link
Contributor

tdas commented Nov 13, 2019

Update the description with the new conf name.

@SparkQA
Copy link

SparkQA commented Nov 13, 2019

Test build #113657 has finished for PR 26225 at commit 3cea936.

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

@SparkQA
Copy link

SparkQA commented Nov 13, 2019

Test build #113667 has finished for PR 26225 at commit bff9162.

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

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 13, 2019

Thanks! Merging to master

@asfgit asfgit closed this in 363af16 Nov 13, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

9 participants