Streaming constructor which takes JavaSparkContext
It's sometimes helpful to directly pass a JavaSparkContext, and take advantage of the various constructors available for that.
This commit is contained in:
parent
f6ec547ea7
commit
7eea64aa4c
|
@ -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
|
||||
|
|
Loading…
Reference in a new issue