-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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-28709][DSTREAMS] Fix StreamingContext leak through Streaming #25439
Changes from all commits
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 |
---|---|---|
|
@@ -38,7 +38,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} | |
import org.apache.spark.deploy.SparkHadoopUtil | ||
import org.apache.spark.input.FixedLengthBinaryInputFormat | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.internal.config.UI._ | ||
import org.apache.spark.rdd.{RDD, RDDOperationScope} | ||
import org.apache.spark.scheduler.LiveListenerBus | ||
import org.apache.spark.serializer.SerializationDebugger | ||
|
@@ -189,10 +188,9 @@ class StreamingContext private[streaming] ( | |
private[streaming] val progressListener = new StreamingJobProgressListener(this) | ||
|
||
private[streaming] val uiTab: Option[StreamingTab] = | ||
if (conf.get(UI_ENABLED)) { | ||
Some(new StreamingTab(this)) | ||
} else { | ||
None | ||
sparkContext.ui match { | ||
case Some(ui) => Some(new StreamingTab(this, ui)) | ||
case None => None | ||
} | ||
|
||
/* Initializing a streamingSource to register metrics */ | ||
|
@@ -511,6 +509,10 @@ class StreamingContext private[streaming] ( | |
scheduler.listenerBus.addListener(streamingListener) | ||
} | ||
|
||
def removeStreamingListener(streamingListener: StreamingListener): Unit = { | ||
scheduler.listenerBus.removeListener(streamingListener) | ||
} | ||
|
||
private def validate() { | ||
assert(graph != null, "Graph is null") | ||
graph.validate() | ||
|
@@ -575,6 +577,8 @@ class StreamingContext private[streaming] ( | |
try { | ||
validate() | ||
|
||
registerProgressListener() | ||
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. I understand you are suggesting that the unregister is important, but do we need this? This seems to be a behavior change. 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. I think so. I believe 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 I tend to agree. The behavior change is registering listeners when when the UI is disabled, but that seems like possibly a fix (we'd have to look deeper to figure out whether something else registers it in this case, like |
||
|
||
// Start the streaming scheduler in a new thread, so that thread local properties | ||
// like call sites and job groups can be reset without affecting those of the | ||
// current thread. | ||
|
@@ -690,6 +694,9 @@ class StreamingContext private[streaming] ( | |
Utils.tryLogNonFatalError { | ||
uiTab.foreach(_.detach()) | ||
} | ||
Utils.tryLogNonFatalError { | ||
unregisterProgressListener() | ||
} | ||
StreamingContext.setActiveContext(null) | ||
Utils.tryLogNonFatalError { | ||
waiter.notifyStop() | ||
|
@@ -716,6 +723,18 @@ class StreamingContext private[streaming] ( | |
// Do not stop SparkContext, let its own shutdown hook stop it | ||
stop(stopSparkContext = false, stopGracefully = stopGracefully) | ||
} | ||
|
||
private def registerProgressListener(): Unit = { | ||
addStreamingListener(progressListener) | ||
sc.addSparkListener(progressListener) | ||
sc.ui.foreach(_.setStreamingJobProgressListener(progressListener)) | ||
} | ||
|
||
private def unregisterProgressListener(): Unit = { | ||
removeStreamingListener(progressListener) | ||
sc.removeSparkListener(progressListener) | ||
sc.ui.foreach(_.clearStreamingJobProgressListener()) | ||
} | ||
} | ||
|
||
/** | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -52,8 +52,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { | |
|
||
// Set up the streaming context and input streams | ||
withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => | ||
ssc.addStreamingListener(ssc.progressListener) | ||
|
||
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. If you remove the behavior change here( https://github.com/apache/spark/pull/25439/files#r313624949 ), it seems that we don't need to change this, right? 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. Correct. |
||
val input = Seq(1, 2, 3, 4, 5) | ||
// Use "batchCount" to make sure we check the result after all batches finish | ||
val batchCounter = new BatchCounter(ssc) | ||
|
@@ -106,8 +104,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { | |
testServer.start() | ||
|
||
withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => | ||
ssc.addStreamingListener(ssc.progressListener) | ||
|
||
val batchCounter = new BatchCounter(ssc) | ||
val networkStream = ssc.socketTextStream( | ||
"localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) | ||
|
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.
Is this new change related?
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.
Yes, handlers store reference to streaming and batch pages which store reference to
StreamingJobProgressListener
which stores reference toStreamingContext
. So all the handlers should be unregistered.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.
I see so that's kind of an additional bug fix, that this never recorded the handler in the list?
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.
I think so, seems that it wasn't an issue in past cause other tabs can't be detached in runtime.