Skip to content
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 @@ -18,7 +18,7 @@
package org.apache.spark.streaming

import java.io.InputStream
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}

import scala.collection.Map
import scala.collection.mutable.Queue
Expand All @@ -28,16 +28,17 @@ import akka.actor.{Props, SupervisorStrategy}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.{BytesWritable, LongWritable, Text}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}

import org.apache.spark._
import org.apache.spark.annotation.Experimental
import org.apache.spark.input.FixedLengthBinaryInputFormat
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream._
import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver}
import org.apache.spark.streaming.scheduler._
import org.apache.spark.streaming.scheduler.{JobScheduler, StreamingListener}
import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab}

/**
Expand Down Expand Up @@ -518,6 +519,7 @@ class StreamingContext private[streaming] (
* @throws SparkException if the context has already been started or stopped.
*/
def start(): Unit = synchronized {
import StreamingContext._
if (state == Started) {
throw new SparkException("StreamingContext has already been started")
}
Expand All @@ -526,8 +528,12 @@ class StreamingContext private[streaming] (
}
validate()
sparkContext.setCallSite(DStream.getCreationSite())
scheduler.start()
state = Started
ACTIVATION_LOCK.synchronized {
assertNoOtherContextIsActive()
scheduler.start()
state = Started
setActiveContext(this)
}
}

/**
Expand Down Expand Up @@ -598,6 +604,7 @@ class StreamingContext private[streaming] (
uiTab.foreach(_.detach())
// The state should always be Stopped after calling `stop()`, even if we haven't started yet:
state = Stopped
StreamingContext.setActiveContext(null)
}
}

Expand All @@ -607,8 +614,27 @@ class StreamingContext private[streaming] (
*/

object StreamingContext extends Logging {
/**
* Lock that guards access to global variables that track active StreamingContext.
*/
private val ACTIVATION_LOCK = new Object()

private[streaming] val DEFAULT_CLEANER_TTL = 3600
private var activeContext: AtomicReference[StreamingContext] =
new AtomicReference[StreamingContext](null)

private def assertNoOtherContextIsActive(): Unit = {
ACTIVATION_LOCK.synchronized {
if (activeContext.get() != null) {
throw new SparkException("Only one StreamingContext may be started in this JVM")
Copy link
Contributor

Choose a reason for hiding this comment

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

To clarify, we're allowed to start multiple StreamingContexts provided that only one context is running at a time, right? Since that's the case, what do you think about rewording this to something like "Only one StreamingContext may be running in this JVM"? I just worry that "started" might be misinterpreted as saying that you can only create one SparkContext per JVM.

We don't need to add an explicit test for starting a new StreamingContext after stopping the old one since this case is implicitly handled by all of our other test suites.

Copy link
Contributor

Choose a reason for hiding this comment

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

Also, I think that core has some logic to print the call site of the active context in the error message, which can sometimes be a useful debugging aid when determining where contexts are being leaked.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Well, let me be a little bit more unambiguous in the choice of verbs. We are allowed to instantiate multiple StreamingContexts provided only one is started (ssc.start()) at a time. That's why I used "started" so that it directly maps to ssc.start(). This is different from SparkContexts where it is started as soon as it is instantiated.

I will add the callsite information.

}
}
}

private def setActiveContext(ssc: StreamingContext): Unit = {
ACTIVATION_LOCK.synchronized {
activeContext.set(ssc)
}
}

@deprecated("Replaced by implicit functions in the DStream companion object. This is " +
"kept here only for backward compatibility.", "1.3.0")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -465,6 +465,24 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
}
}

test("multiple streaming contexts") {
sc = new SparkContext(new SparkConf().setMaster(master).setAppName(appName))
ssc = new StreamingContext(sc, Seconds(1))
val input = addInputStream(ssc)
input.foreachRDD { rdd => rdd.count }
ssc.start()

// Creating another streaming context should not create errors
val anotherSsc = new StreamingContext(sc, Seconds(10))
val anotherInput = addInputStream(anotherSsc)
anotherInput.foreachRDD { rdd => rdd.count }

val exception = intercept[SparkException] {
anotherSsc.start()
}
assert(exception.getMessage.contains("StreamingContext"), "Did not get the right exception")
}

test("DStream and generated RDD creation sites") {
testPackage.test()
}
Expand Down