Skip to content

Commit 1fae66a

Browse files
committed
rename.
1 parent c70c38e commit 1fae66a

File tree

8 files changed

+24
-24
lines changed

8 files changed

+24
-24
lines changed

python/pyspark/sql/readwriter.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -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: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -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: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -463,9 +463,9 @@ def test_wholefile_json(self):
463463
wholeFile=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/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -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: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -111,7 +111,7 @@ abstract class CSVDataSource extends Serializable {
111111

112112
object CSVDataSource {
113113
def apply(options: CSVOptions): CSVDataSource = {
114-
if (options.wholeFile) {
114+
if (options.multiLine) {
115115
WholeFileCSVDataSource
116116
} else {
117117
TextInputCSVDataSource

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/streaming/DataStreamReader.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -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)