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 b528ebbc1992af8dfe4b2f7ea2cd53f860c6fc5e..755407aecc53af0f197f6fa5d29d734fca4a0729 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
@@ -41,7 +41,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @param batchDuration The time interval at which streaming data will be divided into batches
    */
   def this(master: String, appName: String, batchDuration: Duration) =
-    this(new StreamingContext(master, appName, batchDuration))
+    this(new StreamingContext(master, appName, batchDuration, null, Nil, Map()))
 
   /**
    * Creates a StreamingContext.
@@ -58,7 +58,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
       batchDuration: Duration,
       sparkHome: String,
       jars: Array[String]) =
-    this(new StreamingContext(master, appName, batchDuration, sparkHome, jars))
+    this(new StreamingContext(master, appName, batchDuration, sparkHome, jars, Map()))
 
   /**
    * Creates a StreamingContext.
diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
index bdd9f4d7535ea0fb055d29ccc5eccf88dd83b958..f673e5be15485b839d5f894912d3290eecc21637 100644
--- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
@@ -159,7 +159,7 @@ object MasterFailureTest extends Logging {
 
     // Setup the streaming computation with the given operation
     System.clearProperty("spark.driver.port")
-    var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration)
+    var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map())
     ssc.checkpoint(checkpointDir.toString)
     val inputStream = ssc.textFileStream(testDir.toString)
     val operatedStream = operation(inputStream)