diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py
index 18aaae93b05f2e8aa0313c210c8bc102377d932d..b06ab650370bd823bc1a9a394e21cf25f8bf2442 100644
--- a/python/pyspark/streaming/context.py
+++ b/python/pyspark/streaming/context.py
@@ -191,6 +191,15 @@ class StreamingContext(object):
         else:
             self._jssc.awaitTermination(int(timeout * 1000))
 
+    def awaitTerminationOrTimeout(self, timeout):
+        """
+        Wait for the execution to stop. Return `true` if it's stopped; or
+        throw the reported error during the execution; or `false` if the
+        waiting time elapsed before returning from the method.
+        @param timeout: time to wait in seconds
+        """
+        self._jssc.awaitTerminationOrTimeout(int(timeout * 1000))
+
     def stop(self, stopSparkContext=True, stopGraceFully=False):
         """
         Stop the execution of the streams, with option of ensuring all
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 ddc435cf1a2e6748366d5606f412a8d55198d038..ba3f23434f24c3528e9e11606ccb8cd6aa2a906b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -526,10 +526,23 @@ class StreamingContext private[streaming] (
    * will be thrown in this thread.
    * @param timeout time to wait in milliseconds
    */
+  @deprecated("Use awaitTerminationOrTimeout(Long) instead", "1.3.0")
   def awaitTermination(timeout: Long) {
     waiter.waitForStopOrError(timeout)
   }
 
+  /**
+   * Wait for the execution to stop. Any exceptions that occurs during the execution
+   * will be thrown in this thread.
+   *
+   * @param timeout time to wait in milliseconds
+   * @return `true` if it's stopped; or throw the reported error during the execution; or `false`
+   *         if the waiting time elapsed before returning from the method.
+   */
+  def awaitTerminationOrTimeout(timeout: Long): Boolean = {
+    waiter.waitForStopOrError(timeout)
+  }
+
   /**
    * Stop the execution of the streams immediately (does not wait for all received data
    * to be processed).
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 0f7ae7a1c7de86b2692983c1221b0ed5c14bd25c..e3db01c1e12c6da5ed1eb293ce456481db613436 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
@@ -597,10 +597,23 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable {
    * will be thrown in this thread.
    * @param timeout time to wait in milliseconds
    */
+  @deprecated("Use awaitTerminationOrTimeout(Long) instead", "1.3.0")
   def awaitTermination(timeout: Long): Unit = {
     ssc.awaitTermination(timeout)
   }
 
+  /**
+   * Wait for the execution to stop. Any exceptions that occurs during the execution
+   * will be thrown in this thread.
+   *
+   * @param timeout time to wait in milliseconds
+   * @return `true` if it's stopped; or throw the reported error during the execution; or `false`
+   *         if the waiting time elapsed before returning from the method.
+   */
+  def awaitTerminationOrTimeout(timeout: Long): Boolean = {
+    ssc.awaitTerminationOrTimeout(timeout)
+  }
+
   /**
    * Stop the execution of the streams. Will stop the associated JavaSparkContext as well.
    */
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 0b5af25e0f7cceeb368d03af227831d409eca7d8..2aa5e0876b6e022d68cb30e0dbbdb3ffb0cfa845 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
@@ -304,6 +304,30 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
     assert(exception.getMessage.contains("transform"), "Expected exception not thrown")
   }
 
+  test("awaitTerminationOrTimeout") {
+    ssc = new StreamingContext(master, appName, batchDuration)
+    val inputStream = addInputStream(ssc)
+    inputStream.map(x => x).register()
+
+    ssc.start()
+
+    // test whether awaitTerminationOrTimeout() return false after give amount of time
+    failAfter(1000 millis) {
+      assert(ssc.awaitTerminationOrTimeout(500) === false)
+    }
+
+    // test whether awaitTerminationOrTimeout() return true if context is stopped
+    failAfter(10000 millis) { // 10 seconds because spark takes a long time to shutdown
+      new Thread() {
+        override def run() {
+          Thread.sleep(500)
+          ssc.stop()
+        }
+      }.start()
+      assert(ssc.awaitTerminationOrTimeout(10000) === true)
+    }
+  }
+
   test("DStream and generated RDD creation sites") {
     testPackage.test()
   }