Skip to content

Commit

Permalink
Some characters are garbled when opening csv files with Excel
Browse files Browse the repository at this point in the history
  • Loading branch information
wangyum committed Jul 13, 2020
1 parent 6d49964 commit bfab2a5
Show file tree
Hide file tree
Showing 3 changed files with 47 additions and 0 deletions.
Expand Up @@ -135,6 +135,8 @@ class CSVOptions(
val positiveInf = parameters.getOrElse("positiveInf", "Inf")
val negativeInf = parameters.getOrElse("negativeInf", "-Inf")

// Set bom to true to fix some characters are garbled when opening with Excel.
val bom = getBool("bom")

val compressionCodec: Option[String] = {
val name = parameters.get("compression").orElse(parameters.get("codec"))
Expand Down
Expand Up @@ -39,6 +39,10 @@ class CsvOutputWriter(

private val gen = new UnivocityGenerator(dataSchema, writer, params)

if (params.bom) {
writer.write(0xFEFF)
}

if (params.headerFlag) {
gen.writeHeaders()
}
Expand Down
Expand Up @@ -26,6 +26,7 @@ import java.util.Locale
import java.util.zip.GZIPOutputStream

import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.util.Properties

import com.univocity.parsers.common.TextParsingException
Expand All @@ -34,6 +35,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType
import org.apache.hadoop.io.compress.GzipCodec

import org.apache.spark.{SparkConf, SparkException, TestUtils}
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd}
import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -2353,6 +2355,45 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa
assert(df.schema.last == StructField("col_mixed_types", StringType, true))
}
}

test("Some characters are garbled when opening csv files with Excel") {
// scalastyle:off nonascii
val chinese = "我爱中文"
val korean = "나는 한국인을 좋아한다"
val japanese = "私は日本人が好き"
// scalastyle:on nonascii
val english = "I love English"

val df = spark.sql(s"SELECT '$chinese' AS Chinese, '$korean' AS Korean," +
s"'$japanese' AS Japanese, '$english' AS English")

Seq(true, false).foreach { bom =>
withTempPath { p =>
val path = p.getAbsolutePath
df.write.option("bom", bom).csv(path)

val bytesReads = new mutable.ArrayBuffer[Long]()
val bytesReadListener = new SparkListener() {
override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead
}
}
sparkContext.addSparkListener(bytesReadListener)
try {
spark.read.csv(path).limit(1).collect()
sparkContext.listenerBus.waitUntilEmpty(1000L)
if (bom) {
assert(bytesReads.sum === 202)
} else {
assert(bytesReads.sum === 196)
}
} finally {
sparkContext.removeSparkListener(bytesReadListener)
}
checkAnswer(spark.read.csv(path), Seq(Row(chinese, korean, japanese, english)))
}
}
}
}

class CSVv1Suite extends CSVSuite {
Expand Down

0 comments on commit bfab2a5

Please sign in to comment.