You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by hv...@apache.org on 2016/08/26 15:29:43 UTC

spark git commit: [SPARK-16216][SQL][FOLLOWUP] Enable timestamp type tests for JSON and verify all unsupported types in CSV

Repository: spark
Updated Branches:
  refs/heads/master 341e0e778 -> 6063d5963


[SPARK-16216][SQL][FOLLOWUP] Enable timestamp type tests for JSON and verify all unsupported types in CSV

## What changes were proposed in this pull request?

This PR enables the tests for `TimestampType` for JSON and unifies the logics for verifying schema when writing in CSV.

In more details, this PR,

- Enables the tests for `TimestampType` for JSON and

  This was disabled due to an issue in `DatatypeConverter.parseDateTime` which parses dates incorrectly, for example as below:

  ```scala
   val d = javax.xml.bind.DatatypeConverter.parseDateTime("0900-01-01T00:00:00.000").getTime
  println(d.toString)
  ```
  ```
  Fri Dec 28 00:00:00 KST 899
  ```

  However, since we use `FastDateFormat`, it seems we are safe now.

  ```scala
  val d = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSS").parse("0900-01-01T00:00:00.000")
  println(d)
  ```
  ```
  Tue Jan 01 00:00:00 PST 900
  ```

- Verifies all unsupported types in CSV

  There is a separate logics to verify the schemas in `CSVFileFormat`. This is actually not quite correct enough because we don't support `NullType` and `CalanderIntervalType` as well `StructType`, `ArrayType`, `MapType`. So, this PR adds both types.

## How was this patch tested?

Tests in `JsonHadoopFsRelation` and `CSVSuite`

Author: hyukjinkwon <gu...@gmail.com>

Closes #14829 from HyukjinKwon/SPARK-16216-followup.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6063d596
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6063d596
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6063d596

Branch: refs/heads/master
Commit: 6063d5963fcf01768570c1a9b542be6175a3bcbc
Parents: 341e0e7
Author: hyukjinkwon <gu...@gmail.com>
Authored: Fri Aug 26 17:29:37 2016 +0200
Committer: Herman van Hovell <hv...@databricks.com>
Committed: Fri Aug 26 17:29:37 2016 +0200

----------------------------------------------------------------------
 .../execution/datasources/csv/CSVFileFormat.scala  | 17 +++++++++++------
 .../execution/datasources/csv/CSVInferSchema.scala |  1 +
 .../sql/execution/datasources/csv/CSVSuite.scala   | 16 ++++++++++++++--
 .../sql/sources/JsonHadoopFsRelationSuite.scala    |  4 ----
 4 files changed, 26 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/6063d596/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
index 1bf5788..9a118fe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
@@ -186,13 +186,18 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister {
   }
 
   private def verifySchema(schema: StructType): Unit = {
-    schema.foreach { field =>
-      field.dataType match {
-        case _: ArrayType | _: MapType | _: StructType =>
-          throw new UnsupportedOperationException(
-            s"CSV data source does not support ${field.dataType.simpleString} data type.")
+    def verifyType(dataType: DataType): Unit = dataType match {
+        case ByteType | ShortType | IntegerType | LongType | FloatType |
+             DoubleType | BooleanType | _: DecimalType | TimestampType |
+             DateType | StringType =>
+
+        case udt: UserDefinedType[_] => verifyType(udt.sqlType)
+
         case _ =>
-      }
+          throw new UnsupportedOperationException(
+            s"CSV data source does not support ${dataType.simpleString} data type.")
     }
+
+    schema.foreach(field => verifyType(field.dataType))
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/6063d596/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
index f1b4c11..1ca6eff 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
@@ -290,6 +290,7 @@ private[csv] object CSVTypeCast {
             DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime)
           }
       case _: StringType => UTF8String.fromString(datum)
+      case udt: UserDefinedType[_] => castTo(datum, udt.sqlType, nullable, options)
       case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}")
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/6063d596/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index 2befad6..1930862 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -27,8 +27,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType
 import org.apache.hadoop.io.compress.GzipCodec
 
 import org.apache.spark.SparkException
-import org.apache.spark.sql.{DataFrame, QueryTest, Row}
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.{DataFrame, QueryTest, Row, UDT}
 import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
 import org.apache.spark.sql.types._
 
@@ -681,6 +680,19 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
         Seq((1, Array("Tesla", "Chevy", "Ford"))).toDF("id", "brands").write.csv(csvDir)
       }.getMessage
       assert(msg.contains("CSV data source does not support array<string> data type"))
+
+      msg = intercept[UnsupportedOperationException] {
+        Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25)))).toDF("id", "vectors")
+          .write.csv(csvDir)
+      }.getMessage
+      assert(msg.contains("CSV data source does not support array<double> data type"))
+
+      msg = intercept[SparkException] {
+        val schema = StructType(StructField("a", new UDT.MyDenseVectorUDT(), true) :: Nil)
+        spark.range(1).write.csv(csvDir)
+        spark.read.schema(schema).csv(csvDir).collect()
+      }.getCause.getMessage
+      assert(msg.contains("Unsupported type: array"))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/6063d596/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala
index 52486b1..d79edee 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala
@@ -32,10 +32,6 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest {
   override protected def supportsDataType(dataType: DataType): Boolean = dataType match {
     case _: NullType => false
     case _: BinaryType => false
-    // `TimestampType` is disabled because `DatatypeConverter.parseDateTime()`
-    // in `DateTimeUtils` parses the formatted string wrongly when the date is
-    // too early. (e.g. "1600-07-13T08:36:32.847").
-    case _: TimestampType => false
     case _: CalendarIntervalType => false
     case _ => true
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org