@@ -217,53 +217,52 @@ abstract class DStream[T: ClassTag] (
217217 case StreamingContextState .INITIALIZED =>
218218 // good to go
219219 case StreamingContextState .ACTIVE =>
220- throw new SparkException (
220+ throw new IllegalStateException (
221221 " Adding new inputs, transformations, and output operations after " +
222222 " starting a context is not supported" )
223223 case StreamingContextState .STOPPED =>
224- throw new SparkException (
224+ throw new IllegalStateException (
225225 " Adding new inputs, transformations, and output operations after " +
226226 " stopping a context is not supported" )
227227 }
228228 }
229229
230230 private [streaming] def validateAtStart () {
231- assert (rememberDuration != null , " Remember duration is set to null" )
231+ require (rememberDuration != null , " Remember duration is set to null" )
232232
233- assert (
233+ require (
234234 ! mustCheckpoint || checkpointDuration != null ,
235235 " The checkpoint interval for " + this .getClass.getSimpleName + " has not been set." +
236236 " Please use DStream.checkpoint() to set the interval."
237237 )
238238
239- assert (
239+ require (
240240 checkpointDuration == null || context.sparkContext.checkpointDir.isDefined,
241- " The checkpoint directory has not been set. Please use StreamingContext.checkpoint()" +
242- " or SparkContext.checkpoint() to set the checkpoint directory."
241+ " The checkpoint directory has not been set. Please set it by StreamingContext.checkpoint()."
243242 )
244243
245- assert (
244+ require (
246245 checkpointDuration == null || checkpointDuration >= slideDuration,
247246 " The checkpoint interval for " + this .getClass.getSimpleName + " has been set to " +
248247 checkpointDuration + " which is lower than its slide time (" + slideDuration + " ). " +
249248 " Please set it to at least " + slideDuration + " ."
250249 )
251250
252- assert (
251+ require (
253252 checkpointDuration == null || checkpointDuration.isMultipleOf(slideDuration),
254253 " The checkpoint interval for " + this .getClass.getSimpleName + " has been set to " +
255254 checkpointDuration + " which not a multiple of its slide time (" + slideDuration + " ). " +
256- " Please set it to a multiple " + slideDuration + " ."
255+ " Please set it to a multiple of " + slideDuration + " ."
257256 )
258257
259- assert (
258+ require (
260259 checkpointDuration == null || storageLevel != StorageLevel .NONE ,
261260 " " + this .getClass.getSimpleName + " has been marked for checkpointing but the storage " +
262261 " level has not been set to enable persisting. Please use DStream.persist() to set the " +
263262 " storage level to use memory for better checkpointing performance."
264263 )
265264
266- assert (
265+ require (
267266 checkpointDuration == null || rememberDuration > checkpointDuration,
268267 " The remember duration for " + this .getClass.getSimpleName + " has been set to " +
269268 rememberDuration + " which is not more than the checkpoint interval (" +
@@ -272,7 +271,7 @@ abstract class DStream[T: ClassTag] (
272271
273272 val metadataCleanerDelay = MetadataCleaner .getDelaySeconds(ssc.conf)
274273 logInfo(" metadataCleanupDelay = " + metadataCleanerDelay)
275- assert (
274+ require (
276275 metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000 ,
277276 " It seems you are doing some DStream window operation or setting a checkpoint interval " +
278277 " which requires " + this .getClass.getSimpleName + " to remember generated RDDs for more " +
@@ -633,8 +632,8 @@ abstract class DStream[T: ClassTag] (
633632 * 'this' DStream will be registered as an output stream and therefore materialized.
634633 */
635634 def foreachRDD (foreachFunc : (RDD [T ], Time ) => Unit ): Unit = ssc.withScope {
636- // because the DStream is reachable from the outer object here, and because
637- // DStreams can't be serialized with closures, we can't proactively check
635+ // because the DStream is reachable from the outer object here, and because
636+ // DStreams can't be serialized with closures, we can't proactively check
638637 // it for serializability and so we pass the optional false to SparkContext.clean
639638 new ForEachDStream (this , context.sparkContext.clean(foreachFunc, false )).register()
640639 }
@@ -644,8 +643,8 @@ abstract class DStream[T: ClassTag] (
644643 * on each RDD of 'this' DStream.
645644 */
646645 def transform [U : ClassTag ](transformFunc : RDD [T ] => RDD [U ]): DStream [U ] = ssc.withScope {
647- // because the DStream is reachable from the outer object here, and because
648- // DStreams can't be serialized with closures, we can't proactively check
646+ // because the DStream is reachable from the outer object here, and because
647+ // DStreams can't be serialized with closures, we can't proactively check
649648 // it for serializability and so we pass the optional false to SparkContext.clean
650649 val cleanedF = context.sparkContext.clean(transformFunc, false )
651650 transform((r : RDD [T ], t : Time ) => cleanedF(r))
@@ -656,8 +655,8 @@ abstract class DStream[T: ClassTag] (
656655 * on each RDD of 'this' DStream.
657656 */
658657 def transform [U : ClassTag ](transformFunc : (RDD [T ], Time ) => RDD [U ]): DStream [U ] = ssc.withScope {
659- // because the DStream is reachable from the outer object here, and because
660- // DStreams can't be serialized with closures, we can't proactively check
658+ // because the DStream is reachable from the outer object here, and because
659+ // DStreams can't be serialized with closures, we can't proactively check
661660 // it for serializability and so we pass the optional false to SparkContext.clean
662661 val cleanedF = context.sparkContext.clean(transformFunc, false )
663662 val realTransformFunc = (rdds : Seq [RDD [_]], time : Time ) => {
@@ -674,8 +673,8 @@ abstract class DStream[T: ClassTag] (
674673 def transformWith [U : ClassTag , V : ClassTag ](
675674 other : DStream [U ], transformFunc : (RDD [T ], RDD [U ]) => RDD [V ]
676675 ): DStream [V ] = ssc.withScope {
677- // because the DStream is reachable from the outer object here, and because
678- // DStreams can't be serialized with closures, we can't proactively check
676+ // because the DStream is reachable from the outer object here, and because
677+ // DStreams can't be serialized with closures, we can't proactively check
679678 // it for serializability and so we pass the optional false to SparkContext.clean
680679 val cleanedF = ssc.sparkContext.clean(transformFunc, false )
681680 transformWith(other, (rdd1 : RDD [T ], rdd2 : RDD [U ], time : Time ) => cleanedF(rdd1, rdd2))
@@ -688,8 +687,8 @@ abstract class DStream[T: ClassTag] (
688687 def transformWith [U : ClassTag , V : ClassTag ](
689688 other : DStream [U ], transformFunc : (RDD [T ], RDD [U ], Time ) => RDD [V ]
690689 ): DStream [V ] = ssc.withScope {
691- // because the DStream is reachable from the outer object here, and because
692- // DStreams can't be serialized with closures, we can't proactively check
690+ // because the DStream is reachable from the outer object here, and because
691+ // DStreams can't be serialized with closures, we can't proactively check
693692 // it for serializability and so we pass the optional false to SparkContext.clean
694693 val cleanedF = ssc.sparkContext.clean(transformFunc, false )
695694 val realTransformFunc = (rdds : Seq [RDD [_]], time : Time ) => {
0 commit comments