-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-27085][SQL] Migrate CSV to File Data Source V2 #24005
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 6 commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
ce1a4e4
csv data source V2
gengliangwang b363a00
disable csv write path
gengliangwang 5288011
fix
gengliangwang 3f6df2d
address comments
gengliangwang 256e177
use caseSensitive Map for newHadoopConfWithOptions
gengliangwang 2be5277
fix test
gengliangwang 7eb54a3
address comments
gengliangwang 689f17e
revise
gengliangwang File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
2 changes: 1 addition & 1 deletion
2
...core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
37 changes: 37 additions & 0 deletions
37
...ain/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,37 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.spark.sql.execution.datasources.v2 | ||
|
|
||
| import org.apache.spark.sql.sources.v2.reader.PartitionReader | ||
|
|
||
| class PartitionReaderFromIterator[InternalRow]( | ||
| iter: Iterator[InternalRow]) extends PartitionReader[InternalRow] { | ||
| private var nextValue: InternalRow = _ | ||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| override def next(): Boolean = { | ||
| if (iter.hasNext) { | ||
| nextValue = iter.next() | ||
| true | ||
| } else { | ||
| false | ||
| } | ||
| } | ||
|
|
||
| override def get(): InternalRow = nextValue | ||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| override def close(): Unit = {} | ||
| } | ||
45 changes: 45 additions & 0 deletions
45
...core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TextBasedFileScan.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,45 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.spark.sql.execution.datasources.v2 | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} | ||
|
|
||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
| abstract class TextBasedFileScan( | ||
| sparkSession: SparkSession, | ||
| fileIndex: PartitioningAwareFileIndex, | ||
| readSchema: StructType, | ||
| options: CaseInsensitiveStringMap) | ||
| extends FileScan(sparkSession, fileIndex, readSchema, options) { | ||
| private var codecFactory: CompressionCodecFactory = _ | ||
|
|
||
| override def isSplitable(path: Path): Boolean = { | ||
| if (codecFactory == null) { | ||
| codecFactory = new CompressionCodecFactory( | ||
| sparkSession.sessionState.newHadoopConfWithOptions(options.asScala.toMap)) | ||
| } | ||
| val codec = codecFactory.getCodec(path) | ||
| codec == null || codec.isInstanceOf[SplittableCompressionCodec] | ||
| } | ||
| } |
57 changes: 57 additions & 0 deletions
57
...re/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,57 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.spark.sql.execution.datasources.v2.csv | ||
|
|
||
| import org.apache.spark.sql.execution.datasources._ | ||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat | ||
| import org.apache.spark.sql.execution.datasources.v2._ | ||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| import org.apache.spark.sql.sources.v2.Table | ||
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
| class CSVDataSourceV2 extends FileDataSourceV2 { | ||
|
|
||
| override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[CSVFileFormat] | ||
|
|
||
| override def shortName(): String = "csv" | ||
|
|
||
| private def getTableName(paths: Seq[String]): String = { | ||
| shortName() + ":" + paths.mkString(";") | ||
| } | ||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| override def getTable(options: CaseInsensitiveStringMap): Table = { | ||
| val paths = getPaths(options) | ||
| val tableName = getTableName(paths) | ||
| CSVTable(tableName, sparkSession, options, paths, None) | ||
| } | ||
|
|
||
| override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { | ||
| val paths = getPaths(options) | ||
| val tableName = getTableName(paths) | ||
| CSVTable(tableName, sparkSession, options, paths, Some(schema)) | ||
| } | ||
| } | ||
|
|
||
| object CSVDataSourceV2 { | ||
| def supportsDataType(dataType: DataType): Boolean = dataType match { | ||
| case _: AtomicType => true | ||
|
|
||
| case udt: UserDefinedType[_] => supportsDataType(udt.sqlType) | ||
|
|
||
| case _ => false | ||
| } | ||
| } | ||
72 changes: 72 additions & 0 deletions
72
...n/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,72 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.spark.sql.execution.datasources.v2.csv | ||
|
|
||
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} | ||
| import org.apache.spark.sql.execution.datasources.PartitionedFile | ||
| import org.apache.spark.sql.execution.datasources.csv.CSVDataSource | ||
| import org.apache.spark.sql.execution.datasources.v2._ | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.sources.v2.reader.PartitionReader | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.util.SerializableConfiguration | ||
|
|
||
| /** | ||
| * A factory used to create CSV readers. | ||
| * | ||
| * @param sqlConf SQL configuration. | ||
| * @param broadcastedConf Broadcasted serializable Hadoop Configuration. | ||
| * @param dataSchema Schema of CSV files. | ||
| * @param partitionSchema Schema of partitions. | ||
| * @param readSchema Required schema in the batch scan. | ||
| */ | ||
| case class CSVPartitionReaderFactory( | ||
| sqlConf: SQLConf, | ||
| broadcastedConf: Broadcast[SerializableConfiguration], | ||
| dataSchema: StructType, | ||
| partitionSchema: StructType, | ||
| readSchema: StructType, | ||
| parsedOptions: CSVOptions) extends FilePartitionReaderFactory { | ||
| private val columnPruning = sqlConf.csvColumnPruning | ||
| private val readDataSchema = | ||
| getReadDataSchema(readSchema, partitionSchema, sqlConf.caseSensitiveAnalysis) | ||
|
|
||
| override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { | ||
| val conf = broadcastedConf.value.value | ||
|
|
||
| val parser = new UnivocityParser( | ||
| StructType(dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)), | ||
| StructType(readDataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)), | ||
| parsedOptions) | ||
| val schema = if (columnPruning) readDataSchema else dataSchema | ||
| val isStartOfFile = file.start == 0 | ||
| val headerChecker = new CSVHeaderChecker( | ||
| schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) | ||
| val iter = CSVDataSource(parsedOptions).readFile( | ||
| conf, | ||
| file, | ||
| parser, | ||
| headerChecker, | ||
| readSchema) | ||
HyukjinKwon marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| val fileReader = new PartitionReaderFromIterator[InternalRow](iter) | ||
| new PartitionReaderWithPartitionValues(fileReader, readDataSchema, | ||
| partitionSchema, file.partitionValues) | ||
| } | ||
|
|
||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
85 changes: 85 additions & 0 deletions
85
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,85 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.spark.sql.execution.datasources.v2.csv | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.sql.{AnalysisException, SparkSession} | ||
| import org.apache.spark.sql.catalyst.csv.CSVOptions | ||
| import org.apache.spark.sql.catalyst.expressions.ExprUtils | ||
| import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex | ||
| import org.apache.spark.sql.execution.datasources.csv.CSVDataSource | ||
| import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan | ||
| import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory | ||
| import org.apache.spark.sql.types.{DataType, StructType} | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
| import org.apache.spark.util.SerializableConfiguration | ||
|
|
||
| case class CSVScan( | ||
| sparkSession: SparkSession, | ||
| fileIndex: PartitioningAwareFileIndex, | ||
| dataSchema: StructType, | ||
| readSchema: StructType, | ||
| options: CaseInsensitiveStringMap) | ||
| extends TextBasedFileScan(sparkSession, fileIndex, readSchema, options) { | ||
|
|
||
| private val optionsAsScala = options.asScala.toMap | ||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| private lazy val parsedOptions: CSVOptions = new CSVOptions( | ||
| optionsAsScala, | ||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| columnPruning = sparkSession.sessionState.conf.csvColumnPruning, | ||
| sparkSession.sessionState.conf.sessionLocalTimeZone, | ||
| sparkSession.sessionState.conf.columnNameOfCorruptRecord) | ||
|
|
||
| override def isSplitable(path: Path): Boolean = { | ||
| CSVDataSource(parsedOptions).isSplitable && super.isSplitable(path) | ||
| } | ||
|
|
||
| override def createReaderFactory(): PartitionReaderFactory = { | ||
| // Check a field requirement for corrupt records here to throw an exception in a driver side | ||
| ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) | ||
|
|
||
| if (readSchema.length == 1 && | ||
| readSchema.head.name == parsedOptions.columnNameOfCorruptRecord) { | ||
| throw new AnalysisException( | ||
| "Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the\n" + | ||
| "referenced columns only include the internal corrupt record column\n" + | ||
| s"(named _corrupt_record by default). For example:\n" + | ||
dongjoon-hyun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| "spark.read.schema(schema).csv(file).filter($\"_corrupt_record\".isNotNull).count()\n" + | ||
| "and spark.read.schema(schema).csv(file).select(\"_corrupt_record\").show().\n" + | ||
| "Instead, you can cache or save the parsed results and then send the same query.\n" + | ||
| "For example, val df = spark.read.schema(schema).csv(file).cache() and then\n" + | ||
| "df.filter($\"_corrupt_record\".isNotNull).count()." | ||
| ) | ||
| } | ||
|
|
||
| val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap | ||
| // Hadoop Configurations are case sensitive. | ||
| val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) | ||
| val broadcastedConf = sparkSession.sparkContext.broadcast( | ||
| new SerializableConfiguration(hadoopConf)) | ||
| CSVPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, | ||
| dataSchema, fileIndex.partitionSchema, readSchema, parsedOptions) | ||
| } | ||
|
|
||
| override def supportsDataType(dataType: DataType): Boolean = { | ||
| CSVDataSourceV2.supportsDataType(dataType) | ||
| } | ||
|
|
||
| override def formatName: String = "CSV" | ||
| } | ||
37 changes: 37 additions & 0 deletions
37
...ore/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,37 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.execution.datasources.v2.csv | ||
|
|
||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex | ||
| import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder | ||
| import org.apache.spark.sql.sources.v2.reader.Scan | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
| case class CSVScanBuilder( | ||
| sparkSession: SparkSession, | ||
| fileIndex: PartitioningAwareFileIndex, | ||
| schema: StructType, | ||
| dataSchema: StructType, | ||
| options: CaseInsensitiveStringMap) extends FileScanBuilder(schema) { | ||
|
|
||
| override def build(): Scan = { | ||
| CSVScan(sparkSession, fileIndex, dataSchema, readSchema, options) | ||
gengliangwang marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.