-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-16101][SQL] Refactoring CSV schema inference path to be consistent with JSON #16680
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,17 +18,15 @@ | |
| package org.apache.spark.sql.execution.datasources.csv | ||
|
|
||
| import java.math.BigDecimal | ||
| import java.text.NumberFormat | ||
| import java.util.Locale | ||
|
|
||
| import scala.util.control.Exception._ | ||
| import scala.util.Try | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
| import com.univocity.parsers.csv.CsvParser | ||
|
|
||
| import org.apache.spark.sql.catalyst.analysis.TypeCoercion | ||
| import org.apache.spark.sql.catalyst.util.DateTimeUtils | ||
| import org.apache.spark.sql.Dataset | ||
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.unsafe.types.UTF8String | ||
|
|
||
| private[csv] object CSVInferSchema { | ||
|
|
||
|
|
@@ -39,22 +37,76 @@ private[csv] object CSVInferSchema { | |
| * 3. Replace any null types with string type | ||
| */ | ||
| def infer( | ||
| tokenRdd: RDD[Array[String]], | ||
| header: Array[String], | ||
| csv: Dataset[String], | ||
|
||
| caseSensitive: Boolean, | ||
| options: CSVOptions): StructType = { | ||
| val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) | ||
| val rootTypes: Array[DataType] = | ||
| tokenRdd.aggregate(startType)(inferRowType(options), mergeRowTypes) | ||
|
|
||
| val structFields = header.zip(rootTypes).map { case (thisHeader, rootType) => | ||
| val dType = rootType match { | ||
| case _: NullType => StringType | ||
| case other => other | ||
| val firstLine: String = CSVUtils.filterCommentAndEmpty(csv, options).first() | ||
|
||
| val firstRow = new CsvParser(options.asParserSettings).parseLine(firstLine) | ||
| val header = makeSafeHeader(firstRow, caseSensitive, options) | ||
|
|
||
| val fields = if (options.inferSchemaFlag) { | ||
| val tokenRdd = csv.rdd.mapPartitions { iter => | ||
| val filteredLines = CSVUtils.filterCommentAndEmpty(iter, options) | ||
| val linesWithoutHeader = CSVUtils.filterHeaderLine(filteredLines, firstLine, options) | ||
| val parser = new CsvParser(options.asParserSettings) | ||
| linesWithoutHeader.map(parser.parseLine) | ||
| } | ||
|
|
||
| val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) | ||
| val rootTypes: Array[DataType] = | ||
| tokenRdd.aggregate(startType)(inferRowType(options), mergeRowTypes) | ||
|
|
||
| header.zip(rootTypes).map { case (thisHeader, rootType) => | ||
| val dType = rootType match { | ||
| case _: NullType => StringType | ||
| case other => other | ||
| } | ||
| StructField(thisHeader, dType, nullable = true) | ||
| } | ||
| StructField(thisHeader, dType, nullable = true) | ||
| } else { | ||
| // By default fields are assumed to be StringType | ||
| header.map(fieldName => StructField(fieldName, StringType, nullable = true)) | ||
| } | ||
|
|
||
| StructType(structFields) | ||
| StructType(fields) | ||
| } | ||
|
|
||
| /** | ||
| * Generates a header from the given row which is null-safe and duplicate-safe. | ||
| */ | ||
| private def makeSafeHeader( | ||
| row: Array[String], | ||
| caseSensitive: Boolean, | ||
| options: CSVOptions): Array[String] = { | ||
| if (options.headerFlag) { | ||
| val duplicates = { | ||
| val headerNames = row.filter(_ != null) | ||
| .map(name => if (caseSensitive) name else name.toLowerCase) | ||
| headerNames.diff(headerNames.distinct).distinct | ||
| } | ||
|
|
||
| row.zipWithIndex.map { case (value, index) => | ||
| if (value == null || value.isEmpty || value == options.nullValue) { | ||
| // When there are empty strings or the values set in `nullValue`, put the | ||
| // index as the suffix. | ||
| s"_c$index" | ||
| } else if (!caseSensitive && duplicates.contains(value.toLowerCase)) { | ||
| // When there are case-insensitive duplicates, put the index as the suffix. | ||
| s"$value$index" | ||
| } else if (duplicates.contains(value)) { | ||
| // When there are duplicates, put the index as the suffix. | ||
| s"$value$index" | ||
| } else { | ||
| value | ||
| } | ||
| } | ||
| } else { | ||
| row.zipWithIndex.map { case (_, index) => | ||
| // Uses default column names, "_c#" where # is its position of fields | ||
| // when header option is disabled. | ||
| s"_c$index" | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private def inferRowType(options: CSVOptions) | ||
|
|
@@ -215,32 +267,3 @@ private[csv] object CSVInferSchema { | |
| case _ => None | ||
| } | ||
| } | ||
|
|
||
| private[csv] object CSVTypeCast { | ||
| /** | ||
| * Helper method that converts string representation of a character to actual character. | ||
| * It handles some Java escaped strings and throws exception if given string is longer than one | ||
| * character. | ||
| */ | ||
| @throws[IllegalArgumentException] | ||
| def toChar(str: String): Char = { | ||
| if (str.charAt(0) == '\\') { | ||
| str.charAt(1) | ||
| match { | ||
| case 't' => '\t' | ||
| case 'r' => '\r' | ||
| case 'b' => '\b' | ||
| case 'f' => '\f' | ||
| case '\"' => '\"' // In case user changes quote char and uses \" as delimiter in options | ||
| case '\'' => '\'' | ||
| case 'u' if str == """\u0000""" => '\u0000' | ||
| case _ => | ||
| throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") | ||
| } | ||
| } else if (str.length == 1) { | ||
| str.charAt(0) | ||
| } else { | ||
| throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These removed block is all into
CSVInferSchema.infer(...).