Skip to content

Commit f70994c

Browse files
committed
fix.
1 parent 1fae66a commit f70994c

9 files changed

Lines changed: 26 additions & 26 deletions

File tree

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: 4 additions & 4 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:

python/pyspark/sql/streaming.py

Lines changed: 4 additions & 4 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:

python/pyspark/sql/tests.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -457,10 +457,10 @@ 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

466466
def test_multiline_csv(self):

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: 2 additions & 2 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
*

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -86,7 +86,7 @@ abstract class JsonDataSource extends Serializable {
8686

8787
object JsonDataSource {
8888
def apply(options: JSONOptions): JsonDataSource = {
89-
if (options.wholeFile) {
89+
if (options.multiLine) {
9090
WholeFileJsonDataSource
9191
} else {
9292
TextInputJsonDataSource

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

Lines changed: 2 additions & 2 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
*

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -1814,7 +1814,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
18141814

18151815
assert(new File(path).listFiles().exists(_.getName.endsWith(".gz")))
18161816

1817-
val jsonDF = spark.read.option("wholeFile", true).json(path)
1817+
val jsonDF = spark.read.option("multiLine", true).json(path)
18181818
val jsonDir = new File(dir, "json").getCanonicalPath
18191819
jsonDF.coalesce(1).write
18201820
.option("compression", "gZiP")
@@ -1836,7 +1836,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
18361836
.write
18371837
.text(path)
18381838

1839-
val jsonDF = spark.read.option("wholeFile", true).json(path)
1839+
val jsonDF = spark.read.option("multiLine", true).json(path)
18401840
val jsonDir = new File(dir, "json").getCanonicalPath
18411841
jsonDF.coalesce(1).write.json(jsonDir)
18421842

@@ -1865,7 +1865,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
18651865
.write
18661866
.text(path)
18671867

1868-
val jsonDF = spark.read.option("wholeFile", true).json(path)
1868+
val jsonDF = spark.read.option("multiLine", true).json(path)
18691869
// no corrupt record column should be created
18701870
assert(jsonDF.schema === StructType(Seq()))
18711871
// only the first object should be read
@@ -1886,7 +1886,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
18861886
.write
18871887
.text(path)
18881888

1889-
val jsonDF = spark.read.option("wholeFile", true).option("mode", "PERMISSIVE").json(path)
1889+
val jsonDF = spark.read.option("multiLine", true).option("mode", "PERMISSIVE").json(path)
18901890
assert(jsonDF.count() === corruptRecordCount)
18911891
assert(jsonDF.schema === new StructType()
18921892
.add("_corrupt_record", StringType)
@@ -1917,7 +1917,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
19171917
.write
19181918
.text(path)
19191919

1920-
val jsonDF = spark.read.option("wholeFile", true).option("mode", "DROPMALFORMED").json(path)
1920+
val jsonDF = spark.read.option("multiLine", true).option("mode", "DROPMALFORMED").json(path)
19211921
checkAnswer(jsonDF, Seq(Row("test")))
19221922
}
19231923
}
@@ -1940,15 +1940,15 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
19401940
// `FAILFAST` mode should throw an exception for corrupt records.
19411941
val exceptionOne = intercept[SparkException] {
19421942
spark.read
1943-
.option("wholeFile", true)
1943+
.option("multiLine", true)
19441944
.option("mode", "FAILFAST")
19451945
.json(path)
19461946
}
19471947
assert(exceptionOne.getMessage.contains("Failed to infer a common schema"))
19481948

19491949
val exceptionTwo = intercept[SparkException] {
19501950
spark.read
1951-
.option("wholeFile", true)
1951+
.option("multiLine", true)
19521952
.option("mode", "FAILFAST")
19531953
.schema(schema)
19541954
.json(path)

0 commit comments

Comments
 (0)