[jira] [Commented] (SPARK-26325) Interpret timestamp fields in Spark while reading json (timestampFormat)

2021-02-02 Thread Daniel Himmelstein (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-26325?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17277405#comment-17277405
 ] 

Daniel Himmelstein commented on SPARK-26325:


h1. Solution in pyspark 3.0.1

Turns out there is an {{inferTimestamp }}option that must be enabled. >From the 
spark [migration 
guide|https://spark.apache.org/docs/latest/sql-migration-guide.html#upgrading-from-spark-sql-30-to-301]:
{quote}In Spark 3.0, JSON datasource and JSON function {{schema_of_json}} infer 
TimestampType from string values if they match to the pattern defined by the 
JSON option {{timestampFormat}}. Since version 3.0.1, the timestamp type 
inference is disabled by default. Set the JSON option {{inferTimestamp}} to 
{{true}} to enable such type inference.
{quote}
Surprised this would occur in a patch release and is not reflected yet in the 
[latest 
docs|https://spark.apache.org/docs/latest/api/java/org/apache/spark/sql/DataFrameReader.html].
 But looks like it correlated with a major performance decrease so was turned 
off by default: 
[apache/spark#28966|https://github.com/apache/spark/pull/28966], SPARK-26325, 
and SPARK-32130.

So in pyspark 3.0.1:
{code:python}
line = '{"time_field" : "2017-09-30 04:53:39.412496Z"}'
rdd = spark.sparkContext.parallelize([line])
(
spark.read
.option("inferTimestamp", "true")
.option("timestampFormat", "-MM-dd HH:mm:ss.SS'Z'")
.json(path=rdd)
){code}
Returns:
{code:java}
DataFrame[time_field: timestamp]
{code}
Yay!

> Interpret timestamp fields in Spark while reading json (timestampFormat)
> 
>
> Key: SPARK-26325
> URL: https://issues.apache.org/jira/browse/SPARK-26325
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.4.0
>Reporter: Veenit Shah
>Priority: Major
>
> I am trying to read a pretty printed json which has time fields in it. I want 
> to interpret the timestamps columns as timestamp fields while reading the 
> json itself. However, it's still reading them as string when I {{printSchema}}
> E.g. Input json file -
> {code:java}
> [{
> "time_field" : "2017-09-30 04:53:39.412496Z"
> }]
> {code}
> Code -
> {code:java}
> df = spark.read.option("multiLine", 
> "true").option("timestampFormat","-MM-dd 
> HH:mm:ss.SS'Z'").json('path_to_json_file')
> {code}
> Output of df.printSchema() -
> {code:java}
> root
>  |-- time_field: string (nullable = true)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-26325) Interpret timestamp fields in Spark while reading json (timestampFormat)

2021-02-01 Thread Daniel Himmelstein (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-26325?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17276711#comment-17276711
 ] 

Daniel Himmelstein commented on SPARK-26325:


Here's the code from the original post, but using an RDD rather than JSON file 
and applying [~maxgekk]'s suggestion to "try Z instead of 'Z'":
{code:python}
line = '{"time_field" : "2017-09-30 04:53:39.412496Z"}'
rdd = spark.sparkContext.parallelize([line])
(
spark.read
.option("timestampFormat", "-MM-dd HH:mm:ss.SSZ")
.json(path=rdd)
){code}
The output I get with pyspark 3.0.1 is `DataFrame[time_field: string]`. So it 
looks like the issue remains.

I'd be interested if there are any examples where spark infers a timestamp from 
a JSON string or whether timestampFormat​ does not work at all?

> Interpret timestamp fields in Spark while reading json (timestampFormat)
> 
>
> Key: SPARK-26325
> URL: https://issues.apache.org/jira/browse/SPARK-26325
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.4.0
>Reporter: Veenit Shah
>Priority: Major
>
> I am trying to read a pretty printed json which has time fields in it. I want 
> to interpret the timestamps columns as timestamp fields while reading the 
> json itself. However, it's still reading them as string when I {{printSchema}}
> E.g. Input json file -
> {code:java}
> [{
> "time_field" : "2017-09-30 04:53:39.412496Z"
> }]
> {code}
> Code -
> {code:java}
> df = spark.read.option("multiLine", 
> "true").option("timestampFormat","-MM-dd 
> HH:mm:ss.SS'Z'").json('path_to_json_file')
> {code}
> Output of df.printSchema() -
> {code:java}
> root
>  |-- time_field: string (nullable = true)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-26325) Interpret timestamp fields in Spark while reading json (timestampFormat)

2019-03-25 Thread Maxim Gekk (JIRA)


[ 
https://issues.apache.org/jira/browse/SPARK-26325?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16801011#comment-16801011
 ] 

Maxim Gekk commented on SPARK-26325:


Can you try Z instead of 'Z'?

> Interpret timestamp fields in Spark while reading json (timestampFormat)
> 
>
> Key: SPARK-26325
> URL: https://issues.apache.org/jira/browse/SPARK-26325
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.4.0
>Reporter: Veenit Shah
>Priority: Major
>
> I am trying to read a pretty printed json which has time fields in it. I want 
> to interpret the timestamps columns as timestamp fields while reading the 
> json itself. However, it's still reading them as string when I {{printSchema}}
> E.g. Input json file -
> {code:java}
> [{
> "time_field" : "2017-09-30 04:53:39.412496Z"
> }]
> {code}
> Code -
> {code:java}
> df = spark.read.option("multiLine", 
> "true").option("timestampFormat","-MM-dd 
> HH:mm:ss.SS'Z'").json('path_to_json_file')
> {code}
> Output of df.printSchema() -
> {code:java}
> root
>  |-- time_field: string (nullable = true)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org