Github user ueshin commented on the issue:

    https://github.com/apache/spark/pull/18664
  
    I disagree with using `DateTimeUtils.defaultTimeZone()` for the timezone.
    If `DateTimeUtils.defaultTimeZone()` is different from system timezone in 
Python, the return values are different between with-Arrow and without-Arrow.
    Current without-Arrow seems to respect system timezone in Python, so we 
should respect it and keep the timezone for Arrow type what it should be.
    
    Let me show the example (similar to 
https://github.com/apache/spark/pull/18664#issuecomment-318791052):
    
    I'm in Japan, so `DateTimeUtils.defaultTimeZone()` would be `"Asia/Tokyo"`, 
and modify `os.environ['TZ']` to `"America/Los_Angeles"` and session timezone 
to `"America/New_York"`.
    
    ```
    >>> import datetime
    >>> import time
    >>> from pyspark.sql.types import *
    >>>
    >>> os.environ['TZ'] = "America/Los_Angeles"
    >>> time.tzset()
    >>>
    >>> dt = datetime.datetime(1970, 1, 1, 0, 0, 1)
    >>> dt
    datetime.datetime(1970, 1, 1, 0, 0, 1)
    >>>
    >>> spark.conf.set("spark.sql.session.timeZone", "America/New_York")
    >>> df = spark.createDataFrame([(dt,)], 
schema=StructType([StructField("ts", TimestampType(), True)]))
    >>> df.show()
    +-------------------+
    |                 ts|
    +-------------------+
    |1970-01-01 03:00:01|
    +-------------------+
    
    >>> spark.conf.set("spark.sql.execution.arrow.enabled", "false")
    >>> df.toPandas()
                       ts
    0 1970-01-01 00:00:01
    ```
    
    - Current HEAD (c4fd5ae1e61d118877151d713e8ea0e53a7afaf4) + fix compilation 
error:
    
    ```
    >>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
    >>> df.toPandas()
                       ts
    0 1970-01-01 03:00:01
    ```
    
    - Use `DateTimeUtils.defaultTimeZone()` as the timezone:
    
    ```
    >>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
    >>> df.toPandas()
                       ts
    0 1970-01-01 17:00:01
    ```
    
    - Adjust using system timezone in Python:
    
    ```
    >>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
    >>> df.toPandas()
                       ts
    0 1970-01-01 00:00:01
    ```
    
    A patch I used here is:
    
    ```diff
     def _localize_series_timestamps(s):
         """ Convert a tz-aware timestamp to local tz-naive
         """
    -    return s.dt.tz_localize(None)
    +    return s.dt.tz_convert('tzlocal()').dt.tz_localize(None)
    ```



---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to