From 59065bc8fbfc8669b94556e404b1f395046c6573 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 8 Sep 2014 16:00:47 -0700 Subject: [PATCH 1/3] Remove explicit conversion to avoid NPE --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 5acb45c155ba5..92933b67b24d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -460,7 +460,6 @@ class SQLContext(@transient val sparkContext: SparkContext) rdd: RDD[Array[Any]], schema: StructType): SchemaRDD = { import scala.collection.JavaConversions._ - import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} def needsConversion(dataType: DataType): Boolean = dataType match { case ByteType => true @@ -482,8 +481,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case (null, _) => null case (c: java.util.List[_], ArrayType(elementType, _)) => - val converted = c.map { e => convert(e, elementType)} - JListWrapper(converted) + c.map { e => convert(e, elementType)}: Seq[Any] case (c, ArrayType(elementType, _)) if c.getClass.isArray => c.asInstanceOf[Array[_]].map(e => convert(e, elementType)): Seq[Any] From 646976b1912b9f7c299b739db4cf49131c8b28eb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 9 Sep 2014 18:58:19 -0700 Subject: [PATCH 2/3] Fix JSON RDD Conversion too --- .../src/main/scala/org/apache/spark/sql/json/JsonRDD.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 1c0b03c684f10..2b064a93653b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.json import scala.collection.Map -import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} +import scala.collection.JavaConversions._ +import scala.collection.convert.Wrappers.JMapWrapper import scala.math.BigDecimal import com.fasterxml.jackson.databind.ObjectMapper @@ -253,7 +254,7 @@ private[sql] object JsonRDD extends Logging { // This issue is documented at https://issues.scala-lang.org/browse/SI-7005 JMapWrapper(map).mapValues(scalafy).map(identity) case list: java.util.List[_] => - JListWrapper(list).map(scalafy) + (list: Seq[_]).map(scalafy) case atom => atom } From 9634f1158a0f6a1e9afd81232358f9454d1db4ad Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 10 Sep 2014 20:59:04 -0700 Subject: [PATCH 3/3] Rollback JSON RDD changes --- .../src/main/scala/org/apache/spark/sql/json/JsonRDD.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 361768d5f99c7..70062eae3b7ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.json import scala.collection.Map -import scala.collection.JavaConversions._ -import scala.collection.convert.Wrappers.JMapWrapper +import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal import com.fasterxml.jackson.databind.ObjectMapper @@ -247,7 +246,7 @@ private[sql] object JsonRDD extends Logging { // This issue is documented at https://issues.scala-lang.org/browse/SI-7005 JMapWrapper(map).mapValues(scalafy).map(identity) case list: java.util.List[_] => - (list: Seq[_]).map(scalafy) + JListWrapper(list).map(scalafy) case atom => atom }