|
| 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 java.io.InputStream |
| 21 | +import java.nio.charset.{Charset, StandardCharsets} |
| 22 | + |
| 23 | +import com.univocity.parsers.csv.{CsvParser, CsvParserSettings} |
| 24 | +import org.apache.hadoop.conf.Configuration |
| 25 | +import org.apache.hadoop.fs.{FileStatus, Path} |
| 26 | +import org.apache.hadoop.io.{LongWritable, Text} |
| 27 | +import org.apache.hadoop.mapred.TextInputFormat |
| 28 | +import org.apache.hadoop.mapreduce.Job |
| 29 | +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat |
| 30 | + |
| 31 | +import org.apache.spark.TaskContext |
| 32 | +import org.apache.spark.input.{PortableDataStream, StreamInputFormat} |
| 33 | +import org.apache.spark.rdd.{BinaryFileRDD, RDD} |
| 34 | +import org.apache.spark.sql.{Dataset, Encoders, SparkSession} |
| 35 | +import org.apache.spark.sql.catalyst.InternalRow |
| 36 | +import org.apache.spark.sql.execution.datasources._ |
| 37 | +import org.apache.spark.sql.execution.datasources.text.TextFileFormat |
| 38 | +import org.apache.spark.sql.types.StructType |
| 39 | + |
| 40 | +/** |
| 41 | + * Common functions for parsing CSV files |
| 42 | + */ |
| 43 | +abstract class CSVDataSource extends Serializable { |
| 44 | + def isSplitable: Boolean |
| 45 | + |
| 46 | + /** |
| 47 | + * Parse a [[PartitionedFile]] into [[InternalRow]] instances. |
| 48 | + */ |
| 49 | + def readFile( |
| 50 | + conf: Configuration, |
| 51 | + file: PartitionedFile, |
| 52 | + parser: UnivocityParser, |
| 53 | + parsedOptions: CSVOptions): Iterator[InternalRow] |
| 54 | + |
| 55 | + /** |
| 56 | + * Infers the schema from `inputPaths` files. |
| 57 | + */ |
| 58 | + def infer( |
| 59 | + sparkSession: SparkSession, |
| 60 | + inputPaths: Seq[FileStatus], |
| 61 | + parsedOptions: CSVOptions): Option[StructType] |
| 62 | + |
| 63 | + /** |
| 64 | + * Generates a header from the given row which is null-safe and duplicate-safe. |
| 65 | + */ |
| 66 | + protected def makeSafeHeader( |
| 67 | + row: Array[String], |
| 68 | + caseSensitive: Boolean, |
| 69 | + options: CSVOptions): Array[String] = { |
| 70 | + if (options.headerFlag) { |
| 71 | + val duplicates = { |
| 72 | + val headerNames = row.filter(_ != null) |
| 73 | + .map(name => if (caseSensitive) name else name.toLowerCase) |
| 74 | + headerNames.diff(headerNames.distinct).distinct |
| 75 | + } |
| 76 | + |
| 77 | + row.zipWithIndex.map { case (value, index) => |
| 78 | + if (value == null || value.isEmpty || value == options.nullValue) { |
| 79 | + // When there are empty strings or the values set in `nullValue`, put the |
| 80 | + // index as the suffix. |
| 81 | + s"_c$index" |
| 82 | + } else if (!caseSensitive && duplicates.contains(value.toLowerCase)) { |
| 83 | + // When there are case-insensitive duplicates, put the index as the suffix. |
| 84 | + s"$value$index" |
| 85 | + } else if (duplicates.contains(value)) { |
| 86 | + // When there are duplicates, put the index as the suffix. |
| 87 | + s"$value$index" |
| 88 | + } else { |
| 89 | + value |
| 90 | + } |
| 91 | + } |
| 92 | + } else { |
| 93 | + row.zipWithIndex.map { case (_, index) => |
| 94 | + // Uses default column names, "_c#" where # is its position of fields |
| 95 | + // when header option is disabled. |
| 96 | + s"_c$index" |
| 97 | + } |
| 98 | + } |
| 99 | + } |
| 100 | +} |
| 101 | + |
| 102 | +object CSVDataSource { |
| 103 | + def apply(options: CSVOptions): CSVDataSource = { |
| 104 | + if (options.wholeFile) { |
| 105 | + WholeFileCSVDataSource |
| 106 | + } else { |
| 107 | + TextInputCSVDataSource |
| 108 | + } |
| 109 | + } |
| 110 | +} |
| 111 | + |
| 112 | +object TextInputCSVDataSource extends CSVDataSource { |
| 113 | + override val isSplitable: Boolean = true |
| 114 | + |
| 115 | + override def readFile( |
| 116 | + conf: Configuration, |
| 117 | + file: PartitionedFile, |
| 118 | + parser: UnivocityParser, |
| 119 | + parsedOptions: CSVOptions): Iterator[InternalRow] = { |
| 120 | + val lines = { |
| 121 | + val linesReader = new HadoopFileLinesReader(file, conf) |
| 122 | + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) |
| 123 | + linesReader.map { line => |
| 124 | + new String(line.getBytes, 0, line.getLength, parsedOptions.charset) |
| 125 | + } |
| 126 | + } |
| 127 | + |
| 128 | + val shouldDropHeader = parsedOptions.headerFlag && file.start == 0 |
| 129 | + UnivocityParser.parseIterator(lines, shouldDropHeader, parser) |
| 130 | + } |
| 131 | + |
| 132 | + override def infer( |
| 133 | + sparkSession: SparkSession, |
| 134 | + inputPaths: Seq[FileStatus], |
| 135 | + parsedOptions: CSVOptions): Option[StructType] = { |
| 136 | + val csv: Dataset[String] = createBaseDataset(sparkSession, inputPaths, parsedOptions) |
| 137 | + val firstLine: String = CSVUtils.filterCommentAndEmpty(csv, parsedOptions).first() |
| 138 | + val firstRow = new CsvParser(parsedOptions.asParserSettings).parseLine(firstLine) |
| 139 | + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis |
| 140 | + val header = makeSafeHeader(firstRow, caseSensitive, parsedOptions) |
| 141 | + val tokenRDD = csv.rdd.mapPartitions { iter => |
| 142 | + val filteredLines = CSVUtils.filterCommentAndEmpty(iter, parsedOptions) |
| 143 | + val linesWithoutHeader = |
| 144 | + CSVUtils.filterHeaderLine(filteredLines, firstLine, parsedOptions) |
| 145 | + val parser = new CsvParser(parsedOptions.asParserSettings) |
| 146 | + linesWithoutHeader.map(parser.parseLine) |
| 147 | + } |
| 148 | + |
| 149 | + Some(CSVInferSchema.infer(tokenRDD, header, parsedOptions)) |
| 150 | + } |
| 151 | + |
| 152 | + private def createBaseDataset( |
| 153 | + sparkSession: SparkSession, |
| 154 | + inputPaths: Seq[FileStatus], |
| 155 | + options: CSVOptions): Dataset[String] = { |
| 156 | + val paths = inputPaths.map(_.getPath.toString) |
| 157 | + if (Charset.forName(options.charset) == StandardCharsets.UTF_8) { |
| 158 | + sparkSession.baseRelationToDataFrame( |
| 159 | + DataSource.apply( |
| 160 | + sparkSession, |
| 161 | + paths = paths, |
| 162 | + className = classOf[TextFileFormat].getName |
| 163 | + ).resolveRelation(checkFilesExist = false)) |
| 164 | + .select("value").as[String](Encoders.STRING) |
| 165 | + } else { |
| 166 | + val charset = options.charset |
| 167 | + val rdd = sparkSession.sparkContext |
| 168 | + .hadoopFile[LongWritable, Text, TextInputFormat](paths.mkString(",")) |
| 169 | + .mapPartitions(_.map(pair => new String(pair._2.getBytes, 0, pair._2.getLength, charset))) |
| 170 | + sparkSession.createDataset(rdd)(Encoders.STRING) |
| 171 | + } |
| 172 | + } |
| 173 | +} |
| 174 | + |
| 175 | +object WholeFileCSVDataSource extends CSVDataSource { |
| 176 | + override val isSplitable: Boolean = false |
| 177 | + |
| 178 | + override def readFile( |
| 179 | + conf: Configuration, |
| 180 | + file: PartitionedFile, |
| 181 | + parser: UnivocityParser, |
| 182 | + parsedOptions: CSVOptions): Iterator[InternalRow] = { |
| 183 | + UnivocityParser.parseStream( |
| 184 | + CodecStreams.createInputStreamWithCloseResource(conf, file.filePath), |
| 185 | + parsedOptions.headerFlag, |
| 186 | + parser) |
| 187 | + } |
| 188 | + |
| 189 | + override def infer( |
| 190 | + sparkSession: SparkSession, |
| 191 | + inputPaths: Seq[FileStatus], |
| 192 | + parsedOptions: CSVOptions): Option[StructType] = { |
| 193 | + val csv: RDD[PortableDataStream] = createBaseRdd(sparkSession, inputPaths, parsedOptions) |
| 194 | + val maybeFirstRow: Option[Array[String]] = csv.flatMap { lines => |
| 195 | + UnivocityParser.tokenizeStream( |
| 196 | + CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, lines.getPath()), |
| 197 | + false, |
| 198 | + new CsvParser(parsedOptions.asParserSettings)) |
| 199 | + }.take(1).headOption |
| 200 | + |
| 201 | + if (maybeFirstRow.isDefined) { |
| 202 | + val firstRow = maybeFirstRow.get |
| 203 | + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis |
| 204 | + val header = makeSafeHeader(firstRow, caseSensitive, parsedOptions) |
| 205 | + val tokenRDD = csv.flatMap { lines => |
| 206 | + UnivocityParser.tokenizeStream( |
| 207 | + CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, lines.getPath()), |
| 208 | + parsedOptions.headerFlag, |
| 209 | + new CsvParser(parsedOptions.asParserSettings)) |
| 210 | + } |
| 211 | + Some(CSVInferSchema.infer(tokenRDD, header, parsedOptions)) |
| 212 | + } else { |
| 213 | + // If the first row could not be read, just return the empty schema. |
| 214 | + Some(StructType(Nil)) |
| 215 | + } |
| 216 | + } |
| 217 | + |
| 218 | + private def createBaseRdd( |
| 219 | + sparkSession: SparkSession, |
| 220 | + inputPaths: Seq[FileStatus], |
| 221 | + options: CSVOptions): RDD[PortableDataStream] = { |
| 222 | + val paths = inputPaths.map(_.getPath) |
| 223 | + val name = paths.mkString(",") |
| 224 | + val job = Job.getInstance(sparkSession.sessionState.newHadoopConf()) |
| 225 | + FileInputFormat.setInputPaths(job, paths: _*) |
| 226 | + val conf = job.getConfiguration |
| 227 | + |
| 228 | + val rdd = new BinaryFileRDD( |
| 229 | + sparkSession.sparkContext, |
| 230 | + classOf[StreamInputFormat], |
| 231 | + classOf[String], |
| 232 | + classOf[PortableDataStream], |
| 233 | + conf, |
| 234 | + sparkSession.sparkContext.defaultMinPartitions) |
| 235 | + |
| 236 | + // Only returns `PortableDataStream`s without paths. |
| 237 | + rdd.setName(s"CSVFile: $name").values |
| 238 | + } |
| 239 | +} |
0 commit comments