diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index cfe387faec..ad3d8426bd 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -106,6 +106,10 @@ object MimaExcludes { "org.apache.spark.sql.parquet.ParquetTestData$"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.parquet.TestGroupWriteSupport") + ) ++ Seq( + // SPARK-7530 Added StreamingContext.getState() + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.streaming.StreamingContext.state_=") ) case v if v.startsWith("1.3") => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 5abe136775..2c5834defa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -32,10 +32,11 @@ 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.annotation.{DeveloperApi, Experimental} import org.apache.spark.input.FixedLengthBinaryInputFormat import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContextState._ import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver} import org.apache.spark.streaming.scheduler.{JobScheduler, StreamingListener} @@ -195,14 +196,7 @@ class StreamingContext private[streaming] ( assert(env.metricsSystem != null) env.metricsSystem.registerSource(streamingSource) - /** Enumeration to identify current state of the StreamingContext */ - private[streaming] object StreamingContextState extends Enumeration { - type CheckpointState = Value - val Initialized, Started, Stopped = Value - } - - import StreamingContextState._ - private[streaming] var state = Initialized + private var state: StreamingContextState = INITIALIZED private val startSite = new AtomicReference[CallSite](null) @@ -516,6 +510,21 @@ class StreamingContext private[streaming] ( ) } + /** + * :: DeveloperApi :: + * + * Return the current state of the context. The context can be in three possible states - + * - StreamingContextState.INTIALIZED - The context has been created, but not been started yet. + * Input DStreams, transformations and output operations can be created on the context. + * - StreamingContextState.ACTIVE - The context has been started, and been not stopped. + * Input DStreams, transformations and output operations cannot be created on the context. + * - StreamingContextState.STOPPED - The context has been stopped and cannot be used any more. + */ + @DeveloperApi + def getState(): StreamingContextState = synchronized { + state + } + /** * Start the execution of the streams. * @@ -523,11 +532,13 @@ class StreamingContext private[streaming] ( */ 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") + state match { + case INITIALIZED => + // good to start + case ACTIVE => + throw new SparkException("StreamingContext has already been started") + case STOPPED => + throw new SparkException("StreamingContext has already been stopped") } validate() startSite.set(DStream.getCreationSite()) @@ -536,7 +547,7 @@ class StreamingContext private[streaming] ( assertNoOtherContextIsActive() scheduler.start() uiTab.foreach(_.attach()) - state = Started + state = StreamingContextState.ACTIVE setActiveContext(this) } } @@ -598,22 +609,26 @@ class StreamingContext private[streaming] ( * received data to be completed */ def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = synchronized { - state match { - case Initialized => logWarning("StreamingContext has not been started yet") - case Stopped => logWarning("StreamingContext has already been stopped") - case Started => - scheduler.stop(stopGracefully) - logInfo("StreamingContext stopped successfully") - waiter.notifyStop() + try { + state match { + case INITIALIZED => + logWarning("StreamingContext has not been started yet") + case STOPPED => + logWarning("StreamingContext has already been stopped") + case ACTIVE => + scheduler.stop(stopGracefully) + uiTab.foreach(_.detach()) + StreamingContext.setActiveContext(null) + waiter.notifyStop() + logInfo("StreamingContext stopped successfully") + } + // Even if we have already stopped, we still need to attempt to stop the SparkContext because + // a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true). + if (stopSparkContext) sc.stop() + } finally { + // The state should always be Stopped after calling `stop()`, even if we haven't started yet + state = STOPPED } - // Even if the streaming context has not been started, we still need to stop the SparkContext. - // Even if we have already stopped, we still need to attempt to stop the SparkContext because - // a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true). - if (stopSparkContext) sc.stop() - uiTab.foreach(_.detach()) - // The state should always be Stopped after calling `stop()`, even if we haven't started yet: - state = Stopped - StreamingContext.setActiveContext(null) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContextState.java b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContextState.java new file mode 100644 index 0000000000..d7b639383e --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContextState.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming; + +import org.apache.spark.annotation.DeveloperApi; + +/** + * :: DeveloperApi :: + * + * Represents the state of a StreamingContext. + */ +@DeveloperApi +public enum StreamingContextState { + /** + * The context has been created, but not been started yet. + * Input DStreams, transformations and output operations can be created on the context. + */ + INITIALIZED, + + /** + * The context has been started, and been not stopped. + * Input DStreams, transformations and output operations cannot be created on the context. + */ + ACTIVE, + + /** + * The context has been stopped and cannot be used any more. + */ + STOPPED +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 572d7d8e87..d8fbed2c50 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -578,6 +578,28 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { ssc.addStreamingListener(streamingListener) } + /** + * :: DeveloperApi :: + * + * Return the current state of the context. The context can be in three possible states - + *