diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py
index 78247c8fa73723d62ce8c869911c61f662a1ef2f..3fa6895880a973e347358db7f8190100bb58629e 100644
--- a/python/pyspark/sql/readwriter.py
+++ b/python/pyspark/sql/readwriter.py
@@ -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):