Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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,17 +28,19 @@ 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}
import org.apache.spark.util.CallSite

/**
* Main entry point for Spark Streaming functionality. It provides methods used to create
Expand Down Expand Up @@ -202,6 +204,8 @@ class StreamingContext private[streaming] (
import StreamingContextState._
private[streaming] var state = Initialized

private val startSite = new AtomicReference[CallSite](null)

/**
* Return the associated Spark context
*/
Expand Down Expand Up @@ -518,17 +522,23 @@ 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")
}
if (state == Stopped) {
throw new SparkException("StreamingContext has already been stopped")
}
validate()
sparkContext.setCallSite(DStream.getCreationSite())
scheduler.start()
uiTab.foreach(_.attach())
state = Started
startSite.set(DStream.getCreationSite())
sparkContext.setCallSite(startSite.get)
ACTIVATION_LOCK.synchronized {
assertNoOtherContextIsActive()
scheduler.start()
uiTab.foreach(_.attach())
state = Started
setActiveContext(this)
}
}

/**
Expand Down Expand Up @@ -599,6 +609,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 @@ -608,8 +619,29 @@ 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 val activeContext = 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. " +
"Currently running StreamingContext was started at" +
activeContext.get.startSite.get.longForm)
}
}
}

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