Skip to content

Commit eab10f9

Browse files
MaxGekkHyukjinKwon
authored andcommitted
[SPARK-24068][BACKPORT-2.3] Propagating DataFrameReader's options to Text datasource on schema inferring
## What changes were proposed in this pull request? While reading CSV or JSON files, DataFrameReader's options are converted to Hadoop's parameters, for example there: https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L302 but the options are not propagated to Text datasource on schema inferring, for instance: https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala#L184-L188 The PR proposes propagation of user's options to Text datasource on scheme inferring in similar way as user's options are converted to Hadoop parameters if schema is specified. ## How was this patch tested? The changes were tested manually by using https://github.com/twitter/hadoop-lzo: ``` hadoop-lzo> mvn clean package hadoop-lzo> ln -s ./target/hadoop-lzo-0.4.21-SNAPSHOT.jar ./hadoop-lzo.jar ``` Create 2 test files in JSON and CSV format and compress them: ```shell $ cat test.csv col1|col2 a|1 $ lzop test.csv $ cat test.json {"col1":"a","col2":1} $ lzop test.json ``` Run `spark-shell` with hadoop-lzo: ``` bin/spark-shell --jars ~/hadoop-lzo/hadoop-lzo.jar ``` reading compressed CSV and JSON without schema: ```scala spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("inferSchema",true).option("header",true).option("sep","|").csv("test.csv.lzo").show() +----+----+ |col1|col2| +----+----+ | a| 1| +----+----+ ``` ```scala spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("multiLine", true).json("test.json.lzo").printSchema root |-- col1: string (nullable = true) |-- col2: long (nullable = true) ``` Author: Maxim Gekk <[email protected]> Author: Maxim Gekk <[email protected]> Closes #21292 from MaxGekk/text-options-backport-v2.3.
1 parent 8889d78 commit eab10f9

File tree

5 files changed

+16
-12
lines changed

5 files changed

+16
-12
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.util._
3131
* Most of these map directly to Jackson's internal options, specified in [[JsonParser.Feature]].
3232
*/
3333
private[sql] class JSONOptions(
34-
@transient private val parameters: CaseInsensitiveMap[String],
34+
@transient val parameters: CaseInsensitiveMap[String],
3535
defaultTimeZoneId: String,
3636
defaultColumnNameOfCorruptRecord: String)
3737
extends Logging with Serializable {

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -184,7 +184,8 @@ object TextInputCSVDataSource extends CSVDataSource {
184184
DataSource.apply(
185185
sparkSession,
186186
paths = paths,
187-
className = classOf[TextFileFormat].getName
187+
className = classOf[TextFileFormat].getName,
188+
options = options.parameters
188189
).resolveRelation(checkFilesExist = false))
189190
.select("value").as[String](Encoders.STRING)
190191
} else {
@@ -248,7 +249,8 @@ object MultiLineCSVDataSource extends CSVDataSource {
248249
options: CSVOptions): RDD[PortableDataStream] = {
249250
val paths = inputPaths.map(_.getPath)
250251
val name = paths.mkString(",")
251-
val job = Job.getInstance(sparkSession.sessionState.newHadoopConf())
252+
val job = Job.getInstance(sparkSession.sessionState.newHadoopConfWithOptions(
253+
options.parameters))
252254
FileInputFormat.setInputPaths(job, paths: _*)
253255
val conf = job.getConfiguration
254256

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ import org.apache.spark.internal.Logging
2727
import org.apache.spark.sql.catalyst.util._
2828

2929
class CSVOptions(
30-
@transient private val parameters: CaseInsensitiveMap[String],
30+
@transient val parameters: CaseInsensitiveMap[String],
3131
defaultTimeZoneId: String,
3232
defaultColumnNameOfCorruptRecord: String)
3333
extends Logging with Serializable {

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.datasources.csv
1919

2020
import java.io.InputStream
2121
import java.math.BigDecimal
22-
import java.text.NumberFormat
23-
import java.util.Locale
2422

2523
import scala.util.Try
2624
import scala.util.control.NonFatal

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala

Lines changed: 10 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -92,7 +92,7 @@ object TextInputJsonDataSource extends JsonDataSource {
9292
sparkSession: SparkSession,
9393
inputPaths: Seq[FileStatus],
9494
parsedOptions: JSONOptions): StructType = {
95-
val json: Dataset[String] = createBaseDataset(sparkSession, inputPaths)
95+
val json: Dataset[String] = createBaseDataset(sparkSession, inputPaths, parsedOptions)
9696
inferFromDataset(json, parsedOptions)
9797
}
9898

@@ -104,13 +104,15 @@ object TextInputJsonDataSource extends JsonDataSource {
104104

105105
private def createBaseDataset(
106106
sparkSession: SparkSession,
107-
inputPaths: Seq[FileStatus]): Dataset[String] = {
107+
inputPaths: Seq[FileStatus],
108+
parsedOptions: JSONOptions): Dataset[String] = {
108109
val paths = inputPaths.map(_.getPath.toString)
109110
sparkSession.baseRelationToDataFrame(
110111
DataSource.apply(
111112
sparkSession,
112113
paths = paths,
113-
className = classOf[TextFileFormat].getName
114+
className = classOf[TextFileFormat].getName,
115+
options = parsedOptions.parameters
114116
).resolveRelation(checkFilesExist = false))
115117
.select("value").as(Encoders.STRING)
116118
}
@@ -144,16 +146,18 @@ object MultiLineJsonDataSource extends JsonDataSource {
144146
sparkSession: SparkSession,
145147
inputPaths: Seq[FileStatus],
146148
parsedOptions: JSONOptions): StructType = {
147-
val json: RDD[PortableDataStream] = createBaseRdd(sparkSession, inputPaths)
149+
val json: RDD[PortableDataStream] = createBaseRdd(sparkSession, inputPaths, parsedOptions)
148150
val sampled: RDD[PortableDataStream] = JsonUtils.sample(json, parsedOptions)
149151
JsonInferSchema.infer(sampled, parsedOptions, createParser)
150152
}
151153

152154
private def createBaseRdd(
153155
sparkSession: SparkSession,
154-
inputPaths: Seq[FileStatus]): RDD[PortableDataStream] = {
156+
inputPaths: Seq[FileStatus],
157+
parsedOptions: JSONOptions): RDD[PortableDataStream] = {
155158
val paths = inputPaths.map(_.getPath)
156-
val job = Job.getInstance(sparkSession.sessionState.newHadoopConf())
159+
val job = Job.getInstance(sparkSession.sessionState.newHadoopConfWithOptions(
160+
parsedOptions.parameters))
157161
val conf = job.getConfiguration
158162
val name = paths.mkString(",")
159163
FileInputFormat.setInputPaths(job, paths: _*)

0 commit comments

Comments
 (0)