Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-35662

Support Timestamp without time zone data type

    XMLWordPrintableJSON

Details

    • New Feature
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.4.0
    • 3.4.0
    • SQL
    • None

    Description

      Spark SQL today supports the TIMESTAMP data type. However the semantics provided actually match TIMESTAMP WITH LOCAL TIMEZONE as defined by Oracle. Timestamps embedded in a SQL query or passed through JDBC are presumed to be in session local timezone and cast to UTC before being processed.
      These are desirable semantics in many cases, such as when dealing with calendars.
      In many (more) other cases, such as when dealing with log files it is desirable that the provided timestamps not be altered.
      SQL users expect that they can model either behavior and do so by using TIMESTAMP WITHOUT TIME ZONE for time zone insensitive data and TIMESTAMP WITH LOCAL TIME ZONE for time zone sensitive data.
      Most traditional RDBMS map TIMESTAMP to TIMESTAMP WITHOUT TIME ZONE and will be surprised to see TIMESTAMP WITH LOCAL TIME ZONE, a feature that does not exist in the standard.

      In this new feature, we will introduce TIMESTAMP WITH LOCAL TIMEZONE to describe the existing timestamp type and add TIMESTAMP WITHOUT TIME ZONE for standard semantic.
      Using these two types will provide clarity.
      We will also allow users to set the default behavior for TIMESTAMP to either use TIMESTAMP WITH LOCAL TIME ZONE or TIMESTAMP WITHOUT TIME ZONE.

      Milestone 1 – Spark Timestamp equivalency ( The new Timestamp type TimestampWithoutTZ meets or exceeds all function of the existing SQL Timestamp):

      • Add a new DataType implementation for TimestampWithoutTZ.
      • Support TimestampWithoutTZ in Dataset/UDF.
      • TimestampWithoutTZ literals
      • TimestampWithoutTZ arithmetic(e.g. TimestampWithoutTZ - TimestampWithoutTZ, TimestampWithoutTZ - Date)
      • Datetime functions/operators: dayofweek, weekofyear, year, etc
      • Cast to and from TimestampWithoutTZ, cast String/Timestamp to TimestampWithoutTZ, cast TimestampWithoutTZ to string (pretty printing)/Timestamp, with the SQL syntax to specify the types
      • Support sorting TimestampWithoutTZ.

      Milestone 2 – Persistence:

      • Ability to create tables of type TimestampWithoutTZ
      • Ability to write to common file formats such as Parquet and JSON.
      • INSERT, SELECT, UPDATE, MERGE
      • Discovery

      Milestone 3 – Client support

      • JDBC support
      • Hive Thrift server

      Milestone 4 – PySpark and Spark R integration

      • Python UDF can take and return TimestampWithoutTZ
      • DataFrame support

      Attachments

        Issue Links

          1.
          Add Timestamp without time zone type Sub-task Resolved Gengliang Wang
          2.
          Support java.time. LocalDateTime as an external type of TimestampWithoutTZ type Sub-task Resolved Gengliang Wang
          3.
          Test timestamp without time zone in UDF Sub-task Resolved Gengliang Wang
          4.
          Test TimestampWithoutTZType as ordered and atomic type Sub-task Resolved Apache Spark
          5.
          Support casting of timestamp without time zone to strings Sub-task Resolved Gengliang Wang
          6.
          Support casting of timestamp without time zone to timestamp type Sub-task Resolved Gengliang Wang
          7.
          Support casting of timestamp without time zone to date type Sub-task Resolved Gengliang Wang
          8.
          Support casting of Date to timestamp without time zone type Sub-task Resolved Gengliang Wang
          9.
          Support type conversion between timestamp and timestamp without time zone type Sub-task Resolved Gengliang Wang
          10.
          Support casting of String to timestamp without time zone type Sub-task Resolved Gengliang Wang
          11.
          Assign pretty names to TimestampWithoutTZType Sub-task Resolved Gengliang Wang
          12.
          New SQL function: to_timestamp_ntz Sub-task Resolved Gengliang Wang
          13.
          Improve the error message of to_timestamp_ntz with invalid format pattern Sub-task Resolved Gengliang Wang
          14.
          Support adding TimestampWithoutTZ with Interval types Sub-task Resolved Gengliang Wang
          15.
          Support subtracting Intervals from TimestampWithoutTZ Sub-task Resolved Apache Spark
          16.
          Support subtraction among Date/Timestamp/TimestampWithoutTZ Sub-task Resolved Gengliang Wang
          17.
          Remove type collection AllTimestampTypes Sub-task Resolved Gengliang Wang
          18.
          Support extracting hour/minute/second from timestamp without time zone Sub-task Resolved Gengliang Wang
          19.
          Support extracting date fields from timestamp without time zone Sub-task Resolved Gengliang Wang
          20.
          Rename TimestampWithoutTZType to TimestampNTZType Sub-task Resolved Gengliang Wang
          21.
          Rename the type name of TimestampNTZType as "timestamp_ntz" Sub-task Resolved Gengliang Wang
          22.
          New configuration spark.sql.timestampType for the default timestamp type Sub-task Resolved Gengliang Wang
          23.
          Support non-reserved keyword TIMESTAMP_NTZ Sub-task Resolved Gengliang Wang
          24.
          Support non-reserved keyword TIMESTAMP_LTZ Sub-task Resolved Gengliang Wang
          25.
          Return different timestamp literals based on the default timestamp type Sub-task Resolved Gengliang Wang
          26.
          Support TimestampNTZType in the Window spec definition Sub-task Resolved Jiaan Geng
          27.
          Support TimestampNTZType in expression ApproxCountDistinctForIntervals Sub-task Resolved Jiaan Geng
          28.
          Support TimestampNTZType in expression ApproximatePercentile Sub-task Resolved Jiaan Geng
          29.
          Support ANSI SQL LOCALTIMESTAMP datetime value function Sub-task Resolved Jiaan Geng
          30.
          Add end-to-end tests with default timestamp type as TIMESTAMP_NTZ Sub-task Resolved Gengliang Wang
          31.
          Suport TimestampNTZ in functions unix_timestamp/to_unix_timestamp Sub-task Resolved Jiaan Geng
          32.
          TO_UTC_TIMESTAMP and FROM_UTC_TIMESTAMP should return TimestampNTZ Sub-task Resolved Unassigned
          33.
          Support new functions make_timestamp_ntz and make_timestamp_ltz Sub-task Resolved Gengliang Wang
          34.
          Spark doesn’t support reading/writing TIMESTAMP_NTZ with ORC Sub-task Resolved Gengliang Wang
          35.
          Support group by TimestampNTZ column Sub-task Resolved Gengliang Wang
          36.
          Assign pretty SQL string to TimestampNTZ literals Sub-task Resolved Gengliang Wang
          37.
          TO_TIMESTAMP: return different results based on the default timestamp type Sub-task Resolved Gengliang Wang
          38.
          Support TimestampNTZType in SparkGetColumnsOperation Sub-task Resolved Kent Yao 2
          39.
          make_timestamp: return different result based on the default timestamp type Sub-task Resolved Gengliang Wang
          40.
          Support TimestampNTZType in expression Sequence Sub-task Resolved Jiaan Geng
          41.
          Support TimestampNTZ type in expression TimeWindow Sub-task Resolved Jiaan Geng
          42.
          Support casting of timestamp without time zone to numeric type Sub-task Resolved Unassigned
          43.
          Add new SQL function to_timestamp_ltz Sub-task Resolved Gengliang Wang
          44.
          Support TimestampNTZ type in cache table Sub-task Resolved Gengliang Wang
          45.
          Support TimestampNTZ type in file partitioning Sub-task Resolved Gengliang Wang
          46.
          Support TimestampNTZ in Avro data source Sub-task Resolved Jiaan Geng
          47.
          Support TimestampNTZ type in Parquet file source Sub-task Resolved Gengliang Wang
          48.
          Updated the version of TimestampNTZ related changes as 3.3.0 Sub-task Resolved Gengliang Wang
          49.
          Remove TimestampNTZ type support in Spark 3.2 Sub-task Resolved Gengliang Wang
          50.
          Support TimestampNTZ type in Hive Sub-task Resolved Unassigned
          51.
          Support TimestampNTZ type in Orc file source Sub-task Resolved Jiaan Geng
          52.
          Support pushdown Timestamp with local time zone for orc Sub-task Resolved Jiaan Geng
          53.
          Splitting test cases from datetime.sql Sub-task Resolved Wenchen Fan
          54.
          Make from_csv/to_csv to handle timestamp_ntz type properly Sub-task Resolved Kousuke Saruta
          55.
          Make from_json/to_json to handle timestamp_ntz type properly Sub-task Resolved Kousuke Saruta
          56.
          Support TimestampNTZ in Arrow Sub-task Resolved Hyukjin Kwon
          57.
          Support TimestampNTZ in pandas API on Spark Sub-task Resolved Hyukjin Kwon
          58.
          Support TimestampNTZ in createDataFrame/toPandas and Python UDFs Sub-task Resolved Hyukjin Kwon
          59.
          Support TimestampNTZ in Py4J Sub-task Resolved Hyukjin Kwon
          60.
          Support ScriptTransformation for timestamp_ntz Sub-task Resolved Kousuke Saruta
          61.
          Support timestamp_ntz as a type of time column for SessionWindow Sub-task Resolved Kousuke Saruta
          62.
          Support TimestampNTZ in CSV data source Sub-task Resolved Ivan Sadikov
          63.
          Support TimestampNTZ in JSON data source Sub-task Resolved Ivan Sadikov
          64.
          Add tests for TimestampNTZ and TimestampLTZ for Parquet data source Sub-task Resolved Ivan Sadikov
          65.
          Allow store assignment between TimestampNTZ and Date/Timestamp Sub-task Resolved Gengliang Wang
          66.
          Support TimestampNTZ radix sort Sub-task Resolved Gengliang Wang
          67.
          Support TimestampNTZ in RowToColumnConverter Sub-task Resolved Gengliang Wang
          68.
          Remove TimestampNTZ type support in Spark 3.3 Sub-task Resolved Gengliang Wang
          69.
          Remove TimestampNTZ type Python support in Spark 3.3 Sub-task Resolved Haejoon Lee
          70.
          New configuration for controlling timestamp inference of Parquet Sub-task Resolved Ivan Sadikov
          71.
          Update the version of TimestampNTZ related changes as 3.4.0 Sub-task Resolved Gengliang Wang
          72.
          Allow comparison between TimestampNTZ and Timestamp/Date Sub-task Resolved Gengliang Wang
          73.
          Can't read TimestampNTZ as TimestampLTZ Sub-task Resolved Jiaan Geng
          74.
          Read/Write Timestamp ntz from/to Orc uses int64 Sub-task Resolved Jiaan Geng
          75.
          Support TimestampNTZ in JDBC data source Sub-task Resolved Ivan Sadikov
          76.
          Date and timestamp type can up cast to TimestampNTZ Sub-task Resolved Gengliang Wang
          77.
          Introduce a new conf for TimestampNTZ schema inference in JSON/CSV Sub-task Resolved Gengliang Wang
          78.
          Use `spark.sql.inferTimestampNTZInDataSources.enabled` to infer timestamp type on partition columns Sub-task Resolved Gengliang Wang
          79.
          Introduce conf spark.sql.parquet.inferTimestampNTZ.enabled for TimestampNTZ inference on Parquet Sub-task Resolved Gengliang Wang
          80.
          Apply spark.sql.inferTimestampNTZInDataSources.enabled on JDBC data source Sub-task Resolved Gengliang Wang
          81.
          Rename TimestampNTZ inference conf as spark.sql.sources.timestampNTZTypeInference.enabled Sub-task Resolved Gengliang Wang
          82.
          Add documentation for TimestampNTZ type Sub-task Resolved Gengliang Wang
          83.
          Use spark.sql.timestampType for data source inference Sub-task Resolved Gengliang Wang
          84.
          Rename JDBC option inferTimestampNTZType as preferTimestampNTZ Sub-task Resolved Gengliang Wang
          85.
          Support parser data type json "timestamp_ltz" as TimestampType Sub-task Resolved Gengliang Wang
          86.
          Support analyze TimestampNTZ columns Sub-task Resolved Gengliang Wang
          87.
          Support converting TimestampNTZ catalog stats to plan stats Sub-task Resolved Gengliang Wang
          88.
          Support TimestampNTZ in Cached Batch Sub-task Resolved Gengliang Wang
          89.
          Include TIMESTAMP_NTZ in ANSI Compliance doc Sub-task Resolved Gengliang Wang
          90.
          Casting between Timestamp and TimestampNTZ requires timezone Sub-task Resolved Gengliang Wang
          91.
          Remove inferTimestampNTZ config check in ParquetRowConverter Sub-task Resolved Gengliang Wang
          92.
          Add migration doc: TimestampNTZ type inference on Parquet files Sub-task Resolved Gengliang Wang
          93.
          Allow reading Parquet TimestampLTZ as TimestampNTZ Sub-task Resolved Gengliang Wang
          94.
          Disable spark.sql.parquet.inferTimestampNTZ.enabled by default Sub-task Open Gengliang Wang
          95.
          Add migration doc for the behavior change of Parquet timestamp inference since Spark 3.3 Sub-task Resolved Gengliang Wang

          Activity

            People

              apachespark Apache Spark
              Gengliang.Wang Gengliang Wang
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: