Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1482,7 +1482,7 @@ object SQLConf {
" register class names for which data source V2 write paths are disabled. Writes from these" +
" sources will fall back to the V1 sources.")
.stringConf
.createWithDefault("orc")
.createWithDefault("csv,orc")

val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers")
.doc("A comma-separated list of fully qualified data source register class names for which" +
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
org.apache.spark.sql.execution.datasources.csv.CSVFileFormat
org.apache.spark.sql.execution.datasources.v2.csv.CSVDataSourceV2
org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider
org.apache.spark.sql.execution.datasources.json.JsonFileFormat
org.apache.spark.sql.execution.datasources.noop.NoopDataSource
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,9 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier}
import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils}
import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat
import org.apache.spark.sql.execution.datasources.json.JsonFileFormat
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.execution.datasources.v2.csv.CSVDataSourceV2
import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -238,7 +238,7 @@ case class AlterTableAddColumnsCommand(
// TextFileFormat only default to one column "value"
// Hive type is already considered as hive serde table, so the logic will not
// come in here.
case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat | _: OrcDataSourceV2 =>
case _: JsonFileFormat | _: CSVDataSourceV2 | _: ParquetFileFormat | _: OrcDataSourceV2 =>
case s if s.getClass.getCanonicalName.endsWith("OrcFileFormat") =>
case s =>
throw new AnalysisException(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister {
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" +
"(named _corrupt_record by default). For example:\n" +
"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" +
Expand Down Expand Up @@ -163,31 +163,3 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister {

}

private[csv] class CsvOutputWriter(
path: String,
dataSchema: StructType,
context: TaskAttemptContext,
params: CSVOptions) extends OutputWriter with Logging {

private var univocityGenerator: Option[UnivocityGenerator] = None

if (params.headerFlag) {
val gen = getGen()
gen.writeHeaders()
}

private def getGen(): UnivocityGenerator = univocityGenerator.getOrElse {
val charset = Charset.forName(params.charset)
val os = CodecStreams.createOutputStreamWriter(context, new Path(path), charset)
val newGen = new UnivocityGenerator(dataSchema, os, params)
univocityGenerator = Some(newGen)
newGen
}

override def write(row: InternalRow): Unit = {
val gen = getGen()
gen.write(row)
}

override def close(): Unit = univocityGenerator.foreach(_.close())
}
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.csv

import java.nio.charset.Charset

import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.TaskAttemptContext

import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.csv.{CSVOptions, UnivocityGenerator}
import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter}
import org.apache.spark.sql.types.StructType

class CsvOutputWriter(
path: String,
dataSchema: StructType,
context: TaskAttemptContext,
params: CSVOptions) extends OutputWriter with Logging {

private var univocityGenerator: Option[UnivocityGenerator] = None

if (params.headerFlag) {
val gen = getGen()
gen.writeHeaders()
}

private def getGen(): UnivocityGenerator = univocityGenerator.getOrElse {
val charset = Charset.forName(params.charset)
val os = CodecStreams.createOutputStreamWriter(context, new Path(path), charset)
val newGen = new UnivocityGenerator(dataSchema, os, params)
univocityGenerator = Some(newGen)
newGen
}

override def write(row: InternalRow): Unit = {
val gen = getGen()
gen.write(row)
}

override def close(): Unit = univocityGenerator.foreach(_.close())
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,4 +47,8 @@ trait FileDataSourceV2 extends TableProvider with DataSourceRegister {
Option(map.get("path")).toSeq
}
}

protected def getTableName(paths: Seq[String]): String = {
shortName() + ":" + paths.mkString(";")
}
}
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 currentValue: InternalRow = _

override def next(): Boolean = {
if (iter.hasNext) {
currentValue = iter.next()
true
} else {
false
}
}

override def get(): InternalRow = currentValue

override def close(): Unit = {}
}
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]
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.FileFormat
import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat
import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
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"

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
}
}
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.
* @param parsedOptions Options for parsing CSV files.
*/
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,
readDataSchema)
val fileReader = new PartitionReaderFromIterator[InternalRow](iter)
new PartitionReaderWithPartitionValues(fileReader, readDataSchema,
partitionSchema, file.partitionValues)
}
}
Loading