[ https://issues.apache.org/jira/browse/SPARK-26384?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16724201#comment-16724201 ]
ASF GitHub Bot commented on SPARK-26384: ---------------------------------------- asfgit closed pull request #23345: [SPARK-26384][SQL] Propagate SQL configs for CSV schema inferring URL: https://github.com/apache/spark/pull/23345 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala index b46dfb94c133e..375cec597166c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala @@ -35,6 +35,7 @@ import org.apache.spark.rdd.{BinaryFileRDD, RDD} import org.apache.spark.sql.{Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVInferSchema, CSVOptions, UnivocityParser} +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.types.StructType @@ -135,7 +136,9 @@ object TextInputCSVDataSource extends CSVDataSource { val parser = new CsvParser(parsedOptions.asParserSettings) linesWithoutHeader.map(parser.parseLine) } - new CSVInferSchema(parsedOptions).infer(tokenRDD, header) + SQLExecution.withSQLConfPropagated(csv.sparkSession) { + new CSVInferSchema(parsedOptions).infer(tokenRDD, header) + } case _ => // If the first line could not be read, just return the empty schema. StructType(Nil) @@ -208,7 +211,9 @@ object MultiLineCSVDataSource extends CSVDataSource { encoding = parsedOptions.charset) } val sampled = CSVUtils.sample(tokenRDD, parsedOptions) - new CSVInferSchema(parsedOptions).infer(sampled, header) + SQLExecution.withSQLConfPropagated(sparkSession) { + new CSVInferSchema(parsedOptions).infer(sampled, header) + } case None => // If the first row could not be read, just return the empty schema. StructType(Nil) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > CSV schema inferring does not respect spark.sql.legacy.timeParser.enabled > ------------------------------------------------------------------------- > > Key: SPARK-26384 > URL: https://issues.apache.org/jira/browse/SPARK-26384 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 3.0.0 > Reporter: Maxim Gekk > Assignee: Maxim Gekk > Priority: Major > Fix For: 3.0.0 > > > Starting from the commit > [https://github.com/apache/spark/commit/f982ca07e80074bdc1e3b742c5e21cf368e4ede2] > , add logging like in the comment > https://github.com/apache/spark/pull/23150#discussion_r242021998 and run: > {code:shell} > $ ./bin/spark-shell --conf spark.sql.legacy.timeParser.enabled=true > {code} > and in the shell: > {code:scala} > scala> spark.conf.get("spark.sql.legacy.timeParser.enabled") > res0: String = true > scala> > Seq("2010|10|10").toDF.repartition(1).write.mode("overwrite").text("/tmp/foo") > scala> spark.read.option("inferSchema", "true").option("header", > "false").option("timestampFormat", "yyyy|MM|dd").csv("/tmp/foo").printSchema() > 18/12/17 12:11:47 ERROR TimestampFormatter: Iso8601TimestampFormatter is > being used > root > |-- _c0: timestamp (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