-
Notifications
You must be signed in to change notification settings - Fork 29.1k
[SPARK-56253][PYTHON][CONNECT] Make spark.read.json accept DataFrame input #55097
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -320,7 +320,7 @@ def load( | |
|
|
||
| def json( | ||
| self, | ||
| path: Union[str, List[str], "RDD[str]"], | ||
| path: Union[str, List[str], "RDD[str]", "DataFrame"], | ||
| schema: Optional[Union[StructType, str]] = None, | ||
| primitivesAsString: Optional[Union[bool, str]] = None, | ||
| prefersDecimal: Optional[Union[bool, str]] = None, | ||
|
|
@@ -361,11 +361,15 @@ def json( | |
| .. versionchanged:: 3.4.0 | ||
| Supports Spark Connect. | ||
|
|
||
| .. versionchanged:: 4.1.0 | ||
| Supports DataFrame input. | ||
|
|
||
| Parameters | ||
| ---------- | ||
| path : str, list or :class:`RDD` | ||
| path : str, list, :class:`RDD`, or :class:`DataFrame` | ||
| string represents path to the JSON dataset, or a list of paths, | ||
| or RDD of Strings storing JSON objects. | ||
| or RDD of Strings storing JSON objects, | ||
| or a DataFrame with a single string column containing JSON strings. | ||
| schema : :class:`pyspark.sql.types.StructType` or str, optional | ||
| an optional :class:`pyspark.sql.types.StructType` for the input schema or | ||
| a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``). | ||
|
|
@@ -434,6 +438,20 @@ def json( | |
| +----+---+ | ||
| | Bob| 30| | ||
| +----+---+ | ||
|
|
||
| Example 4: Parse JSON from a DataFrame with a single string column. | ||
|
|
||
| >>> json_df = spark.createDataFrame( | ||
| ... [('{"name": "Alice", "age": 25}',), ('{"name": "Bob", "age": 30}',)], | ||
| ... schema="value STRING", | ||
| ... ) | ||
| >>> spark.read.json(json_df).sort("name").show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 25|Alice| | ||
| | 30| Bob| | ||
| +---+-----+ | ||
| """ | ||
| self._set_opts( | ||
| schema=schema, | ||
|
|
@@ -486,12 +504,20 @@ def func(iterator: Iterable) -> Iterable: | |
| assert self._spark._jvm is not None | ||
| jrdd = keyed._jrdd.map(self._spark._jvm.BytesToString()) | ||
| return self._df(self._jreader.json(jrdd)) | ||
|
|
||
| from pyspark.sql.dataframe import DataFrame | ||
|
|
||
| if isinstance(path, DataFrame): | ||
| assert self._spark._jvm is not None | ||
| string_encoder = self._spark._jvm.Encoders.STRING() | ||
| jdataset = getattr(path._jdf, "as")(string_encoder) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we add a private overload in JVM side
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What would be the benefit?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it can simplify the python side callsite a bit. |
||
| return self._df(self._jreader.json(jdataset)) | ||
| else: | ||
| raise PySparkTypeError( | ||
| errorClass="NOT_EXPECTED_TYPE", | ||
| messageParameters={ | ||
| "arg_name": "path", | ||
| "expected_type": "str or list[RDD]", | ||
| "expected_type": "str, list, RDD, or DataFrame", | ||
| "arg_type": type(path).__name__, | ||
| }, | ||
| ) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -93,6 +93,24 @@ def test_linesep_json(self): | |
| finally: | ||
| shutil.rmtree(tpath) | ||
|
|
||
| def test_json_with_dataframe_input(self): | ||
| json_df = self.spark.createDataFrame( | ||
| [('{"name": "Alice", "age": 25}',), ('{"name": "Bob", "age": 30}',)], | ||
| schema="value STRING", | ||
| ) | ||
| result = self.spark.read.json(json_df) | ||
| expected = [Row(age=25, name="Alice"), Row(age=30, name="Bob")] | ||
| self.assertEqual(sorted(result.collect(), key=lambda r: r.name), expected) | ||
|
|
||
| def test_json_with_dataframe_input_and_schema(self): | ||
| json_df = self.spark.createDataFrame( | ||
| [('{"name": "Alice", "age": 25}',), ('{"name": "Bob", "age": 30}',)], | ||
| schema="value STRING", | ||
| ) | ||
| result = self.spark.read.json(json_df, schema="name STRING, age INT") | ||
| expected = [Row(name="Alice", age=25), Row(name="Bob", age=30)] | ||
| self.assertEqual(sorted(result.collect(), key=lambda r: r.name), expected) | ||
|
|
||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The JVM side implementations accept So I think we need to add negative tests cases like: |
||
| def test_multiline_csv(self): | ||
| ages_newlines = self.spark.read.csv( | ||
| "python/test_support/sql/ages_newlines.csv", multiLine=True | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
do we need to check the schema in python side that it should only contain a string type column? or we depends on the check in the JVM side (or connect server)