Skip to content

Commit 80ee0e6

Browse files
committed
STARTED --> ACTIVE
1 parent 3da6547 commit 80ee0e6

File tree

5 files changed

+11
-11
lines changed

5 files changed

+11
-11
lines changed

streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -516,7 +516,7 @@ class StreamingContext private[streaming] (
516516
* Return the current state of the context. The context can be in three possible states -
517517
* - StreamingContextState.INTIALIZED - The context has been created, but not been started yet.
518518
* Input DStreams, transformations and output operations can be created on the context.
519-
* - StreamingContextState.STARTED - The context has been started, and been not stopped.
519+
* - StreamingContextState.ACTIVE - The context has been started, and been not stopped.
520520
* Input DStreams, transformations and output operations cannot be created on the context.
521521
* - StreamingContextState.STOPPED - The context has been stopped and cannot be used any more.
522522
*/
@@ -535,7 +535,7 @@ class StreamingContext private[streaming] (
535535
state match {
536536
case INITIALIZED =>
537537
// good to start
538-
case STARTED =>
538+
case ACTIVE =>
539539
throw new SparkException("StreamingContext has already been started")
540540
case STOPPED =>
541541
throw new SparkException("StreamingContext has already been stopped")
@@ -547,7 +547,7 @@ class StreamingContext private[streaming] (
547547
assertNoOtherContextIsActive()
548548
scheduler.start()
549549
uiTab.foreach(_.attach())
550-
state = StreamingContextState.STARTED
550+
state = StreamingContextState.ACTIVE
551551
setActiveContext(this)
552552
}
553553
}
@@ -615,7 +615,7 @@ class StreamingContext private[streaming] (
615615
logWarning("StreamingContext has not been started yet")
616616
case STOPPED =>
617617
logWarning("StreamingContext has already been stopped")
618-
case STARTED =>
618+
case ACTIVE =>
619619
scheduler.stop(stopGracefully)
620620
uiTab.foreach(_.detach())
621621
StreamingContext.setActiveContext(null)

streaming/src/main/scala/org/apache/spark/streaming/StreamingContextState.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ public enum StreamingContextState {
3636
* The context has been started, and been not stopped.
3737
* Input DStreams, transformations and output operations cannot be created on the context.
3838
*/
39-
STARTED,
39+
ACTIVE,
4040

4141
/**
4242
* The context has been stopped and cannot be used any more.

streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -588,7 +588,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable {
588588
* Input DStreams, transformations and output operations can be created on the context.
589589
* </li>
590590
* <li>
591-
* StreamingContextState.STARTED - The context has been started, and been not stopped.
591+
* StreamingContextState.ACTIVE - The context has been started, and been not stopped.
592592
* Input DStreams, transformations and output operations cannot be created on the context.
593593
* </li>
594594
* <li>

streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -79,7 +79,7 @@ public void testContextState() {
7979
JavaTestUtils.attachTestOutputStream(stream);
8080
Assert.assertTrue(ssc.getState() == StreamingContextState.INITIALIZED);
8181
ssc.start();
82-
Assert.assertTrue(ssc.getState() == StreamingContextState.STARTED);
82+
Assert.assertTrue(ssc.getState() == StreamingContextState.ACTIVE);
8383
ssc.stop();
8484
Assert.assertTrue(ssc.getState() == StreamingContextState.STOPPED);
8585
}

streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -112,7 +112,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
112112
test("state matching") {
113113
import StreamingContextState._
114114
assert(INITIALIZED === INITIALIZED)
115-
assert(INITIALIZED != STARTED)
115+
assert(INITIALIZED != ACTIVE)
116116
}
117117

118118
test("start and stop state check") {
@@ -121,7 +121,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
121121

122122
assert(ssc.getState() === StreamingContextState.INITIALIZED)
123123
ssc.start()
124-
assert(ssc.getState() === StreamingContextState.STARTED)
124+
assert(ssc.getState() === StreamingContextState.ACTIVE)
125125
ssc.stop()
126126
assert(ssc.getState() === StreamingContextState.STOPPED)
127127

@@ -135,11 +135,11 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
135135
ssc = new StreamingContext(master, appName, batchDuration)
136136
addInputStream(ssc).register()
137137
ssc.start()
138-
assert(ssc.getState() === StreamingContextState.STARTED)
138+
assert(ssc.getState() === StreamingContextState.ACTIVE)
139139
intercept[SparkException] {
140140
ssc.start()
141141
}
142-
assert(ssc.getState() === StreamingContextState.STARTED)
142+
assert(ssc.getState() === StreamingContextState.ACTIVE)
143143
}
144144

145145
test("stop multiple times") {

0 commit comments

Comments
 (0)