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

Willi Raschkowski commented on SPARK-35324:
-------------------------------------------

Same reading JSON:

{code:title=test.json}
{"date": "2/6/18"}
{code}

{code}
scala> spark.conf.set("spark.sql.legacy.timeParserPolicy", "legacy")

scala> spark.read.option("dateFormat", "dd/MM/yy").schema("date 
date").json("/Users/wraschkowski/Downloads/test.json").collect
res5: Array[org.apache.spark.sql.Row] = Array([2018-06-02])

scala> spark.read.option("dateFormat", "dd/MM/yy").schema("date 
date").json("/Users/wraschkowski/Downloads/test.json").rdd.collect
21/05/06 00:22:27 ERROR Executor: Exception in task 0.0 in stage 4.0 (TID 4)
org.apache.spark.SparkUpgradeException: You may get a different result due to 
the upgrading of Spark 3.0: Fail to parse '2/6/18' in the new parser. You can 
set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before 
Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
        at 
org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper$$anonfun$checkParsedDiff$1.applyOrElse(DateTimeFormatterHelper.scala:150)
        at 
org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper$$anonfun$checkParsedDiff$1.applyOrElse(DateTimeFormatterHelper.scala:141)
        at 
scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38)
        at 
org.apache.spark.sql.catalyst.util.Iso8601DateFormatter.$anonfun$parse$1(DateFormatter.scala:61)
        at 
scala.runtime.java8.JFunction0$mcI$sp.apply(JFunction0$mcI$sp.java:23)
        at scala.Option.getOrElse(Option.scala:189)
        at 
org.apache.spark.sql.catalyst.util.Iso8601DateFormatter.parse(DateFormatter.scala:58)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser$$anonfun$$nestedInanonfun$makeConverter$10$1.applyOrElse(JacksonParser.scala:260)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser$$anonfun$$nestedInanonfun$makeConverter$10$1.applyOrElse(JacksonParser.scala:257)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser.parseJsonToken(JacksonParser.scala:343)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser.$anonfun$makeConverter$10(JacksonParser.scala:257)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser.org$apache$spark$sql$catalyst$json$JacksonParser$$convertObject(JacksonParser.scala:397)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser$$anonfun$$nestedInanonfun$makeStructRootConverter$3$1.applyOrElse(JacksonParser.scala:96)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser$$anonfun$$nestedInanonfun$makeStructRootConverter$3$1.applyOrElse(JacksonParser.scala:95)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser.parseJsonToken(JacksonParser.scala:343)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser.$anonfun$makeStructRootConverter$3(JacksonParser.scala:95)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser.$anonfun$parse$2(JacksonParser.scala:467)
        at org.apache.spark.util.Utils$.tryWithResource(Utils.scala:2611)
        at 
org.apache.spark.sql.catalyst.json.JacksonParser.parse(JacksonParser.scala:462)
        at 
org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource$.$anonfun$readFile$7(JsonDataSource.scala:140)
        at 
org.apache.spark.sql.catalyst.util.FailureSafeParser.parse(FailureSafeParser.scala:60)
        at 
org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource$.$anonfun$readFile$9(JsonDataSource.scala:144)
        at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:93)
        at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:173)
        at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:93)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at scala.collection.Iterator.foreach(Iterator.scala:941)
        at scala.collection.Iterator.foreach$(Iterator.scala:941)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
        at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
        at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
        at scala.collection.TraversableOnce.to(TraversableOnce.scala:315)
        at scala.collection.TraversableOnce.to$(TraversableOnce.scala:313)
        at scala.collection.AbstractIterator.to(Iterator.scala:1429)
        at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:307)
        at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:307)
        at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1429)
        at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:294)
        at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:288)
        at scala.collection.AbstractIterator.toArray(Iterator.scala:1429)
        at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1030)
        at 
org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2242)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:131)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:834)
Caused by: java.time.format.DateTimeParseException: Text '2/6/18' could not be 
parsed at index 0
        at 
java.base/java.time.format.DateTimeFormatter.parseResolved0(DateTimeFormatter.java:2046)
        at 
java.base/java.time.format.DateTimeFormatter.parse(DateTimeFormatter.java:1874)
        at 
org.apache.spark.sql.catalyst.util.Iso8601DateFormatter.$anonfun$parse$1(DateFormatter.scala:59)
        ... 53 more
{code}

