@@ -1450,103 +1450,105 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
14501450 }
14511451
14521452 test(" backward compatibility" ) {
1453- // This test we make sure our JSON support can read JSON data generated by previous version
1454- // of Spark generated through toJSON method and JSON data source.
1455- // The data is generated by the following program.
1456- // Here are a few notes:
1457- // - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13)
1458- // in the JSON object.
1459- // - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to
1460- // JSON objects generated by those Spark versions (col17).
1461- // - If the type is NullType, we do not write data out.
1462-
1463- // Create the schema.
1464- val struct =
1465- StructType (
1466- StructField (" f1" , FloatType , true ) ::
1467- StructField (" f2" , ArrayType (BooleanType ), true ) :: Nil )
1453+ withSQLConf(SQLConf .LEGACY_TIME_PARSER_ENABLED .key -> " true" ) {
1454+ // This test we make sure our JSON support can read JSON data generated by previous version
1455+ // of Spark generated through toJSON method and JSON data source.
1456+ // The data is generated by the following program.
1457+ // Here are a few notes:
1458+ // - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13)
1459+ // in the JSON object.
1460+ // - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to
1461+ // JSON objects generated by those Spark versions (col17).
1462+ // - If the type is NullType, we do not write data out.
1463+
1464+ // Create the schema.
1465+ val struct =
1466+ StructType (
1467+ StructField (" f1" , FloatType , true ) ::
1468+ StructField (" f2" , ArrayType (BooleanType ), true ) :: Nil )
14681469
1469- val dataTypes =
1470- Seq (
1471- StringType , BinaryType , NullType , BooleanType ,
1472- ByteType , ShortType , IntegerType , LongType ,
1473- FloatType , DoubleType , DecimalType (25 , 5 ), DecimalType (6 , 5 ),
1474- DateType , TimestampType ,
1475- ArrayType (IntegerType ), MapType (StringType , LongType ), struct,
1476- new UDT .MyDenseVectorUDT ())
1477- val fields = dataTypes.zipWithIndex.map { case (dataType, index) =>
1478- StructField (s " col $index" , dataType, nullable = true )
1479- }
1480- val schema = StructType (fields)
1470+ val dataTypes =
1471+ Seq (
1472+ StringType , BinaryType , NullType , BooleanType ,
1473+ ByteType , ShortType , IntegerType , LongType ,
1474+ FloatType , DoubleType , DecimalType (25 , 5 ), DecimalType (6 , 5 ),
1475+ DateType , TimestampType ,
1476+ ArrayType (IntegerType ), MapType (StringType , LongType ), struct,
1477+ new UDT .MyDenseVectorUDT ())
1478+ val fields = dataTypes.zipWithIndex.map { case (dataType, index) =>
1479+ StructField (s " col $index" , dataType, nullable = true )
1480+ }
1481+ val schema = StructType (fields)
14811482
1482- val constantValues =
1483- Seq (
1484- " a string in binary" .getBytes(StandardCharsets .UTF_8 ),
1485- null ,
1486- true ,
1487- 1 .toByte,
1488- 2 .toShort,
1489- 3 ,
1490- Long .MaxValue ,
1491- 0.25 .toFloat,
1492- 0.75 ,
1493- new java.math.BigDecimal (s " 1234.23456 " ),
1494- new java.math.BigDecimal (s " 1.23456 " ),
1495- java.sql.Date .valueOf(" 2015-01-01" ),
1496- java.sql.Timestamp .valueOf(" 2015-01-01 23:50:59.123" ),
1497- Seq (2 , 3 , 4 ),
1498- Map (" a string" -> 2000L ),
1499- Row (4.75 .toFloat, Seq (false , true )),
1500- new UDT .MyDenseVector (Array (0.25 , 2.25 , 4.25 )))
1501- val data =
1502- Row .fromSeq(Seq (" Spark " + spark.sparkContext.version) ++ constantValues) :: Nil
1483+ val constantValues =
1484+ Seq (
1485+ " a string in binary" .getBytes(StandardCharsets .UTF_8 ),
1486+ null ,
1487+ true ,
1488+ 1 .toByte,
1489+ 2 .toShort,
1490+ 3 ,
1491+ Long .MaxValue ,
1492+ 0.25 .toFloat,
1493+ 0.75 ,
1494+ new java.math.BigDecimal (s " 1234.23456 " ),
1495+ new java.math.BigDecimal (s " 1.23456 " ),
1496+ java.sql.Date .valueOf(" 2015-01-01" ),
1497+ java.sql.Timestamp .valueOf(" 2015-01-01 23:50:59.123" ),
1498+ Seq (2 , 3 , 4 ),
1499+ Map (" a string" -> 2000L ),
1500+ Row (4.75 .toFloat, Seq (false , true )),
1501+ new UDT .MyDenseVector (Array (0.25 , 2.25 , 4.25 )))
1502+ val data =
1503+ Row .fromSeq(Seq (" Spark " + spark.sparkContext.version) ++ constantValues) :: Nil
15031504
1504- // Data generated by previous versions.
1505- // scalastyle:off
1506- val existingJSONData =
1505+ // Data generated by previous versions.
1506+ // scalastyle:off
1507+ val existingJSONData =
15071508 """ {"col0":"Spark 1.2.2","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1508- """ {"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1509- """ {"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1510- """ {"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1511- """ {"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1512- """ {"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1513- """ {"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"16436","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil
1514- // scalastyle:on
1515-
1516- // Generate data for the current version.
1517- val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1 ), schema)
1518- withTempPath { path =>
1519- df.write.format(" json" ).mode(" overwrite" ).save(path.getCanonicalPath)
1509+ """ {"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1510+ """ {"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1511+ """ {"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1512+ """ {"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1513+ """ {"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
1514+ """ {"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"16436","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil
1515+ // scalastyle:on
1516+
1517+ // Generate data for the current version.
1518+ val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1 ), schema)
1519+ withTempPath { path =>
1520+ df.write.format(" json" ).mode(" overwrite" ).save(path.getCanonicalPath)
15201521
1521- // df.toJSON will convert internal rows to external rows first and then generate
1522- // JSON objects. While, df.write.format("json") will write internal rows directly.
1523- val allJSON =
1522+ // df.toJSON will convert internal rows to external rows first and then generate
1523+ // JSON objects. While, df.write.format("json") will write internal rows directly.
1524+ val allJSON =
15241525 existingJSONData ++
15251526 df.toJSON.collect() ++
15261527 sparkContext.textFile(path.getCanonicalPath).collect()
15271528
1528- Utils .deleteRecursively(path)
1529- sparkContext.parallelize(allJSON, 1 ).saveAsTextFile(path.getCanonicalPath)
1530-
1531- // Read data back with the schema specified.
1532- val col0Values =
1533- Seq (
1534- " Spark 1.2.2" ,
1535- " Spark 1.3.1" ,
1536- " Spark 1.3.1" ,
1537- " Spark 1.4.1" ,
1538- " Spark 1.4.1" ,
1539- " Spark 1.5.0" ,
1540- " Spark 1.5.0" ,
1541- " Spark " + spark.sparkContext.version,
1542- " Spark " + spark.sparkContext.version)
1543- val expectedResult = col0Values.map { v =>
1544- Row .fromSeq(Seq (v) ++ constantValues)
1529+ Utils .deleteRecursively(path)
1530+ sparkContext.parallelize(allJSON, 1 ).saveAsTextFile(path.getCanonicalPath)
1531+
1532+ // Read data back with the schema specified.
1533+ val col0Values =
1534+ Seq (
1535+ " Spark 1.2.2" ,
1536+ " Spark 1.3.1" ,
1537+ " Spark 1.3.1" ,
1538+ " Spark 1.4.1" ,
1539+ " Spark 1.4.1" ,
1540+ " Spark 1.5.0" ,
1541+ " Spark 1.5.0" ,
1542+ " Spark " + spark.sparkContext.version,
1543+ " Spark " + spark.sparkContext.version)
1544+ val expectedResult = col0Values.map { v =>
1545+ Row .fromSeq(Seq (v) ++ constantValues)
1546+ }
1547+ checkAnswer(
1548+ spark.read.format(" json" ).schema(schema).load(path.getCanonicalPath),
1549+ expectedResult
1550+ )
15451551 }
1546- checkAnswer(
1547- spark.read.format(" json" ).schema(schema).load(path.getCanonicalPath),
1548- expectedResult
1549- )
15501552 }
15511553 }
15521554
0 commit comments