From edbac3f80fdbcb5558f4d978299cb517003f3e6c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 16 Jul 2014 09:36:21 -0700 Subject: [PATCH] Removed some compiler type erasure warnings. --- .../scala/org/apache/spark/sql/SchemaRDD.scala | 18 +++++++++--------- .../org/apache/spark/sql/json/JsonRDD.scala | 18 +++++++++--------- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 0c95b668545f..993d085c7508 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -380,32 +380,32 @@ class SchemaRDD( val fields = structType.fields.map(field => (field.name, field.dataType)) val map: JMap[String, Any] = new java.util.HashMap row.zip(fields).foreach { - case (obj, (name, dataType)) => + case (obj, (attrName, dataType)) => dataType match { - case struct: StructType => map.put(name, rowToMap(obj.asInstanceOf[Row], struct)) + case struct: StructType => map.put(attrName, rowToMap(obj.asInstanceOf[Row], struct)) case array @ ArrayType(struct: StructType) => val arrayValues = obj match { case seq: Seq[Any] => seq.map(element => rowToMap(element.asInstanceOf[Row], struct)).asJava - case list: JList[Any] => + case list: JList[_] => list.map(element => rowToMap(element.asInstanceOf[Row], struct)) - case set: JSet[Any] => + case set: JSet[_] => set.map(element => rowToMap(element.asInstanceOf[Row], struct)) - case array if array != null && array.getClass.isArray => - array.asInstanceOf[Array[Any]].map { + case arr if arr != null && arr.getClass.isArray => + arr.asInstanceOf[Array[Any]].map { element => rowToMap(element.asInstanceOf[Row], struct) } case other => other } - map.put(name, arrayValues) + map.put(attrName, arrayValues) case array: ArrayType => { val arrayValues = obj match { case seq: Seq[Any] => seq.asJava case other => other } - map.put(name, arrayValues) + map.put(attrName, arrayValues) } - case other => map.put(name, obj) + case other => map.put(attrName, obj) } } 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 f6cbca96483e..df80dfb98b93 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 @@ -204,14 +204,14 @@ private[sql] object JsonRDD extends Logging { case (key, value) => (s"`$key`", value) }.toSet keyValuePairs.flatMap { - case (key: String, struct: Map[String, Any]) => { - // The value associted with the key is an JSON object. - allKeysWithValueTypes(struct).map { + case (key: String, struct: Map[_, _]) => { + // The value associated with the key is an JSON object. + allKeysWithValueTypes(struct.asInstanceOf[Map[String, Any]]).map { case (k, dataType) => (s"$key.$k", dataType) } ++ Set((key, StructType(Nil))) } - case (key: String, array: List[Any]) => { - // The value associted with the key is an array. + case (key: String, array: List[_]) => { + // The value associated with the key is an array. typeOfArray(array) match { case ArrayType(StructType(Nil)) => { // The elements of this arrays are structs. @@ -235,12 +235,12 @@ private[sql] object JsonRDD extends Logging { * the parsing very slow. */ private def scalafy(obj: Any): Any = obj match { - case map: java.util.Map[String, Object] => + case map: java.util.Map[_, _] => // .map(identity) is used as a workaround of non-serializable Map // generated by .mapValues. // This issue is documented at https://issues.scala-lang.org/browse/SI-7005 map.toMap.mapValues(scalafy).map(identity) - case list: java.util.List[Object] => + case list: java.util.List[_] => list.toList.map(scalafy) case atom => atom } @@ -320,8 +320,8 @@ private[sql] object JsonRDD extends Logging { private def toString(value: Any): String = { value match { - case value: Map[String, Any] => toJsonObjectString(value) - case value: Seq[Any] => toJsonArrayString(value) + case value: Map[_, _] => toJsonObjectString(value.asInstanceOf[Map[String, Any]]) + case value: Seq[_] => toJsonArrayString(value) case value => Option(value).map(_.toString).orNull } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a623d29b5397..d57e99db1858 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -390,7 +390,7 @@ class HiveQuerySuite extends HiveComparisonTest { hql("CREATE TABLE m(value MAP)") hql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") hql("SELECT * FROM m").collect().zip(hql("SELECT * FROM src LIMIT 10").collect()).map { - case (Row(map: Map[Int, String]), Row(key: Int, value: String)) => + case (Row(map: Map[_, _]), Row(key: Int, value: String)) => assert(map.size === 1) assert(map.head === (key, value)) }