Skip to content

Commit 2051428

Browse files
gatorsmilecloud-fan
authored andcommitted
[SPARK-20980][SQL] Rename wholeFile to multiLine for both CSV and JSON
### What changes were proposed in this pull request? The current option name `wholeFile` is misleading for CSV users. Currently, it is not representing a record per file. Actually, one file could have multiple records. Thus, we should rename it. Now, the proposal is `multiLine`. ### How was this patch tested? N/A Author: Xiao Li <gatorsmile@gmail.com> Closes #18202 from gatorsmile/renameCVSOption.
1 parent fffeb6d commit 2051428

File tree

12 files changed

+54
-54
lines changed

12 files changed

+54
-54
lines changed

R/pkg/R/SQLContext.R

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -334,7 +334,7 @@ setMethod("toDF", signature(x = "RDD"),
334334
#'
335335
#' Loads a JSON file, returning the result as a SparkDataFrame
336336
#' By default, (\href{http://jsonlines.org/}{JSON Lines text format or newline-delimited JSON}
337-
#' ) is supported. For JSON (one record per file), set a named property \code{wholeFile} to
337+
#' ) is supported. For JSON (one record per file), set a named property \code{multiLine} to
338338
#' \code{TRUE}.
339339
#' It goes through the entire dataset once to determine the schema.
340340
#'
@@ -348,7 +348,7 @@ setMethod("toDF", signature(x = "RDD"),
348348
#' sparkR.session()
349349
#' path <- "path/to/file.json"
350350
#' df <- read.json(path)
351-
#' df <- read.json(path, wholeFile = TRUE)
351+
#' df <- read.json(path, multiLine = TRUE)
352352
#' df <- jsonFile(path)
353353
#' }
354354
#' @name read.json
@@ -598,7 +598,7 @@ tableToDF <- function(tableName) {
598598
#' df1 <- read.df("path/to/file.json", source = "json")
599599
#' schema <- structType(structField("name", "string"),
600600
#' structField("info", "map<string,double>"))
601-
#' df2 <- read.df(mapTypeJsonPath, "json", schema, wholeFile = TRUE)
601+
#' df2 <- read.df(mapTypeJsonPath, "json", schema, multiLine = TRUE)
602602
#' df3 <- loadDF("data/test_table", "parquet", mergeSchema = "true")
603603
#' }
604604
#' @name read.df

python/pyspark/sql/readwriter.py

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -174,12 +174,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
174174
allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None,
175175
allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None,
176176
mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None,
177-
wholeFile=None):
177+
multiLine=None):
178178
"""
179179
Loads JSON files and returns the results as a :class:`DataFrame`.
180180
181181
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default.
182-
For JSON (one record per file), set the ``wholeFile`` parameter to ``true``.
182+
For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
183183
184184
If the ``schema`` parameter is not specified, this function goes
185185
through the input once to determine the input schema.
@@ -230,7 +230,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
230230
formats follow the formats at ``java.text.SimpleDateFormat``.
231231
This applies to timestamp type. If None is set, it uses the
232232
default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``.
233-
:param wholeFile: parse one record, which may span multiple lines, per file. If None is
233+
:param multiLine: parse one record, which may span multiple lines, per file. If None is
234234
set, it uses the default value, ``false``.
235235
236236
>>> df1 = spark.read.json('python/test_support/sql/people.json')
@@ -248,7 +248,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
248248
allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero,
249249
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
250250
mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat,
251-
timestampFormat=timestampFormat, wholeFile=wholeFile)
251+
timestampFormat=timestampFormat, multiLine=multiLine)
252252
if isinstance(path, basestring):
253253
path = [path]
254254
if type(path) == list:
@@ -322,7 +322,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
322322
ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None,
323323
negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None,
324324
maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None,
325-
columnNameOfCorruptRecord=None, wholeFile=None):
325+
columnNameOfCorruptRecord=None, multiLine=None):
326326
"""Loads a CSV file and returns the result as a :class:`DataFrame`.
327327
328328
This function will go through the input once to determine the input schema if
@@ -396,7 +396,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
396396
``spark.sql.columnNameOfCorruptRecord``. If None is set,
397397
it uses the value specified in
398398
``spark.sql.columnNameOfCorruptRecord``.
399-
:param wholeFile: parse records, which may span multiple lines. If None is
399+
:param multiLine: parse records, which may span multiple lines. If None is
400400
set, it uses the default value, ``false``.
401401
402402
>>> df = spark.read.csv('python/test_support/sql/ages.csv')
@@ -411,7 +411,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
411411
dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns,
412412
maxCharsPerColumn=maxCharsPerColumn,
413413
maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode,
414-
columnNameOfCorruptRecord=columnNameOfCorruptRecord, wholeFile=wholeFile)
414+
columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine)
415415
if isinstance(path, basestring):
416416
path = [path]
417417
return self._df(self._jreader.csv(self._spark._sc._jvm.PythonUtils.toSeq(path)))

