Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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
6 changes: 4 additions & 2 deletions python/pyspark/sql/readwriter.py
Original file line number Diff line number Diff line change
Expand Up @@ -661,7 +661,7 @@ def text(self, path, compression=None):
@since(2.0)
def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None,
header=None, nullValue=None, escapeQuotes=None, quoteAll=None, dateFormat=None,
timestampFormat=None):
timestampFormat=None, encoding=None):
"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.

:param path: the path in any Hadoop supported file system
Expand Down Expand Up @@ -701,13 +701,15 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No
formats follow the formats at ``java.text.SimpleDateFormat``.
This applies to timestamp type. If None is set, it uses the
default value value, ``yyyy-MM-dd'T'HH:mm:ss.SSSZZ``.
:param encoding: encodes the CSV files by the given encoding type. If None is set,
it uses the default value, ``UTF-8``.

>>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data'))
"""
self.mode(mode)
self._set_opts(compression=compression, sep=sep, quote=quote, escape=escape, header=header,
nullValue=nullValue, escapeQuotes=escapeQuotes, quoteAll=quoteAll,
dateFormat=dateFormat, timestampFormat=timestampFormat)
dateFormat=dateFormat, timestampFormat=timestampFormat, encoding=encoding)
self._jwrite.csv(path)

@since(1.5)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -572,6 +572,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that
* indicates a timestamp format. Custom date formats follow the formats at
* `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
* <li>`encoding` (default `UTF-8`): encodes the CSV files by the given encoding
* type.</li>
* </ul>
*
* @since 2.0.0
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@

package org.apache.spark.sql.execution.datasources.csv

import java.io.{CharArrayWriter, OutputStream, StringReader}
import java.nio.charset.StandardCharsets
import java.io.OutputStream
import java.nio.charset.Charset

import com.univocity.parsers.csv._

Expand Down Expand Up @@ -71,6 +71,7 @@ private[csv] class LineCsvWriter(
output: OutputStream) extends Logging {
private val writerSettings = new CsvWriterSettings
private val format = writerSettings.getFormat
private val writerCharset = Charset.forName(params.charset)

format.setDelimiter(params.delimiter)
format.setQuote(params.quote)
Expand All @@ -84,7 +85,7 @@ private[csv] class LineCsvWriter(
writerSettings.setHeaders(headers: _*)
writerSettings.setQuoteEscapingEnabled(params.escapeQuotes)

private val writer = new CsvWriter(output, StandardCharsets.UTF_8, writerSettings)
private val writer = new CsvWriter(output, writerCharset, writerSettings)

def writeRow(row: Seq[String], includeHeader: Boolean): Unit = {
if (includeHeader) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -905,4 +905,23 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
checkAnswer(df, Row(1, null))
}
}

test("save data with gb18030") {
withTempPath { path =>
// scalastyle:off
val df = Seq(("1", "中文")).toDF("num", "lanaguage")
// scalastyle:on
df.write
.option("header", "true")
.option("encoding", "GB18030")
.csv(path.getAbsolutePath)

val readBack = spark.read
.option("header", "true")
.option("encoding", "GB18030")
.csv(path.getAbsolutePath)

checkAnswer(df, readBack)
}
}
}