Skip to content

Commit e3de6ab

Browse files
MaxGekkHyukjinKwon
authored andcommitted
[SPARK-24068] 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 #21182 from MaxGekk/text-options.
1 parent 487faf1 commit e3de6ab

File tree

6 files changed

+12
-12
lines changed

6 files changed

+12
-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
@@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.util._
3232
* Most of these map directly to Jackson's internal options, specified in [[JsonParser.Feature]].
3333
*/
3434
private[sql] class JSONOptions(
35-
@transient private val parameters: CaseInsensitiveMap[String],
35+
@transient val parameters: CaseInsensitiveMap[String],
3636
defaultTimeZoneId: String,
3737
defaultColumnNameOfCorruptRecord: String)
3838
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
@@ -185,7 +185,8 @@ object TextInputCSVDataSource extends CSVDataSource {
185185
DataSource.apply(
186186
sparkSession,
187187
paths = paths,
188-
className = classOf[TextFileFormat].getName
188+
className = classOf[TextFileFormat].getName,
189+
options = options.parameters
189190
).resolveRelation(checkFilesExist = false))
190191
.select("value").as[String](Encoders.STRING)
191192
} else {
@@ -250,7 +251,8 @@ object MultiLineCSVDataSource extends CSVDataSource {
250251
options: CSVOptions): RDD[PortableDataStream] = {
251252
val paths = inputPaths.map(_.getPath)
252253
val name = paths.mkString(",")
253-
val job = Job.getInstance(sparkSession.sessionState.newHadoopConf())
254+
val job = Job.getInstance(sparkSession.sessionState.newHadoopConfWithOptions(
255+
options.parameters))
254256
FileInputFormat.setInputPaths(job, paths: _*)
255257
val conf = job.getConfiguration
256258

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/CSVUtils.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,8 @@
1717

1818
package org.apache.spark.sql.execution.datasources.csv
1919

20-
import org.apache.spark.input.PortableDataStream
2120
import org.apache.spark.rdd.RDD
2221
import org.apache.spark.sql.Dataset
23-
import org.apache.spark.sql.catalyst.json.JSONOptions
2422
import org.apache.spark.sql.functions._
2523
import org.apache.spark.sql.types._
2624

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: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -121,7 +121,7 @@ object TextInputJsonDataSource extends JsonDataSource {
121121
sparkSession,
122122
paths = paths,
123123
className = classOf[TextFileFormat].getName,
124-
options = textOptions
124+
options = parsedOptions.parameters
125125
).resolveRelation(checkFilesExist = false))
126126
.select("value").as(Encoders.STRING)
127127
}
@@ -159,7 +159,7 @@ object MultiLineJsonDataSource extends JsonDataSource {
159159
sparkSession: SparkSession,
160160
inputPaths: Seq[FileStatus],
161161
parsedOptions: JSONOptions): StructType = {
162-
val json: RDD[PortableDataStream] = createBaseRdd(sparkSession, inputPaths)
162+
val json: RDD[PortableDataStream] = createBaseRdd(sparkSession, inputPaths, parsedOptions)
163163
val sampled: RDD[PortableDataStream] = JsonUtils.sample(json, parsedOptions)
164164
val parser = parsedOptions.encoding
165165
.map(enc => createParser(enc, _: JsonFactory, _: PortableDataStream))
@@ -170,9 +170,11 @@ object MultiLineJsonDataSource extends JsonDataSource {
170170

171171
private def createBaseRdd(
172172
sparkSession: SparkSession,
173-
inputPaths: Seq[FileStatus]): RDD[PortableDataStream] = {
173+
inputPaths: Seq[FileStatus],
174+
parsedOptions: JSONOptions): RDD[PortableDataStream] = {
174175
val paths = inputPaths.map(_.getPath)
175-
val job = Job.getInstance(sparkSession.sessionState.newHadoopConf())
176+
val job = Job.getInstance(sparkSession.sessionState.newHadoopConfWithOptions(
177+
parsedOptions.parameters))
176178
val conf = job.getConfiguration
177179
val name = paths.mkString(",")
178180
FileInputFormat.setInputPaths(job, paths: _*)

0 commit comments

Comments
 (0)