Skip to content

Commit

Permalink
fix.
Browse files Browse the repository at this point in the history
  • Loading branch information
gatorsmile committed Jun 5, 2017
1 parent 1fae66a commit f70994c
Show file tree
Hide file tree
Showing 9 changed files with 26 additions and 26 deletions.
6 changes: 3 additions & 3 deletions R/pkg/R/SQLContext.R
Original file line number Diff line number Diff line change
Expand Up @@ -334,7 +334,7 @@ setMethod("toDF", signature(x = "RDD"),
#'
#' Loads a JSON file, returning the result as a SparkDataFrame
#' By default, (\href{http://jsonlines.org/}{JSON Lines text format or newline-delimited JSON}
#' ) is supported. For JSON (one record per file), set a named property \code{wholeFile} to
#' ) is supported. For JSON (one record per file), set a named property \code{multiLine} to
#' \code{TRUE}.
#' It goes through the entire dataset once to determine the schema.
#'
Expand All @@ -348,7 +348,7 @@ setMethod("toDF", signature(x = "RDD"),
#' sparkR.session()
#' path <- "path/to/file.json"
#' df <- read.json(path)
#' df <- read.json(path, wholeFile = TRUE)
#' df <- read.json(path, multiLine = TRUE)
#' df <- jsonFile(path)
#' }
#' @name read.json
Expand Down Expand Up @@ -598,7 +598,7 @@ tableToDF <- function(tableName) {
#' df1 <- read.df("path/to/file.json", source = "json")
#' schema <- structType(structField("name", "string"),
#' structField("info", "map<string,double>"))
#' df2 <- read.df(mapTypeJsonPath, "json", schema, wholeFile = TRUE)
#' df2 <- read.df(mapTypeJsonPath, "json", schema, multiLine = TRUE)
#' df3 <- loadDF("data/test_table", "parquet", mergeSchema = "true")
#' }
#' @name read.df
Expand Down
8 changes: 4 additions & 4 deletions python/pyspark/sql/readwriter.py
Original file line number Diff line number Diff line change
Expand Up @@ -174,12 +174,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None,
allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None,
mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None,
wholeFile=None):
multiLine=None):
"""
Loads JSON files and returns the results as a :class:`DataFrame`.
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default.
For JSON (one record per file), set the ``wholeFile`` parameter to ``true``.
For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
If the ``schema`` parameter is not specified, this function goes
through the input once to determine the input schema.
Expand Down Expand Up @@ -230,7 +230,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
formats follow the formats at ``java.text.SimpleDateFormat``.
This applies to timestamp type. If None is set, it uses the
default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``.
:param wholeFile: parse one record, which may span multiple lines, per file. If None is
:param multiLine: parse one record, which may span multiple lines, per file. If None is
set, it uses the default value, ``false``.
>>> df1 = spark.read.json('python/test_support/sql/people.json')
Expand All @@ -248,7 +248,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero,
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat,
timestampFormat=timestampFormat, wholeFile=wholeFile)
timestampFormat=timestampFormat, multiLine=multiLine)
if isinstance(path, basestring):
path = [path]
if type(path) == list:
Expand Down
8 changes: 4 additions & 4 deletions python/pyspark/sql/streaming.py
Original file line number Diff line number Diff line change
Expand Up @@ -401,12 +401,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None,
allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None,
mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None,
wholeFile=None):
multiLine=None):
"""
Loads a JSON file stream and returns the results as a :class:`DataFrame`.
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default.
For JSON (one record per file), set the ``wholeFile`` parameter to ``true``.
For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
If the ``schema`` parameter is not specified, this function goes
through the input once to determine the input schema.
Expand Down Expand Up @@ -458,7 +458,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
formats follow the formats at ``java.text.SimpleDateFormat``.
This applies to timestamp type. If None is set, it uses the
default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``.
:param wholeFile: parse one record, which may span multiple lines, per file. If None is
:param multiLine: parse one record, which may span multiple lines, per file. If None is
set, it uses the default value, ``false``.
>>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema)
Expand All @@ -473,7 +473,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero,
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat,
timestampFormat=timestampFormat, wholeFile=wholeFile)
timestampFormat=timestampFormat, multiLine=multiLine)
if isinstance(path, basestring):
return self._df(self._jreader.json(path))
else:
Expand Down
4 changes: 2 additions & 2 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -457,10 +457,10 @@ def test_udf_registration_returns_udf(self):
df.select(add_three("id").alias("plus_three")).collect()
)

def test_wholefile_json(self):
def test_multiLine_json(self):
people1 = self.spark.read.json("python/test_support/sql/people.json")
people_array = self.spark.read.json("python/test_support/sql/people_array.json",
wholeFile=True)
multiLine=True)
self.assertEqual(people1.collect(), people_array.collect())

def test_multiline_csv(self):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ private[sql] class JSONOptions(
FastDateFormat.getInstance(
parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), timeZone, Locale.US)

val wholeFile = parameters.get("wholeFile").map(_.toBoolean).getOrElse(false)
val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false)

/** Sets config options on a Jackson [[JsonFactory]]. */
def setJacksonOptions(factory: JsonFactory): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -295,7 +295,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
* Loads JSON files and returns the results as a `DataFrame`.
*
* <a href="http://jsonlines.org/">JSON Lines</a> (newline-delimited JSON) is supported by
* default. For JSON (one record per file), set the `wholeFile` option to true.
* default. For JSON (one record per file), set the `multiLine` option to true.
*
* This function goes through the input once to determine the input schema. If you know the
* schema in advance, use the version that specifies the schema to avoid the extra scan.
Expand Down Expand Up @@ -335,7 +335,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): 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>`wholeFile` (default `false`): parse one record, which may span multiple lines,
* <li>`multiLine` (default `false`): parse one record, which may span multiple lines,
* per file</li>
* </ul>
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ abstract class JsonDataSource extends Serializable {

object JsonDataSource {
def apply(options: JSONOptions): JsonDataSource = {
if (options.wholeFile) {
if (options.multiLine) {
WholeFileJsonDataSource
} else {
TextInputJsonDataSource
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
* Loads a JSON file stream and returns the results as a `DataFrame`.
*
* <a href="http://jsonlines.org/">JSON Lines</a> (newline-delimited JSON) is supported by
* default. For JSON (one record per file), set the `wholeFile` option to true.
* default. For JSON (one record per file), set the `multiLine` option to true.
*
* This function goes through the input once to determine the input schema. If you know the
* schema in advance, use the version that specifies the schema to avoid the extra scan.
Expand Down Expand Up @@ -205,7 +205,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): 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>`wholeFile` (default `false`): parse one record, which may span multiple lines,
* <li>`multiLine` (default `false`): parse one record, which may span multiple lines,
* per file</li>
* </ul>
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1814,7 +1814,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {

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

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

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

Expand Down Expand Up @@ -1865,7 +1865,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
.write
.text(path)

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

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

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

val exceptionTwo = intercept[SparkException] {
spark.read
.option("wholeFile", true)
.option("multiLine", true)
.option("mode", "FAILFAST")
.schema(schema)
.json(path)
Expand Down

0 comments on commit f70994c

Please sign in to comment.