Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-35662 Support Timestamp without time zone data type
  3. SPARK-36491

Make from_json/to_json to handle timestamp_ntz type properly

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.3.0
    • 3.3.0
    • SQL
    • None

    Description

      In the current master, to_json/from_json can handle timestamp type like as follows.

      SELECT to_json(map("a", TIMESTAMP"2021-11-23 11:22:33"));
      {"a":"2021-11-23T11:22:33.000+09:00"}
      
      SELECT from_json('{"a":"2021-11-23 11:22:33"}', "a TIMESTAMP");
      {"a":2021-11-23 11:22:33}
      

      But they cannot handle timestamp_ntz type properly.

      SELECT to_json(map("a", TIMESTAMP_NTZ"2021-11-23 11:22:33"));
      21/08/12 16:14:07 ERROR SparkSQLDriver: Failed in [SELECT to_json(map("a", TIMESTAMP_NTZ"2021-11-23 11:22:33"))]
      java.lang.RuntimeException: Failed to convert value 1637666553000000 (class of class java.lang.Long) with the type of TimestampNTZType to JSON.
              at org.apache.spark.sql.errors.QueryExecutionErrors$.failToConvertValueToJsonError(QueryExecutionErrors.scala:294)
              at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$makeWriter$25(JacksonGenerator.scala:201)
              at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$makeWriter$25$adapted(JacksonGenerator.scala:199)
              at org.apache.spark.sql.catalyst.json.JacksonGenerator.writeMapData(JacksonGenerator.scala:253)
              at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$write$3(JacksonGenerator.scala:293)
              at org.apache.spark.sql.catalyst.json.JacksonGenerator.writeObject(JacksonGenerator.scala:206)
              at org.apache.spark.sql.catalyst.json.JacksonGenerator.write(JacksonGenerator.scala:292)
      
      SELECT from_json('{"a":"2021-11-23 11:22:33"}', "a TIMESTAMP_NTZ");
      21/08/12 16:16:00 ERROR SparkSQLDriver: Failed in [SELECT from_json('{"a":"2021-11-23 11:22:33"}', "a TIMESTAMP_NTZ")]
      java.lang.Exception: Unsupported type: timestamp_ntz
              at org.apache.spark.sql.errors.QueryExecutionErrors$.unsupportedTypeError(QueryExecutionErrors.scala:777)
              at org.apache.spark.sql.catalyst.json.JacksonParser.makeConverter(JacksonParser.scala:339)
              at org.apache.spark.sql.catalyst.json.JacksonParser.$anonfun$makeConverter$17(JacksonParser.scala:313)
      

      Attachments

        Activity

          People

            sarutak Kousuke Saruta
            sarutak Kousuke Saruta
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: