spark-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Enrico Minack <m...@Enrico.Minack.dev>
Subject Re: Unablee to get to_timestamp with Timezone Information
Date Thu, 02 Apr 2020 15:49:10 GMT
Once parsed into a Timestamp the timestamp is store internally as UTC 
and printed as your local timezone (e.g. as defined by 
spark.sql.session.timeZone). Spark is good at hiding timezone 
information from you.

You can get the timezone information via date_format(column, format):

import org.apache.spark.sql.types.TimestampType
import org.apache.spark.sql.functions._

val sampleDF = Seq("2020-04-11T20:40:00-05:00").toDF("value")
val timestampDF = sampleDF.select($"value".cast(TimestampType))
timestampDF.select(date_format($"value", 
"yyyy-MM-dd'T'HH:mm:ssZZZZ")).show(false)
+---------------------------------------------+
|date_format(value, yyyy-MM-dd'T'HH:mm:ssZZZZ)|
+---------------------------------------------+
|2020-04-12T03:40:00+0200                     |
+---------------------------------------------+

If you want the timezone only, use 
timestampDF.select(date_format($"value", "ZZZZ")).show.
+------------------------+
|date_format(value, ZZZZ)|
+------------------------+
|                   +0200|
+------------------------+

It all depends how you get the data "downstream". If you go through 
parquet or csv files, they will retain the timezone information. If you 
go through strings, you should format them as above. If you use 
Dataset.map you can access the timestamps as java.sql.Timestamp objects 
(but that might not be necessary):

import java.sql.Timestamp
case class Times(value: Timestamp)
timestampDF.as[Times].map(t => t.value.getTimezoneOffset).show
+-----+
|value|
+-----+
| -120|
+-----+


Enrico


Am 31.03.20 um 21:40 schrieb Chetan Khatri:
> Sorry misrepresentation the question also. Thanks for your great help.
>
> What I want is the time zone information as it is 
> 2020-04-11T20:40:00-05:00 in timestamp datatype. so I can write to 
> downstream application as it is. I can correct the lacking UTC offset 
> info.
>
>
> On Tue, Mar 31, 2020 at 1:15 PM Magnus Nilsson <magnn@kth.se 
> <mailto:magnn@kth.se>> wrote:
>
>     And to answer your question (sorry, read too fast). The string is
>     not in proper ISO8601. Extended form must be used throughout, ie
>     2020-04-11T20:40:00-05:00, there's a colon (:) lacking in the UTC
>     offset info.
>
>     br,
>
>     Magnus
>
>     On Tue, Mar 31, 2020 at 7:11 PM Magnus Nilsson <magnn@kth.se
>     <mailto:magnn@kth.se>> wrote:
>
>         Timestamps aren't timezoned. If you parse ISO8601 strings they
>         will be converted to UTC automatically.
>
>         If you parse timestamps without timezone they will converted
>         to the the timezone the server Spark is running on uses. You
>         can change the timezone Spark uses with
>         spark.conf.set("spark.sql.session.timeZone","UTC"). Timestamps
>         represent a point in time, the clock representation of that
>         instant is dependent on sparks timezone settings both for
>         parsing (non ISO8601) strings and showing timestamps.
>
>         br,
>
>         Magnus
>
>         On Tue, Mar 31, 2020 at 6:14 PM Chetan Khatri
>         <chetan.opensource@gmail.com
>         <mailto:chetan.opensource@gmail.com>> wrote:
>
>             Hi Spark Users,
>
>             I am losing the timezone value from below format, I tried
>             couple of formats but not able to make it. Can someone
>             throw lights?
>
>             scala> val sampleDF =
>             Seq("2020-04-11T20:40:00-0500").toDF("value")
>             sampleDF: org.apache.spark.sql.DataFrame = [value: string]
>
>             scala> sampleDF.select('value, to_timestamp('value,
>             "yyyy-MM-dd\'T\'HH:mm:ss")).show(false)
>             +------------------------+------------------------------------------------+
>             |value                   |to_timestamp(`value`,
>             'yyyy-MM-dd\'T\'HH:mm:ss')|
>             +------------------------+------------------------------------------------+
>             |2020-04-11T20:40:00-0500|2020-04-11 20:40:00            
>                           |
>             +------------------------+------------------------------------------------+
>
>             Thanks
>


Mime
View raw message