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-7217][STREAMING] Add configuration to control the default behavior of StreamingContext.stop() implicitly calling SparkContext.stop() #5929

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -561,6 +561,15 @@ class StreamingContext private[streaming] (
waiter.waitForStopOrError(timeout)
}

/**
* Stop the execution of the streams immediately (does not wait for all received data
* to be processed). The underlying SparkContext will also be stopped. Note that this can
* be configured using the SparkConf configuration spark.streaming.stopSparkContextByDefault.
*/
def stop(): Unit = synchronized {
stop(conf.getBoolean("spark.streaming.stopSparkContextByDefault", true), false)
}

/**
* Stop the execution of the streams immediately (does not wait for all received data
* to be processed).
Expand All @@ -569,7 +578,7 @@ class StreamingContext private[streaming] (
* will be stopped regardless of whether this StreamingContext has been
* started.
*/
def stop(stopSparkContext: Boolean = true): Unit = synchronized {
def stop(stopSparkContext: Boolean): Unit = synchronized {
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, this is actually api incompatible change no? I am wondering if there is a workaround here which does not break compat.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I thought it wasnt, but it seems to be. I will implement it differently.
On May 5, 2015 6:16 PM, "Hari Shreedharan" [email protected] wrote:

In
streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
#5929 (comment):

* to be processed).
*
* @param stopSparkContext if true, stops the associated SparkContext. The underlying SparkContext
*                         will be stopped regardless of whether this StreamingContext has been
*                         started.
*/
  • def stop(stopSparkContext: Boolean = true): Unit = synchronized {
  • def stop(stopSparkContext: Boolean): Unit = synchronized {

Hmm, this is actually api incompatible change no? I am wondering if there
is a workaround here which does not break compat.


Reply to this email directly or view it on GitHub
https://github.com/apache/spark/pull/5929/files#r29728863.

stop(stopSparkContext, false)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,11 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
assert(ssc.state === ssc.StreamingContextState.Started)
ssc.stop()
assert(ssc.state === ssc.StreamingContextState.Stopped)

// Make sure that the SparkContext is also stopped by default
intercept[Exception] {
ssc.sparkContext.makeRDD(1 to 10)
}
}

test("start multiple times") {
Expand Down Expand Up @@ -154,16 +159,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
}

test("stop only streaming context") {
ssc = new StreamingContext(master, appName, batchDuration)
val conf = new SparkConf().setMaster(master).setAppName(appName)

// Explicitly do not stop SparkContext
ssc = new StreamingContext(conf, batchDuration)
sc = ssc.sparkContext
addInputStream(ssc).register()
ssc.start()
ssc.stop(stopSparkContext = false)
assert(sc.makeRDD(1 to 100).collect().size === 100)
ssc = new StreamingContext(sc, batchDuration)
sc.stop()

// Implicitly do not stop SparkContext
conf.set("spark.streaming.stopSparkContextByDefault", "false")
ssc = new StreamingContext(conf, batchDuration)
sc = ssc.sparkContext
addInputStream(ssc).register()
ssc.start()
ssc.stop()
assert(sc.makeRDD(1 to 100).collect().size === 100)
sc.stop()
}

test("stop(stopSparkContext=true) after stop(stopSparkContext=false)") {
Expand Down