python/pyspark/sql/streaming.py

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -401,12 +401,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
401401
allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None,
402402
allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None,
403403
mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None,
404-
wholeFile=None):
404+
multiLine=None):
405405
"""
406406
Loads a JSON file stream and returns the results as a :class:`DataFrame`.
407407
408408
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default.
409-
For JSON (one record per file), set the ``wholeFile`` parameter to ``true``.
409+
For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
410410
411411
If the ``schema`` parameter is not specified, this function goes
412412
through the input once to determine the input schema.
@@ -458,7 +458,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
458458
formats follow the formats at ``java.text.SimpleDateFormat``.
459459
This applies to timestamp type. If None is set, it uses the
460460
default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``.
461-
:param wholeFile: parse one record, which may span multiple lines, per file. If None is
461+
:param multiLine: parse one record, which may span multiple lines, per file. If None is
462462
set, it uses the default value, ``false``.
463463
464464
>>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema)
@@ -473,7 +473,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
473473
allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero,
474474
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
475475
mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat,
476-
timestampFormat=timestampFormat, wholeFile=wholeFile)
476+
timestampFormat=timestampFormat, multiLine=multiLine)
477477
if isinstance(path, basestring):
478478
return self._df(self._jreader.json(path))
479479
else:
@@ -532,7 +532,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
532532
ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None,
533533
negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None,
534534
maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None,
535-
columnNameOfCorruptRecord=None, wholeFile=None):
535+
columnNameOfCorruptRecord=None, multiLine=None):
536536
"""Loads a CSV file stream and returns the result as a :class:`DataFrame`.
537537
538538
This function will go through the input once to determine the input schema if
@@ -607,7 +607,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
607607
``spark.sql.columnNameOfCorruptRecord``. If None is set,
608608
it uses the value specified in
609609
``spark.sql.columnNameOfCorruptRecord``.
610-
:param wholeFile: parse one record, which may span multiple lines. If None is
610+
:param multiLine: parse one record, which may span multiple lines. If None is
611611
set, it uses the default value, ``false``.
612612
613613
>>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema)
@@ -624,7 +624,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
624624
dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns,
625625
maxCharsPerColumn=maxCharsPerColumn,
626626
maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode,
627-
columnNameOfCorruptRecord=columnNameOfCorruptRecord, wholeFile=wholeFile)
627+
columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine)
628628
if isinstance(path, basestring):
629629
return self._df(self._jreader.csv(path))
630630
else:

python/pyspark/sql/tests.py

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -457,15 +457,15 @@ def test_udf_registration_returns_udf(self):
457457
df.select(add_three("id").alias("plus_three")).collect()
458458
)
459459

460-
def test_wholefile_json(self):
460+
def test_multiLine_json(self):
461461
people1 = self.spark.read.json("python/test_support/sql/people.json")
462462
people_array = self.spark.read.json("python/test_support/sql/people_array.json",
463-
wholeFile=True)
463+
multiLine=True)
464464
self.assertEqual(people1.collect(), people_array.collect())
465465

466-
def test_wholefile_csv(self):
466+
def test_multiline_csv(self):
467467
ages_newlines = self.spark.read.csv(
468-
"python/test_support/sql/ages_newlines.csv", wholeFile=True)
468+
"python/test_support/sql/ages_newlines.csv", multiLine=True)
469469
expected = [Row(_c0=u'Joe', _c1=u'20', _c2=u'Hi,\nI am Jeo'),
470470
Row(_c0=u'Tom', _c1=u'30', _c2=u'My name is Tom'),
471471
Row(_c0=u'Hyukjin', _c1=u'25', _c2=u'I am Hyukjin\n\nI love Spark!')]

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,7 @@ private[sql] class JSONOptions(
8181
FastDateFormat.getInstance(
8282
parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), timeZone, Locale.US)
8383

