Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

Commit af4f89c

Browse files
gatorsmilecloud-fan
authored andcommitted
[SPARK-20980][SQL] Rename wholeFile to multiLine for both CSV and JSON
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`. N/A Author: Xiao Li <[email protected]> Closes apache#18202 from gatorsmile/renameCVSOption. (cherry picked from commit 2051428) Signed-off-by: Wenchen Fan <[email protected]>
1 parent e02e063 commit af4f89c

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
@@ -169,12 +169,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
169169
allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None,
170170
allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None,
171171
mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None,
172-
wholeFile=None):
172+
multiLine=None):
173173
"""
174174
Loads JSON files and returns the results as a :class:`DataFrame`.
175175
176176
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default.
177-
For JSON (one record per file), set the ``wholeFile`` parameter to ``true``.
177+
For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
178178
179179
If the ``schema`` parameter is not specified, this function goes
180180
through the input once to determine the input schema.
@@ -224,7 +224,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
224224
formats follow the formats at ``java.text.SimpleDateFormat``.
225225
This applies to timestamp type. If None is set, it uses the
226226
default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``.
227-
:param wholeFile: parse one record, which may span multiple lines, per file. If None is
227+
:param multiLine: parse one record, which may span multiple lines, per file. If None is
228228
set, it uses the default value, ``false``.
229229
230230
>>> df1 = spark.read.json('python/test_support/sql/people.json')
@@ -242,7 +242,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
242242
allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero,
243243
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
244244
mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat,
245-
timestampFormat=timestampFormat, wholeFile=wholeFile)
245+
timestampFormat=timestampFormat, multiLine=multiLine)
246246
if isinstance(path, basestring):
247247
path = [path]
248248
if type(path) == list:
@@ -316,7 +316,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
316316
ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None,
317317
negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None,
318318
maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None,
319-
columnNameOfCorruptRecord=None, wholeFile=None):
319+
columnNameOfCorruptRecord=None, multiLine=None):
320320
"""Loads a CSV file and returns the result as a :class:`DataFrame`.
321321
322322
This function will go through the input once to determine the input schema if
@@ -389,7 +389,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
389389
``spark.sql.columnNameOfCorruptRecord``. If None is set,
390390
it uses the value specified in
391391
``spark.sql.columnNameOfCorruptRecord``.
392-
:param wholeFile: parse records, which may span multiple lines. If None is
392+
:param multiLine: parse records, which may span multiple lines. If None is
393393
set, it uses the default value, ``false``.
394394
395395
>>> df = spark.read.csv('python/test_support/sql/ages.csv')
@@ -404,7 +404,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non
404404
dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns,
405405
maxCharsPerColumn=maxCharsPerColumn,
406406
maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode,
407-
columnNameOfCorruptRecord=columnNameOfCorruptRecord, wholeFile=wholeFile)
407+
columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine)
408408
if isinstance(path, basestring):
409409
path = [path]
410410
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
@@ -442,15 +442,15 @@ def test_udf_with_order_by_and_limit(self):
442442
res.explain(True)
443443
self.assertEqual(res.collect(), [Row(id=0, copy=0)])
444444

445-
def test_wholefile_json(self):
445+
def test_multiLine_json(self):
446446
people1 = self.spark.read.json("python/test_support/sql/people.json")
447447
people_array = self.spark.read.json("python/test_support/sql/people_array.json",
448-
wholeFile=True)
448+
multiLine=True)
449449
self.assertEqual(people1.collect(), people_array.collect())
450450

451-
def test_wholefile_csv(self):
451+
def test_multiLine_csv(self):
452452
ages_newlines = self.spark.read.csv(
453-
"python/test_support/sql/ages_newlines.csv", wholeFile=True)
453+
"python/test_support/sql/ages_newlines.csv", multiLine=True)
454454
expected = [Row(_c0=u'Joe', _c1=u'20', _c2=u'Hi,\nI am Jeo'),
455455
Row(_c0=u'Tom', _c1=u'30', _c2=u'My name is Tom'),
456456
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
@@ -283,7 +283,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
283283
* Loads JSON files and returns the results as a `DataFrame`.
284284
*
285285
* <a href="http://jsonlines.org/">JSON Lines</a> (newline-delimited JSON) is supported by
286-
* default. For JSON (one record per file), set the `wholeFile` option to true.
286+
* default. For JSON (one record per file), set the `multiLine` option to true.
287287
*
288288
* This function goes through the input once to determine the input schema. If you know the
289289
* schema in advance, use the version that specifies the schema to avoid the extra scan.
@@ -323,7 +323,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
323323
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that
324324
* indicates a timestamp format. Custom date formats follow the formats at
325325
* `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
326-
* <li>`wholeFile` (default `false`): parse one record, which may span multiple lines,
326+
* <li>`multiLine` (default `false`): parse one record, which may span multiple lines,
327327
* per file</li>
328328
* </ul>
329329
*
@@ -525,7 +525,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
525525
* <li>`columnNameOfCorruptRecord` (default is the value specified in
526526
* `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string
527527
* created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.</li>
528-
* <li>`wholeFile` (default `false`): parse one record, which may span multiple lines.</li>
528+
* <li>`multiLine` (default `false`): parse one record, which may span multiple lines.</li>
529529
* </ul>
530530
* @since 2.0.0
531531
*/

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
}
@@ -196,7 +196,7 @@ object TextInputCSVDataSource extends CSVDataSource {
196196
}
197197
}
198198

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

202202
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

0 commit comments

Comments
 (0)