diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
index f82e6a37ccb95284fe367e123a8934e46b3bd851..e7f446a49b581ba249ae84c7436dc7b417a9f1b9 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
@@ -33,6 +33,14 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   def this(master: String, frameworkName: String, batchDuration: Duration) =
     this(new StreamingContext(master, frameworkName, batchDuration))
 
+  /**
+   * Creates a StreamingContext.
+   * @param sparkContext The underlying JavaSparkContext to use
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   */
+  def this(sparkContext: JavaSparkContext, batchDuration: Duration) =
+    this(new StreamingContext(sparkContext.sc, batchDuration))
+
   /**
    * Re-creates a StreamingContext from a checkpoint file.
    * @param path Path either to the directory that was specified as the checkpoint directory, or