84-
val wholeFile = parameters.get("wholeFile").map(_.toBoolean).getOrElse(false)
84+
val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false)
8585

8686
/** Sets config options on a Jackson [[JsonFactory]]. */
8787
def setJacksonOptions(factory: JsonFactory): Unit = {

sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -295,7 +295,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
295295
* Loads JSON files and returns the results as a `DataFrame`.
296296
*
297297
* <a href="http://jsonlines.org/">JSON Lines</a> (newline-delimited JSON) is supported by
298-
* default. For JSON (one record per file), set the `wholeFile` option to true.
298+
* default. For JSON (one record per file), set the `multiLine` option to true.
299299
*
300300
* This function goes through the input once to determine the input schema. If you know the
301301
* schema in advance, use the version that specifies the schema to avoid the extra scan.
@@ -335,7 +335,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
335335
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that
336336
* indicates a timestamp format. Custom date formats follow the formats at
337337
* `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
338-
* <li>`wholeFile` (default `false`): parse one record, which may span multiple lines,
338+
* <li>`multiLine` (default `false`): parse one record, which may span multiple lines,
339339
* per file</li>
340340
* </ul>
341341
*
@@ -537,7 +537,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
537537
* <li>`columnNameOfCorruptRecord` (default is the value specified in
538538
* `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string
539539
* created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.</li>
540-
* <li>`wholeFile` (default `false`): parse one record, which may span multiple lines.</li>
540+
* <li>`multiLine` (default `false`): parse one record, which may span multiple lines.</li>
541541
* </ul>
542542
* @since 2.0.0
543543
*/

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -111,8 +111,8 @@ abstract class CSVDataSource extends Serializable {
111111

112112
object CSVDataSource {
113113
def apply(options: CSVOptions): CSVDataSource = {
114-
if (options.wholeFile) {
115-
WholeFileCSVDataSource
114+
if (options.multiLine) {
115+
MultiLineCSVDataSource
116116
} else {
117117
TextInputCSVDataSource
118118
}
@@ -197,7 +197,7 @@ object TextInputCSVDataSource extends CSVDataSource {
197197
}
198198
}
199199

200-
object WholeFileCSVDataSource extends CSVDataSource {
200+
object MultiLineCSVDataSource extends CSVDataSource {
201201
override val isSplitable: Boolean = false
202202

203203
override def readFile(

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -128,7 +128,7 @@ class CSVOptions(
128128
FastDateFormat.getInstance(
129129
parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), timeZone, Locale.US)
130130

131-
val wholeFile = parameters.get("wholeFile").map(_.toBoolean).getOrElse(false)
131+
val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false)
132132

133133
val maxColumns = getInt("maxColumns", 20480)
134134

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -86,8 +86,8 @@ abstract class JsonDataSource extends Serializable {
8686

8787
object JsonDataSource {
8888
def apply(options: JSONOptions): JsonDataSource = {
89-
if (options.wholeFile) {
90-
WholeFileJsonDataSource
89+
if (options.multiLine) {
90+
MultiLineJsonDataSource
9191
} else {
9292
TextInputJsonDataSource
9393
}
@@ -147,7 +147,7 @@ object TextInputJsonDataSource extends JsonDataSource {
147147
}
148148
}
149149

150-
object WholeFileJsonDataSource extends JsonDataSource {
150+
object MultiLineJsonDataSource extends JsonDataSource {
151151
override val isSplitable: Boolean = {
152152
false
153153
}

sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -163,7 +163,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
163163
* Loads a JSON file stream and returns the results as a `DataFrame`.
164164
*
165165
* <a href="http://jsonlines.org/">JSON Lines</a> (newline-delimited JSON) is supported by
166-
* default. For JSON (one record per file), set the `wholeFile` option to true.
166+
* default. For JSON (one record per file), set the `multiLine` option to true.
167167
*
168168
* This function goes through the input once to determine the input schema. If you know the
169169
* schema in advance, use the version that specifies the schema to avoid the extra scan.
@@ -205,7 +205,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
205205
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that
206206
* indicates a timestamp format. Custom date formats follow the formats at
207207
* `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
208-
* <li>`wholeFile` (default `false`): parse one record, which may span multiple lines,
208+
* <li>`multiLine` (default `false`): parse one record, which may span multiple lines,
209209
* per file</li>
210210
* </ul>
211211
*
@@ -276,7 +276,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
276276
* <li>`columnNameOfCorruptRecord` (default is the value specified in
277277
* `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string
278278
* created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.</li>
279-
* <li>`wholeFile` (default `false`): parse one record, which may span multiple lines.</li>
279+
* <li>`multiLine` (default `false`): parse one record, which may span multiple lines.</li>
280280
* </ul>
281281
*
282282
* @since 2.0.0

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala

Lines changed: 12 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -261,10 +261,10 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
261261
}
262262

263263
test("test for DROPMALFORMED parsing mode") {
264-
Seq(false, true).foreach { wholeFile =>
264+
Seq(false, true).foreach { multiLine =>
265265
val cars = spark.read
266266
.format("csv")
267-
.option("wholeFile", wholeFile)
267+
.option("multiLine", multiLine)
268268
.options(Map("header" -> "true", "mode" -> "dropmalformed"))
269269
.load(testFile(carsFile))
270270

@@ -284,11 +284,11 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
284284
}
285285

286286
test("test for FAILFAST parsing mode") {
287-
Seq(false, true).foreach { wholeFile =>
287+
Seq(false, true).foreach { multiLine =>
288288
val exception = intercept[SparkException] {
289289
spark.read
290290
.format("csv")
291-
.option("wholeFile", wholeFile)
291+
.option("multiLine", multiLine)
292292
.options(Map("header" -> "true", "mode" -> "failfast"))
293293
.load(testFile(carsFile)).collect()
294294
}
@@ -990,13 +990,13 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
990990
}
991991

992992
test("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") {
993-
Seq(false, true).foreach { wholeFile =>
993+
Seq(false, true).foreach { multiLine =>
994994
val schema = new StructType().add("a", IntegerType).add("b", TimestampType)
995995
// We use `PERMISSIVE` mode by default if invalid string is given.
996996
val df1 = spark
997997
.read
998998
.option("mode", "abcd")
999-
.option("wholeFile", wholeFile)
999+
.option("multiLine", multiLine)
10001000
.schema(schema)
10011001
.csv(testFile(valueMalformedFile))
10021002
checkAnswer(df1,
@@ -1011,7 +1011,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
10111011
.read
10121012
.option("mode", "Permissive")
10131013
.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
1014-
.option("wholeFile", wholeFile)
1014+
.option("multiLine", multiLine)
10151015
.schema(schemaWithCorrField1)
10161016
.csv(testFile(valueMalformedFile))
10171017
checkAnswer(df2,
@@ -1028,7 +1028,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
10281028
.read
10291029
.option("mode", "permissive")
10301030
.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
1031-
.option("wholeFile", wholeFile)
1031+
.option("multiLine", multiLine)
10321032
.schema(schemaWithCorrField2)
10331033
.csv(testFile(valueMalformedFile))
10341034
checkAnswer(df3,
@@ -1041,7 +1041,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
10411041
.read
10421042
.option("mode", "PERMISSIVE")
10431043
.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
1044-
.option("wholeFile", wholeFile)
1044+
.option("multiLine", multiLine)
10451045
.schema(schema.add(columnNameOfCorruptRecord, IntegerType))
10461046
.csv(testFile(valueMalformedFile))
10471047
.collect
@@ -1073,7 +1073,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
10731073

10741074
val df = spark.read
10751075
.option("header", true)
1076-
.option("wholeFile", true)
1076+
.option("multiLine", true)
10771077
.csv(path.getAbsolutePath)
10781078

10791079
// Check if headers have new lines in the names.
@@ -1096,10 +1096,10 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
10961096
}
10971097

10981098
test("Empty file produces empty dataframe with empty schema") {
1099-
Seq(false, true).foreach { wholeFile =>
1099+
Seq(false, true).foreach { multiLine =>
11001100
val df = spark.read.format("csv")
11011101
.option("header", true)
1102-
.option("wholeFile", wholeFile)
1102+
.option("multiLine", multiLine)
11031103
.load(testFile(emptyFile))
11041104

11051105
assert(df.schema === spark.emptyDataFrame.schema)

0 commit comments

Comments
 (0)