[SPARK-9964] [PYSPARK] [SQL] PySpark DataFrameReader accept RDD of String for JSON

PySpark DataFrameReader should could accept an RDD of Strings (like the Scala version does) for JSON, rather than only taking a path.
If this PR is merged, it should be duplicated to cover the other input types (not just JSON).

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #8444 from yanboliang/spark-9964.
This commit is contained in:
Yanbo Liang 2015-08-26 22:19:11 -07:00 committed by Reynold Xin
parent 0fac144f6b
commit ce97834dc0

View file

@ -15,8 +15,14 @@
# limitations under the License.
#
import sys
if sys.version >= '3':
basestring = unicode = str
from py4j.java_gateway import JavaClass
from pyspark import RDD
from pyspark.sql import since
from pyspark.sql.column import _to_seq
from pyspark.sql.types import *
@ -125,23 +131,33 @@ class DataFrameReader(object):
@since(1.4)
def json(self, path, schema=None):
"""
Loads a JSON file (one object per line) and returns the result as
a :class`DataFrame`.
Loads a JSON file (one object per line) or an RDD of Strings storing JSON objects
(one object per record) and returns the result as a :class`DataFrame`.
If the ``schema`` parameter is not specified, this function goes
through the input once to determine the input schema.
:param path: string, path to the JSON dataset.
:param path: string represents path to the JSON dataset,
or RDD of Strings storing JSON objects.
:param schema: an optional :class:`StructType` for the input schema.
>>> df = sqlContext.read.json('python/test_support/sql/people.json')
>>> df.dtypes
>>> df1 = sqlContext.read.json('python/test_support/sql/people.json')
>>> df1.dtypes
[('age', 'bigint'), ('name', 'string')]
>>> rdd = sc.textFile('python/test_support/sql/people.json')
>>> df2 = sqlContext.read.json(rdd)
>>> df2.dtypes
[('age', 'bigint'), ('name', 'string')]
"""
if schema is not None:
self.schema(schema)
return self._df(self._jreader.json(path))
if isinstance(path, basestring):
return self._df(self._jreader.json(path))
elif isinstance(path, RDD):
return self._df(self._jreader.json(path._jrdd))
else:
raise TypeError("path can be only string or RDD")
@since(1.4)
def table(self, tableName):