diff options
author | Jacker Hu <gt.hu.chang@gmail.com> | 2015-10-10 11:36:18 +0100 |
---|---|---|
committer | Sean Owen <sowen@cloudera.com> | 2015-10-10 11:36:18 +0100 |
commit | a16396df76cc27099011bfb96b28cbdd7f964ca8 (patch) | |
tree | fa24e961cd38f01993cee88208bf99acaf008444 /streaming | |
parent | 864de3bf4041c829e95d278b9569e91448bab0cc (diff) | |
download | spark-a16396df76cc27099011bfb96b28cbdd7f964ca8.tar.gz spark-a16396df76cc27099011bfb96b28cbdd7f964ca8.tar.bz2 spark-a16396df76cc27099011bfb96b28cbdd7f964ca8.zip |
[SPARK-10772] [STREAMING] [SCALA] NullPointerException when transform function in DStream returns NULL
Currently, the ```TransformedDStream``` will using ```Some(transformFunc(parentRDDs, validTime))``` as compute return value, when the ```transformFunc``` somehow returns null as return value, the followed operator will have NullPointerExeception.
This fix uses the ```Option()``` instead of ```Some()``` to deal with the possible null value. When ```transformFunc``` returns ```null```, the option will transform null to ```None```, the downstream can handle ```None``` correctly.
NOTE (2015-09-25): The latest fix will check the return value of transform function, if it is ```NULL```, a spark exception will be thrown out
Author: Jacker Hu <gt.hu.chang@gmail.com>
Author: jhu-chang <gt.hu.chang@gmail.com>
Closes #8881 from jhu-chang/Fix_Transform.
Diffstat (limited to 'streaming')
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala | 12 | ||||
-rw-r--r-- | streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala | 13 |
2 files changed, 23 insertions, 2 deletions
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 5d46ca0715..ab01f47d5c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -17,9 +17,11 @@ package org.apache.spark.streaming.dstream +import scala.reflect.ClassTag + +import org.apache.spark.SparkException import org.apache.spark.rdd.{PairRDDFunctions, RDD} import org.apache.spark.streaming.{Duration, Time} -import scala.reflect.ClassTag private[streaming] class TransformedDStream[U: ClassTag] ( @@ -38,6 +40,12 @@ class TransformedDStream[U: ClassTag] ( override def compute(validTime: Time): Option[RDD[U]] = { val parentRDDs = parents.map(_.getOrCompute(validTime).orNull).toSeq - Some(transformFunc(parentRDDs, validTime)) + val transformedRDD = transformFunc(parentRDDs, validTime) + if (transformedRDD == null) { + throw new SparkException("Transform function must not return null. " + + "Return SparkContext.emptyRDD() instead to represent no element " + + "as the result of transformation.") + } + Some(transformedRDD) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 255376807c..9988f410f0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -211,6 +211,19 @@ class BasicOperationsSuite extends TestSuiteBase { ) } + test("transform with NULL") { + val input = Seq(1 to 4) + intercept[SparkException] { + testOperation( + input, + (r: DStream[Int]) => r.transform(rdd => null.asInstanceOf[RDD[Int]]), + Seq(Seq()), + 1, + false + ) + } + } + test("transformWith") { val inputData1 = Seq( Seq("a", "b"), Seq("a", ""), Seq(""), Seq() ) val inputData2 = Seq( Seq("a", "b"), Seq("b", ""), Seq(), Seq("") ) |