> Time parser policy not respected in RDD
> ---------------------------------------
>
>                 Key: SPARK-35324
>                 URL: https://issues.apache.org/jira/browse/SPARK-35324
>             Project: Spark
>          Issue Type: Bug
>          Components: Input/Output, SQL
>    Affects Versions: 3.0.2, 3.1.1
>            Reporter: Willi Raschkowski
>            Priority: Major
>
> When reading a CSV file, {{spark.sql.timeParserPolicy}} is respected in 
> actions on the resulting dataframe. But it's ignored in actions on 
> dataframe's RDD.
> E.g. say to parse dates in a CSV you need {{spark.sql.timeParserPolicy}} to 
> be set to {{LEGACY}}. If you set the config, {{df.count}} will work as you'd 
> expect. However, {{df.count.rdd}} will fail because it'll ignore the override 
> and read the config value as {{EXCEPTION}}.
> For instance:
> {code:java|title=test.csv}
> date
> 2/6/18
> {code}
> {code:java}
> scala> spark.conf.set("spark.sql.legacy.timeParserPolicy", "legacy")
> scala> val df = {
>      |   spark.read
>      |     .option("header", "true")
>      |     .option("dateFormat", "MM/dd/yy")
>      |     .schema("date date")
>      |     .csv("/Users/wraschkowski/Downloads/test.csv")
>      | }
> df: org.apache.spark.sql.DataFrame = [date: date]
> scala> df.show
> +----------+                                                                  
>   
> |      date|
> +----------+
> |2018-02-06|
> +----------+
> scala> df.count
> res3: Long = 1
> scala> df.rdd.count
> 21/05/06 00:06:18 ERROR Executor: Exception in task 0.0 in stage 4.0 (TID 4)
> org.apache.spark.SparkUpgradeException: You may get a different result due to 
> the upgrading of Spark 3.0: Fail to parse '2/6/18' in the new parser. You can 
> set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior 
> before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime 
> string.
>       at 
> org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper$$anonfun$checkParsedDiff$1.applyOrElse(DateTimeFormatterHelper.scala:150)
>       at 
> org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper$$anonfun$checkParsedDiff$1.applyOrElse(DateTimeFormatterHelper.scala:141)
>       at 
> scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38)
>       at 
> org.apache.spark.sql.catalyst.util.Iso8601DateFormatter.$anonfun$parse$1(DateFormatter.scala:61)
>       at 
> scala.runtime.java8.JFunction0$mcI$sp.apply(JFunction0$mcI$sp.java:23)
>       at scala.Option.getOrElse(Option.scala:189)
>       at 
> org.apache.spark.sql.catalyst.util.Iso8601DateFormatter.parse(DateFormatter.scala:58)
>       at 
> org.apache.spark.sql.catalyst.csv.UnivocityParser.$anonfun$makeConverter$21(UnivocityParser.scala:202)
>       at 
> org.apache.spark.sql.catalyst.csv.UnivocityParser.nullSafeDatum(UnivocityParser.scala:238)
>       at 
> org.apache.spark.sql.catalyst.csv.UnivocityParser.$anonfun$makeConverter$20(UnivocityParser.scala:200)
>       at 
> org.apache.spark.sql.catalyst.csv.UnivocityParser.org$apache$spark$sql$catalyst$csv$UnivocityParser$$convert(UnivocityParser.scala:291)
>       at 
> org.apache.spark.sql.catalyst.csv.UnivocityParser.$anonfun$parse$2(UnivocityParser.scala:254)
>       at 
> org.apache.spark.sql.catalyst.csv.UnivocityParser$.$anonfun$parseIterator$1(UnivocityParser.scala:396)
>       at 
> org.apache.spark.sql.catalyst.util.FailureSafeParser.parse(FailureSafeParser.scala:60)
>       at 
> org.apache.spark.sql.catalyst.csv.UnivocityParser$.$anonfun$parseIterator$2(UnivocityParser.scala:400)
>       at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
>       at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
>       at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:93)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:173)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:93)
>       at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
>       at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
>       at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
>       at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1866)
>       at org.apache.spark.rdd.RDD.$anonfun$count$1(RDD.scala:1253)
>       at org.apache.spark.rdd.RDD.$anonfun$count$1$adapted(RDD.scala:1253)
>       at 
> org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2242)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>       at org.apache.spark.scheduler.Task.run(Task.scala:131)
>       at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
>       at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
>       at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>       at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>       at java.base/java.lang.Thread.run(Thread.java:834)
> Caused by: java.time.format.DateTimeParseException: Text '2/6/18' could not 
> be parsed at index 0
>       at 
> java.base/java.time.format.DateTimeFormatter.parseResolved0(DateTimeFormatter.java:2046)
>       at 
> java.base/java.time.format.DateTimeFormatter.parse(DateTimeFormatter.java:1874)
>       at 
> org.apache.spark.sql.catalyst.util.Iso8601DateFormatter.$anonfun$parse$1(DateFormatter.scala:59)
>       ... 32 more
> {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

Reply via email to