aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--python/pyspark/streaming/context.py2
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala10
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala2
3 files changed, 7 insertions, 7 deletions
diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py
index b06ab65037..2c73083c9f 100644
--- a/python/pyspark/streaming/context.py
+++ b/python/pyspark/streaming/context.py
@@ -189,7 +189,7 @@ class StreamingContext(object):
if timeout is None:
self._jssc.awaitTermination()
else:
- self._jssc.awaitTermination(int(timeout * 1000))
+ self._jssc.awaitTerminationOrTimeout(int(timeout * 1000))
def awaitTerminationOrTimeout(self, timeout):
"""
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
index 2aa5e0876b..6a7cd97aa3 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
@@ -190,7 +190,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
logInfo("Count = " + count + ", Running count = " + runningCount)
}
ssc.start()
- ssc.awaitTermination(500)
+ ssc.awaitTerminationOrTimeout(500)
ssc.stop(stopSparkContext = false, stopGracefully = true)
logInfo("Running count = " + runningCount)
logInfo("TestReceiver.counter = " + TestReceiver.counter.get())
@@ -223,7 +223,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
logInfo("Count = " + count + ", Running count = " + runningCount)
}
ssc.start()
- ssc.awaitTermination(500)
+ ssc.awaitTerminationOrTimeout(500)
ssc.stop(stopSparkContext = false, stopGracefully = true)
logInfo("Running count = " + runningCount)
assert(runningCount > 0)
@@ -243,7 +243,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
// test whether awaitTermination() exits after give amount of time
failAfter(1000 millis) {
- ssc.awaitTermination(500)
+ ssc.awaitTerminationOrTimeout(500)
}
// test whether awaitTermination() does not exit if not time is given
@@ -288,7 +288,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
val exception = intercept[Exception] {
ssc.start()
- ssc.awaitTermination(5000)
+ ssc.awaitTerminationOrTimeout(5000)
}
assert(exception.getMessage.contains("map task"), "Expected exception not thrown")
}
@@ -299,7 +299,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
inputStream.transform { rdd => throw new TestException("error in transform"); rdd }.register()
val exception = intercept[TestException] {
ssc.start()
- ssc.awaitTermination(5000)
+ ssc.awaitTerminationOrTimeout(5000)
}
assert(exception.getMessage.contains("transform"), "Expected exception not thrown")
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index c2375ff65e..3565d621e8 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -348,7 +348,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
val startTime = System.currentTimeMillis()
while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput)
- ssc.awaitTermination(50)
+ ssc.awaitTerminationOrTimeout(50)
}
val timeTaken = System.currentTimeMillis() - startTime
logInfo("Output generated in " + timeTaken + " milliseconds")