From 5d7b591cfe14177f083814fe3e81745c5d279810 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 19:34:32 -0800 Subject: Pass a code JAR to SparkContext in our examples. Fixes SPARK-594. --- .../spark/streaming/api/java/JavaStreamingContext.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) (limited to 'streaming') 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 755407aecc..3d149a742c 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -43,6 +43,23 @@ class JavaStreamingContext(val ssc: StreamingContext) { def this(master: String, appName: String, batchDuration: Duration) = this(new StreamingContext(master, appName, batchDuration, null, Nil, Map())) + /** + * Creates a StreamingContext. + * @param master Name of the Spark Master + * @param appName Name to be used when registering with the scheduler + * @param batchDuration The time interval at which streaming data will be divided into batches + * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param jarFile JAR file containing job code, to ship to cluster. This can be a path on the local + * file system or an HDFS, HTTP, HTTPS, or FTP URL. + */ + def this( + master: String, + appName: String, + batchDuration: Duration, + sparkHome: String, + jarFile: String) = + this(new StreamingContext(master, appName, batchDuration, sparkHome, Seq(jarFile), Map())) + /** * Creates a StreamingContext. * @param master Name of the Spark Master -- cgit v1.2.3