[SPARK-24691][SQL] Dispatch the type support check in FileFormat implementation
[spark.git] / sql / core / src / main / scala / org / apache / spark / sql / execution / datasources / csv / CSVFileFormat.scala
1 /*
2  * Licensed to the Apache Software Foundation (ASF) under one or more
3  * contributor license agreements.  See the NOTICE file distributed with
4  * this work for additional information regarding copyright ownership.
5  * The ASF licenses this file to You under the Apache License, Version 2.0
6  * (the "License"); you may not use this file except in compliance with
7  * the License.  You may obtain a copy of the License at
8  *
9  *    http://www.apache.org/licenses/LICENSE-2.0
10  *
11  * Unless required by applicable law or agreed to in writing, software
12  * distributed under the License is distributed on an "AS IS" BASIS,
13  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14  * See the License for the specific language governing permissions and
15  * limitations under the License.
16  */
17
18 package org.apache.spark.sql.execution.datasources.csv
19
20 import org.apache.hadoop.conf.Configuration
21 import org.apache.hadoop.fs.{FileStatus, Path}
22 import org.apache.hadoop.mapreduce._
23
24 import org.apache.spark.internal.Logging
25 import org.apache.spark.sql.{AnalysisException, SparkSession}
26 import org.apache.spark.sql.catalyst.InternalRow
27 import org.apache.spark.sql.catalyst.util.CompressionCodecs
28 import org.apache.spark.sql.execution.datasources._
29 import org.apache.spark.sql.sources._
30 import org.apache.spark.sql.types._
31 import org.apache.spark.util.SerializableConfiguration
32
33 /**
34  * Provides access to CSV data from pure SQL statements.
35  */
36 class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister {
37
38   override def shortName(): String = "csv"
39
40   override def isSplitable(
41       sparkSession: SparkSession,
42       options: Map[String, String],
43       path: Path): Boolean = {
44     val parsedOptions = new CSVOptions(
45       options,
46       columnPruning = sparkSession.sessionState.conf.csvColumnPruning,
47       sparkSession.sessionState.conf.sessionLocalTimeZone)
48     val csvDataSource = CSVDataSource(parsedOptions)
49     csvDataSource.isSplitable && super.isSplitable(sparkSession, options, path)
50   }
51
52   override def inferSchema(
53       sparkSession: SparkSession,
54       options: Map[String, String],
55       files: Seq[FileStatus]): Option[StructType] = {
56     val parsedOptions = new CSVOptions(
57       options,
58       columnPruning = sparkSession.sessionState.conf.csvColumnPruning,
59       sparkSession.sessionState.conf.sessionLocalTimeZone)
60
61     CSVDataSource(parsedOptions).inferSchema(sparkSession, files, parsedOptions)
62   }
63
64   override def prepareWrite(
65       sparkSession: SparkSession,
66       job: Job,
67       options: Map[String, String],
68       dataSchema: StructType): OutputWriterFactory = {
69     val conf = job.getConfiguration
70     val csvOptions = new CSVOptions(
71       options,
72       columnPruning = sparkSession.sessionState.conf.csvColumnPruning,
73       sparkSession.sessionState.conf.sessionLocalTimeZone)
74     csvOptions.compressionCodec.foreach { codec =>
75       CompressionCodecs.setCodecConfiguration(conf, codec)
76     }
77
78     new OutputWriterFactory {
79       override def newInstance(
80           path: String,
81           dataSchema: StructType,
82           context: TaskAttemptContext): OutputWriter = {
83         new CsvOutputWriter(path, dataSchema, context, csvOptions)
84       }
85
86       override def getFileExtension(context: TaskAttemptContext): String = {
87         ".csv" + CodecStreams.getCompressionExtension(context)
88       }
89     }
90   }
91
92   override def buildReader(
93       sparkSession: SparkSession,
94       dataSchema: StructType,
95       partitionSchema: StructType,
96       requiredSchema: StructType,
97       filters: Seq[Filter],
98       options: Map[String, String],
99       hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
100     val broadcastedHadoopConf =
101       sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
102
103     val parsedOptions = new CSVOptions(
104       options,
105       sparkSession.sessionState.conf.csvColumnPruning,
106       sparkSession.sessionState.conf.sessionLocalTimeZone,
107       sparkSession.sessionState.conf.columnNameOfCorruptRecord)
108
109     // Check a field requirement for corrupt records here to throw an exception in a driver side
110     dataSchema.getFieldIndex(parsedOptions.columnNameOfCorruptRecord).foreach { corruptFieldIndex =>
111       val f = dataSchema(corruptFieldIndex)
112       if (f.dataType != StringType || !f.nullable) {
113         throw new AnalysisException(
114           "The field for corrupt records must be string type and nullable")
115       }
116     }
117
118     if (requiredSchema.length == 1 &&
119       requiredSchema.head.name == parsedOptions.columnNameOfCorruptRecord) {
120       throw new AnalysisException(
121         "Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the\n" +
122           "referenced columns only include the internal corrupt record column\n" +
123           s"(named _corrupt_record by default). For example:\n" +
124           "spark.read.schema(schema).csv(file).filter($\"_corrupt_record\".isNotNull).count()\n" +
125           "and spark.read.schema(schema).csv(file).select(\"_corrupt_record\").show().\n" +
126           "Instead, you can cache or save the parsed results and then send the same query.\n" +
127           "For example, val df = spark.read.schema(schema).csv(file).cache() and then\n" +
128           "df.filter($\"_corrupt_record\".isNotNull).count()."
129       )
130     }
131     val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
132
133     (file: PartitionedFile) => {
134       val conf = broadcastedHadoopConf.value.value
135       val parser = new UnivocityParser(
136         StructType(dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)),
137         StructType(requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)),
138         parsedOptions)
139       CSVDataSource(parsedOptions).readFile(
140         conf,
141         file,
142         parser,
143         requiredSchema,
144         dataSchema,
145         caseSensitive)
146     }
147   }
148
149   override def toString: String = "CSV"
150
151   override def hashCode(): Int = getClass.hashCode()
152
153   override def equals(other: Any): Boolean = other.isInstanceOf[CSVFileFormat]
154
155   override def supportDataType(dataType: DataType, isReadPath: Boolean): Boolean = dataType match {
156     case _: AtomicType => true
157
158     case udt: UserDefinedType[_] => supportDataType(udt.sqlType, isReadPath)
159
160     case _ => false
161   }
162
163 }
164
165 private[csv] class CsvOutputWriter(
166     path: String,
167     dataSchema: StructType,
168     context: TaskAttemptContext,
169     params: CSVOptions) extends OutputWriter with Logging {
170
171   private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path))
172
173   private val gen = new UnivocityGenerator(dataSchema, writer, params)
174
175   override def write(row: InternalRow): Unit = gen.write(row)
176
177   override def close(): Unit = gen.close()
178 }