You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by yhuai <gi...@git.apache.org> on 2015/09/18 02:42:15 UTC

[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values stored in...

GitHub user yhuai opened a pull request:

    https://github.com/apache/spark/pull/8806

    [SPARK-10495] [SQL] Read date values stored in Spark 1.5.0.

    https://issues.apache.org/jira/browse/SPARK-10681

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/yhuai/spark SPARK-10495

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/8806.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #8806
    
----
commit 593f2b2fca6da1209742add8b61f4522a5e95778
Author: Yin Huai <yh...@databricks.com>
Date:   2015-09-18T00:39:35Z

    Read date values stored in Spark 1.5.0.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39890833
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala ---
    @@ -1159,4 +1159,104 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
               "SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
         })
       }
    +
    +  // scalastyle:off
    +  test("backward compatibility") {
    +    // This test we make sure our JSON support can read JSON data generated by previous version of Spark
    +    // generated through toJSON method and JSON data source.
    +    // The data is generated by the following program.
    +    // Here are a few notes:
    +    //  - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13)
    +    //      in the JSON object.
    +    //  - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to
    +    //      JSON objects generated by those Spark versions (col17).
    +    //  - If the type is NullType, we do not write data out.
    +
    +    // Create the schema.
    +    val struct =
    +      StructType(
    +        StructField("f1", FloatType, true) ::
    +          StructField("f2", ArrayType(BooleanType), true) :: Nil)
    +
    +    val dataTypes =
    +      Seq(
    +        StringType, BinaryType, NullType, BooleanType,
    +        ByteType, ShortType, IntegerType, LongType,
    +        FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5),
    +        DateType, TimestampType,
    +        ArrayType(IntegerType), MapType(StringType, LongType), struct,
    +        new MyDenseVectorUDT())
    +    val fields = dataTypes.zipWithIndex.map { case (dataType, index) =>
    +        StructField(s"col$index", dataType, nullable = true)
    +      }
    +    val schema = StructType(fields)
    +
    +    val constantValues =
    +      Seq(
    +        "a string in binary".getBytes("UTF-8"),
    +        null,
    +        true,
    +        1.toByte,
    +        2.toShort,
    +        3,
    +        Long.MaxValue,
    +        0.25.toFloat,
    +        0.75,
    +        new java.math.BigDecimal(s"1234.23456"),
    +        new java.math.BigDecimal(s"1.23456"),
    +        java.sql.Date.valueOf("2015-01-01"),
    +        java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"),
    +        Seq(2, 3, 4),
    +        Map("a string" -> 2000L),
    +        Row(4.75.toFloat, Seq(false, true)),
    +        new MyDenseVector(Array(0.25, 2.25, 4.25)))
    +    val data = Row.fromSeq(Seq("Spark " + sqlContext.sparkContext.version) ++ constantValues) :: Nil
    +
    +    // Data generated by previous versions.
    +    val existingJSONData =
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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
    +
    +    // Generate data for the current version.
    +    val df = sqlContext.createDataFrame(sqlContext.sparkContext.parallelize(data, 1), schema)
    +    withTempPath { path =>
    +      df.write.format("json").mode("overwrite").save(path.getCanonicalPath)
    +
    +      // df.toJSON will convert internal rows to external rows first and then generate JSON objects.
    +      // While, df.write.format("json") will write internal rows directly.
    --- End diff --
    
    How about stop converting internal rows to external rows in `DataFrame.toJSON`? This can be done in a separate PR though.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39889195
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -62,10 +62,23 @@ private[sql] object JacksonParser {
             // guard the non string type
             null
     
    +      case (VALUE_STRING, BinaryType) =>
    +        parser.getBinaryValue
    +
           case (VALUE_STRING, DateType) =>
    -        DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +        val stringValue = parser.getText
    +        if (stringValue.contains("-")) {
    +          // The format of this string will probably be "yyyy-mm-dd".
    --- End diff --
    
    Not sure whether this is still true if we take different locales into consideration. Maybe use a `try` block here to parse the string as a date and then fallback to int?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141586833
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42687/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values stored in...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141297874
  
      [Test build #42630 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42630/consoleFull) for   PR 8806 at commit [`593f2b2`](https://github.com/apache/spark/commit/593f2b2fca6da1209742add8b61f4522a5e95778).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/8806


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141320859
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42630/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141555635
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39889832
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala ---
    @@ -73,6 +73,38 @@ private[sql] object JacksonGenerator {
                 valWriter(field.dataType, v)
             }
             gen.writeEndObject()
    +
    +      // For UDT, udt.serialize will produce SQL types. So, we need the following three cases.
    +      case (ArrayType(ty, _), v: ArrayData) =>
    +        gen.writeStartArray()
    +        v.foreach(ty, (_, value) => valWriter(ty, value))
    --- End diff --
    
    This foreach is provided by `ArrayData`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141561824
  
      [Test build #1774 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1774/consoleFull) for   PR 8806 at commit [`f0505de`](https://github.com/apache/spark/commit/f0505def78b32d0938ebef1adc7d9cf3c1803490).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39891131
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala ---
    @@ -1159,4 +1159,104 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
               "SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
         })
       }
    +
    +  // scalastyle:off
    +  test("backward compatibility") {
    +    // This test we make sure our JSON support can read JSON data generated by previous version of Spark
    +    // generated through toJSON method and JSON data source.
    +    // The data is generated by the following program.
    +    // Here are a few notes:
    +    //  - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13)
    +    //      in the JSON object.
    +    //  - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to
    +    //      JSON objects generated by those Spark versions (col17).
    +    //  - If the type is NullType, we do not write data out.
    +
    +    // Create the schema.
    +    val struct =
    +      StructType(
    +        StructField("f1", FloatType, true) ::
    +          StructField("f2", ArrayType(BooleanType), true) :: Nil)
    +
    +    val dataTypes =
    +      Seq(
    +        StringType, BinaryType, NullType, BooleanType,
    +        ByteType, ShortType, IntegerType, LongType,
    +        FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5),
    +        DateType, TimestampType,
    +        ArrayType(IntegerType), MapType(StringType, LongType), struct,
    +        new MyDenseVectorUDT())
    +    val fields = dataTypes.zipWithIndex.map { case (dataType, index) =>
    +        StructField(s"col$index", dataType, nullable = true)
    +      }
    +    val schema = StructType(fields)
    +
    +    val constantValues =
    +      Seq(
    +        "a string in binary".getBytes("UTF-8"),
    +        null,
    +        true,
    +        1.toByte,
    +        2.toShort,
    +        3,
    +        Long.MaxValue,
    +        0.25.toFloat,
    +        0.75,
    +        new java.math.BigDecimal(s"1234.23456"),
    +        new java.math.BigDecimal(s"1.23456"),
    +        java.sql.Date.valueOf("2015-01-01"),
    +        java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"),
    +        Seq(2, 3, 4),
    +        Map("a string" -> 2000L),
    +        Row(4.75.toFloat, Seq(false, true)),
    +        new MyDenseVector(Array(0.25, 2.25, 4.25)))
    +    val data = Row.fromSeq(Seq("Spark " + sqlContext.sparkContext.version) ++ constantValues) :: Nil
    +
    +    // Data generated by previous versions.
    +    val existingJSONData =
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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
    +
    +    // Generate data for the current version.
    +    val df = sqlContext.createDataFrame(sqlContext.sparkContext.parallelize(data, 1), schema)
    +    withTempPath { path =>
    +      df.write.format("json").mode("overwrite").save(path.getCanonicalPath)
    +
    +      // df.toJSON will convert internal rows to external rows first and then generate JSON objects.
    +      // While, df.write.format("json") will write internal rows directly.
    --- End diff --
    
    Especially, `udt.serialize` produces internal types, while all others are external types, which is pretty weird.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141320291
  
      [Test build #42630 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42630/console) for   PR 8806 at commit [`593f2b2`](https://github.com/apache/spark/commit/593f2b2fca6da1209742add8b61f4522a5e95778).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class TaskCommitDenied(`
      * `class Interaction(override val uid: String) extends Transformer`
      * `  final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities")`
      * `          s"Failed to convert value $v (class of $`
      * `          s"Failed to convert value $v (class of $`
      * `abstract class LocalNode(conf: SQLConf) extends QueryPlan[LocalNode] with Logging `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-142151935
  
    LGTM, merged to master and branch-1.5.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141582670
  
      [Test build #1774 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1774/console) for   PR 8806 at commit [`f0505de`](https://github.com/apache/spark/commit/f0505def78b32d0938ebef1adc7d9cf3c1803490).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class Interaction(override val uid: String) extends Transformer`
      * `          s"Failed to convert value $v (class of $`
      * `          s"Failed to convert value $v (class of $`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39891704
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala ---
    @@ -1159,4 +1159,104 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
               "SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
         })
       }
    +
    +  // scalastyle:off
    +  test("backward compatibility") {
    +    // This test we make sure our JSON support can read JSON data generated by previous version of Spark
    +    // generated through toJSON method and JSON data source.
    +    // The data is generated by the following program.
    +    // Here are a few notes:
    +    //  - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13)
    +    //      in the JSON object.
    +    //  - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to
    +    //      JSON objects generated by those Spark versions (col17).
    +    //  - If the type is NullType, we do not write data out.
    +
    +    // Create the schema.
    +    val struct =
    +      StructType(
    +        StructField("f1", FloatType, true) ::
    +          StructField("f2", ArrayType(BooleanType), true) :: Nil)
    +
    +    val dataTypes =
    +      Seq(
    +        StringType, BinaryType, NullType, BooleanType,
    +        ByteType, ShortType, IntegerType, LongType,
    +        FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5),
    +        DateType, TimestampType,
    +        ArrayType(IntegerType), MapType(StringType, LongType), struct,
    +        new MyDenseVectorUDT())
    +    val fields = dataTypes.zipWithIndex.map { case (dataType, index) =>
    +        StructField(s"col$index", dataType, nullable = true)
    +      }
    +    val schema = StructType(fields)
    +
    +    val constantValues =
    +      Seq(
    +        "a string in binary".getBytes("UTF-8"),
    +        null,
    +        true,
    +        1.toByte,
    +        2.toShort,
    +        3,
    +        Long.MaxValue,
    +        0.25.toFloat,
    +        0.75,
    +        new java.math.BigDecimal(s"1234.23456"),
    +        new java.math.BigDecimal(s"1.23456"),
    +        java.sql.Date.valueOf("2015-01-01"),
    +        java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"),
    +        Seq(2, 3, 4),
    +        Map("a string" -> 2000L),
    +        Row(4.75.toFloat, Seq(false, true)),
    +        new MyDenseVector(Array(0.25, 2.25, 4.25)))
    +    val data = Row.fromSeq(Seq("Spark " + sqlContext.sparkContext.version) ++ constantValues) :: Nil
    +
    +    // Data generated by previous versions.
    +    val existingJSONData =
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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
    +
    +    // Generate data for the current version.
    +    val df = sqlContext.createDataFrame(sqlContext.sparkContext.parallelize(data, 1), schema)
    +    withTempPath { path =>
    +      df.write.format("json").mode("overwrite").save(path.getCanonicalPath)
    +
    +      // df.toJSON will convert internal rows to external rows first and then generate JSON objects.
    +      // While, df.write.format("json") will write internal rows directly.
    --- End diff --
    
    Filed https://issues.apache.org/jira/browse/SPARK-10705 to track this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141562842
  
      [Test build #42687 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42687/consoleFull) for   PR 8806 at commit [`f0505de`](https://github.com/apache/spark/commit/f0505def78b32d0938ebef1adc7d9cf3c1803490).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values stored in...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141297280
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141561090
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141562019
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141562002
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39890261
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala ---
    @@ -1159,4 +1159,104 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
               "SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
         })
       }
    +
    +  // scalastyle:off
    +  test("backward compatibility") {
    +    // This test we make sure our JSON support can read JSON data generated by previous version of Spark
    +    // generated through toJSON method and JSON data source.
    +    // The data is generated by the following program.
    +    // Here are a few notes:
    +    //  - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13)
    +    //      in the JSON object.
    +    //  - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to
    +    //      JSON objects generated by those Spark versions (col17).
    +    //  - If the type is NullType, we do not write data out.
    +
    +    // Create the schema.
    +    val struct =
    +      StructType(
    +        StructField("f1", FloatType, true) ::
    +          StructField("f2", ArrayType(BooleanType), true) :: Nil)
    +
    +    val dataTypes =
    +      Seq(
    +        StringType, BinaryType, NullType, BooleanType,
    +        ByteType, ShortType, IntegerType, LongType,
    +        FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5),
    +        DateType, TimestampType,
    +        ArrayType(IntegerType), MapType(StringType, LongType), struct,
    +        new MyDenseVectorUDT())
    +    val fields = dataTypes.zipWithIndex.map { case (dataType, index) =>
    +        StructField(s"col$index", dataType, nullable = true)
    +      }
    --- End diff --
    
    Indentations are off.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141555620
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141320850
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values stored in...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141297268
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141561504
  
    test this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141586296
  
      [Test build #42687 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42687/console) for   PR 8806 at commit [`f0505de`](https://github.com/apache/spark/commit/f0505def78b32d0938ebef1adc7d9cf3c1803490).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `          s"Failed to convert value $v (class of $`
      * `          s"Failed to convert value $v (class of $`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141561091
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42683/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39888881
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala ---
    @@ -73,6 +73,38 @@ private[sql] object JacksonGenerator {
                 valWriter(field.dataType, v)
             }
             gen.writeEndObject()
    +
    +      // For UDT, udt.serialize will produce SQL types. So, we need the following three cases.
    +      case (ArrayType(ty, _), v: ArrayData) =>
    +        gen.writeStartArray()
    +        v.foreach(ty, (_, value) => valWriter(ty, value))
    +        gen.writeEndArray()
    +
    +      case (MapType(kt, vt, _), v: MapData) =>
    +        gen.writeStartObject()
    +        v.foreach(kt, vt, { (k, v) =>
    --- End diff --
    
    Same as above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39889891
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala ---
    @@ -1159,4 +1159,104 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
               "SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
         })
       }
    +
    +  // scalastyle:off
    --- End diff --
    
    Let's narrow the scope of this switch down to the JSON data part.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8806#issuecomment-141586830
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39891065
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala ---
    @@ -1159,4 +1159,104 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
               "SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
         })
       }
    +
    +  // scalastyle:off
    +  test("backward compatibility") {
    +    // This test we make sure our JSON support can read JSON data generated by previous version of Spark
    +    // generated through toJSON method and JSON data source.
    +    // The data is generated by the following program.
    +    // Here are a few notes:
    +    //  - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13)
    +    //      in the JSON object.
    +    //  - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to
    +    //      JSON objects generated by those Spark versions (col17).
    +    //  - If the type is NullType, we do not write data out.
    +
    +    // Create the schema.
    +    val struct =
    +      StructType(
    +        StructField("f1", FloatType, true) ::
    +          StructField("f2", ArrayType(BooleanType), true) :: Nil)
    +
    +    val dataTypes =
    +      Seq(
    +        StringType, BinaryType, NullType, BooleanType,
    +        ByteType, ShortType, IntegerType, LongType,
    +        FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5),
    +        DateType, TimestampType,
    +        ArrayType(IntegerType), MapType(StringType, LongType), struct,
    +        new MyDenseVectorUDT())
    +    val fields = dataTypes.zipWithIndex.map { case (dataType, index) =>
    +        StructField(s"col$index", dataType, nullable = true)
    +      }
    +    val schema = StructType(fields)
    +
    +    val constantValues =
    +      Seq(
    +        "a string in binary".getBytes("UTF-8"),
    +        null,
    +        true,
    +        1.toByte,
    +        2.toShort,
    +        3,
    +        Long.MaxValue,
    +        0.25.toFloat,
    +        0.75,
    +        new java.math.BigDecimal(s"1234.23456"),
    +        new java.math.BigDecimal(s"1.23456"),
    +        java.sql.Date.valueOf("2015-01-01"),
    +        java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"),
    +        Seq(2, 3, 4),
    +        Map("a string" -> 2000L),
    +        Row(4.75.toFloat, Seq(false, true)),
    +        new MyDenseVector(Array(0.25, 2.25, 4.25)))
    +    val data = Row.fromSeq(Seq("Spark " + sqlContext.sparkContext.version) ++ constantValues) :: Nil
    +
    +    // Data generated by previous versions.
    +    val existingJSONData =
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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]}""" ::
    +      """{"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
    +
    +    // Generate data for the current version.
    +    val df = sqlContext.createDataFrame(sqlContext.sparkContext.parallelize(data, 1), schema)
    +    withTempPath { path =>
    +      df.write.format("json").mode("overwrite").save(path.getCanonicalPath)
    +
    +      // df.toJSON will convert internal rows to external rows first and then generate JSON objects.
    +      // While, df.write.format("json") will write internal rows directly.
    +      val allJSON =
    +        existingJSONData ++
    +          df.toJSON.collect() ++
    +          sparkContext.textFile(path.getCanonicalPath).collect()
    +
    +      Utils.deleteRecursively(path)
    +      sparkContext.parallelize(allJSON, 1).saveAsTextFile(path.getCanonicalPath)
    +
    +      // Read data back with the schema specified.
    +      val col0Values =
    +        Seq(
    +          "Spark 1.2.2",
    +          "Spark 1.3.1",
    +          "Spark 1.3.1",
    +          "Spark 1.4.1",
    +          "Spark 1.4.1",
    +          "Spark 1.5.0",
    +          "Spark 1.5.0",
    +          "Spark " + sqlContext.sparkContext.version,
    +          "Spark " + sqlContext.sparkContext.version)
    +      val expectedResult = col0Values.map { v =>
    +        Row.fromSeq(Seq(v) ++ constantValues)
    +      }
    +      checkAnswer(
    +        sqlContext.read.format("json").schema(schema).load(path.getCanonicalPath),
    +          expectedResult
    --- End diff --
    
    Indentation is off.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-10495] [SQL] Read date values in JSON d...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8806#discussion_r39888865
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala ---
    @@ -73,6 +73,38 @@ private[sql] object JacksonGenerator {
                 valWriter(field.dataType, v)
             }
             gen.writeEndObject()
    +
    +      // For UDT, udt.serialize will produce SQL types. So, we need the following three cases.
    +      case (ArrayType(ty, _), v: ArrayData) =>
    +        gen.writeStartArray()
    +        v.foreach(ty, (_, value) => valWriter(ty, value))
    --- End diff --
    
    Should probably replace this `foreach` with a `while`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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