Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Next Next commit
Adding emptyValue option to will allow user to speify the mapping for…
… empty string on read/write
  • Loading branch information
sureshthalamati committed Oct 20, 2016
commit b128fbb92792ecceb67f5fec3c1feb1ae7fcff69
Original file line number Diff line number Diff line change
Expand Up @@ -280,7 +280,8 @@ private[csv] object CSVTypeCast {
// compatibility.
DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime)
}
case _: StringType => UTF8String.fromString(datum)
case _: StringType =>
if (datum == options.emptyValue) UTF8String.EMPTY_UTF8 else UTF8String.fromString(datum)
case udt: UserDefinedType[_] => castTo(datum, udt.sqlType, nullable, options)
case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ private[csv] class CSVOptions(@transient private val parameters: Map[String, Str
val permissive = ParseModes.isPermissiveMode(parseMode)

val nullValue = parameters.getOrElse("nullValue", "")
val emptyAsNull = getBool("emptyAsNull", true)
val emptyValue = parameters.getOrElse("emptyValue", "")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When nullValue and emptyValue are both "" in default, don't they conflict?

Copy link
Member

@HyukjinKwon HyukjinKwon Oct 20, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 for documenting the explicit precedence

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, null and empty can not be differentiated when they are set to same value. Currently null value check has higher precedence than the empty value.

input.csv
1,
2,””

Output will be:
1, null
2, null

I think this behavior is ok. By default Univocity CSV parser used in spark also returns null for empty strings.

I agree we should document this behavior.


val nanValue = parameters.getOrElse("nanValue", "NaN")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ private[csv] class CsvReader(params: CSVOptions) {
settings.setInputBufferSize(params.inputBufferSize)
settings.setMaxColumns(params.maxColumns)
settings.setNullValue(params.nullValue)
settings.setEmptyValue(if (params.emptyAsNull) null else "")
settings.setEmptyValue(params.emptyValue)
settings.setMaxCharsPerColumn(params.maxCharsPerColumn)
settings.setUnescapedQuoteHandling(UnescapedQuoteHandling.STOP_AT_DELIMITER)

Expand Down Expand Up @@ -75,7 +75,7 @@ private[csv] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten
format.setComment(params.comment)

writerSettings.setNullValue(params.nullValue)
writerSettings.setEmptyValue(if (params.emptyAsNull) params.nullValue else "")
writerSettings.setEmptyValue(params.emptyValue)
writerSettings.setSkipEmptyLines(true)
writerSettings.setQuoteAllFields(params.quoteAll)
writerSettings.setHeaders(headers: _*)
Expand Down
4 changes: 0 additions & 4 deletions sql/core/src/test/resources/emptystring-values.csv

This file was deleted.

5 changes: 5 additions & 0 deletions sql/core/src/test/resources/test-data/emptystring.csv
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
col1,col2
1,"-"
2,""
3,
4,"A"
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
private val numbersFile = "test-data/numbers.csv"
private val datesFile = "test-data/dates.csv"
private val unescapedQuotesFile = "test-data/unescaped-quotes.csv"
private val emptyStringValuesFile = "test-data/emptystring-values.csv"
private val emptyStringValuesFile = "test-data/emptystring.csv"

private def testFile(fileName: String): String = {
Thread.currentThread().getContextClassLoader.getResource(fileName).toString
Expand Down Expand Up @@ -891,77 +891,66 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
}
}

test("load data with empty quoted string fields.") {
val results = sqlContext
.read
.format("csv")
.options(Map(
"header" -> "true",
"nullValue" -> null,
"inferSchema" -> "true"))
.load(testFile(emptyStringValuesFile))
.collect()

assert(results(0).toSeq === Array(2017, "Tesla", "Mode 3", "looks nice.", 35000.99d))
assert(results(1).toSeq === Array(2016, "Chevy", "Bolt", null, 29000.00d))
assert(results(2).toSeq === Array(2015, "Porsche", null, null, null))
}

test("load data with empty quoted string fields using user specified empty value option.") {
val results = sqlContext
.read
test("SPARK-17916 load data with empty strings and null values") {
val rows = spark.read
.format("csv")
.options(Map(
"header" -> "true",
"inferSchema" -> "true",
"nullValue" -> null,
"emptyAsNull" -> "false"))
.option("header", "true")
.option("inferSchema", "true")
.option("nullValue", "-")
.load(testFile(emptyStringValuesFile))
.collect()

assert(results(0).toSeq === Array(2017, "Tesla", "Mode 3", "looks nice.", 35000.99d))
assert(results(1).toSeq === Array(2016, "Chevy", "Bolt", "", 29000.00d))
assert(results(2).toSeq === Array(2015, "Porsche", "", null, null))
val expectedRows = Seq(Row(1, null), Row(2, ""), Row(3, null), Row(4, "A"))
checkAnswer(rows, expectedRows)
}

test("save empty string fields.") {
test("save and load data with empty strings using all quotes option") {
withTempDir { dir =>
val csvDir = new File(dir, "csv").getCanonicalPath
val data = Seq((2015, "Tesla", "Model X", "", 129000.00d))
val df = data.toDF("year", "make", "model", "comment", "price")
df.coalesce(1).write.format("csv").save(csvDir)
val results = sqlContext
.read
val data = Seq((1, "", "abcd"), (2, null, "xyz"))
val df = spark.createDataFrame(data)
df.coalesce(1).write
.format("csv")
.schema(df.schema)
.option("quoteAll", "true")
.option("nullValue", "NUL")
.save(csvDir)
val results = spark.read.format("text").load(csvDir).collect()
val expected = Seq(Seq("\"1\",\"\",\"abcd\""), Seq("\"2\",\"NUL\",\"xyz\""))
assert(results.toSeq.map(_.toSeq) === expected)

val rows = spark.read
.format("csv")
.option("nullValue", "NUL")
.option("inferSchema", "true")
.load(csvDir)
.collect()

assert(results(0).toSeq === Array(2015, "Tesla", "Model X", "", 129000.00d))
val expectedRows = Seq(Row(1, "", "abcd"), Row(2, null, "xyz"))
checkAnswer(rows, expectedRows)
}
}

test("save empty string fields with user specified emptyValue option.") {
test("save and load data using the user specified emptyValue option") {
withTempDir { dir =>
val csvDir = new File(dir, "csv").getCanonicalPath
val data = Seq((2015, "Tesla", "Model X", "", 129000.00d))
val df = data.toDF("year", "make", "model", "comment", "price")
df.coalesce(1)
.write.format("csv")
.option("nullValue", "\\N")
.option("emptyAsNull", "false")
val data = Seq((1, "", "abcd"), (2, null, "xyz"))
val df = spark.createDataFrame(data)
df.coalesce(1).write
.format("csv")
.option("nullValue", "NUL")
.option("emptyValue", "<EMPTY>")
.save(csvDir)
val results = sqlContext
.read
val results = spark.read.format("text").load(csvDir).collect()
val expected = Seq(Seq("1,<EMPTY>,abcd"), Seq("2,NUL,xyz"))
assert(results.toSeq.map(_.toSeq) === expected)

val rows = spark.read
.format("csv")
.schema(df.schema)
.option("nullValue", "\\N")
.option("emptyAsNull", "false")
.option("nullValue", "NUL")
.option("emptyValue", "<EMPTY>")
.option("inferSchema", "true")
.load(csvDir)
.collect()
// CSV writer does not seem to distinguish between empty string value vs null value,
// null value (\\N) gets written in both the cases.
assert(results(0).toSeq === Array(2015, "Tesla", "Model X", "\\N", 129000.00d))

val expectedRows = Seq(Row(1, "", "abcd"), Row(2, null, "xyz"))
checkAnswer(rows, expectedRows)
}
}
}