Skip to content

Commit

Permalink
[SPARK-45220][PYTHON][DOCS] Refine docstring of DataFrame.join
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This PR refines the docstring of `DataFrame.join` by adding more examples and explanations.

### Why are the changes needed?

To improve PySpark documentation.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

doctest

### Was this patch authored or co-authored using generative AI tooling?

No

Closes apache#43039 from allisonwang-db/spark-45220-refine-join.

Authored-by: allisonwang-db <[email protected]>
Signed-off-by: Hyukjin Kwon <[email protected]>
  • Loading branch information
allisonwang-db authored and HyukjinKwon committed Oct 19, 2023
1 parent 47d3fa8 commit db16236
Showing 1 changed file with 124 additions and 43 deletions.
167 changes: 124 additions & 43 deletions python/pyspark/sql/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -2646,7 +2646,8 @@ def join(
on: Optional[Union[str, List[str], Column, List[Column]]] = None,
how: Optional[str] = None,
) -> "DataFrame":
"""Joins with another :class:`DataFrame`, using the given join expression.
"""
Joins with another :class:`DataFrame`, using the given join expression.
.. versionadded:: 1.3.0
Expand Down Expand Up @@ -2675,67 +2676,147 @@ def join(
Examples
--------
The following performs a full outer join between ``df1`` and ``df2``.
The following examples demonstrate various join types among ``df1``, ``df2``, and ``df3``.
>>> import pyspark.sql.functions as sf
>>> from pyspark.sql import Row
>>> from pyspark.sql.functions import desc
>>> df = spark.createDataFrame([(2, "Alice"), (5, "Bob")]).toDF("age", "name")
>>> df2 = spark.createDataFrame([Row(height=80, name="Tom"), Row(height=85, name="Bob")])
>>> df3 = spark.createDataFrame([Row(age=2, name="Alice"), Row(age=5, name="Bob")])
>>> df4 = spark.createDataFrame([
... Row(age=10, height=80, name="Alice"),
... Row(age=5, height=None, name="Bob"),
... Row(age=None, height=None, name="Tom"),
... Row(age=None, height=None, name=None),
>>> df = spark.createDataFrame([Row(name="Alice", age=2), Row(name="Bob", age=5)])
>>> df2 = spark.createDataFrame([Row(name="Tom", height=80), Row(name="Bob", height=85)])
>>> df3 = spark.createDataFrame([
... Row(name="Alice", age=10, height=80),
... Row(name="Bob", age=5, height=None),
... Row(name="Tom", age=None, height=None),
... Row(name=None, age=None, height=None),
... ])
Inner join on columns (default)
>>> df.join(df2, 'name').select(df.name, df2.height).show()
+----+------+
|name|height|
+----+------+
| Bob| 85|
+----+------+
>>> df.join(df4, ['name', 'age']).select(df.name, df.age).show()
+----+---+
|name|age|
+----+---+
| Bob| 5|
+----+---+
Outer join for both DataFrames on the 'name' column.
>>> df.join(df2, df.name == df2.name, 'outer').select(
... df.name, df2.height).sort(desc("name")).show()
>>> df.join(df2, "name").show()
+----+---+------+
|name|age|height|
+----+---+------+
| Bob| 5| 85|
+----+---+------+
>>> df.join(df3, ["name", "age"]).show()
+----+---+------+
|name|age|height|
+----+---+------+
| Bob| 5| NULL|
+----+---+------+
Outer join on a single column with an explicit join condition.
When the join condition is explicited stated: `df.name == df2.name`, this will
produce all records where the names match, as well as those that don't (since
it's an outer join). If there are names in `df2` that are not present in `df`,
they will appear with `NULL` in the `name` column of `df`, and vice versa for `df2`.
>>> joined = df.join(df2, df.name == df2.name, "outer").sort(sf.desc(df.name))
>>> joined.show() # doctest: +SKIP
+-----+----+----+------+
| name| age|name|height|
+-----+----+----+------+
| Bob| 5| Bob| 85|
|Alice| 2|NULL| NULL|
| NULL|NULL| Tom| 80|
+-----+----+----+------+
To unambiguously select output columns, specify the dataframe along with the column name:
>>> joined.select(df.name, df2.height).show() # doctest: +SKIP
+-----+------+
| name|height|
+-----+------+
| Bob| 85|
|Alice| NULL|
| NULL| 80|
+-----+------+
>>> df.join(df2, 'name', 'outer').select('name', 'height').sort(desc("name")).show()
+-----+------+
| name|height|
+-----+------+
| Tom| 80|
| Bob| 85|
|Alice| NULL|
+-----+------+
Outer join for both DataFrams with multiple columns.
However, in self-joins, direct column references can cause ambiguity:
>>> df.join(df, df.name == df.name, "outer").select(df.name).show() # doctest: +SKIP
Traceback (most recent call last):
...
pyspark.errors.exceptions.captured.AnalysisException: Column name#0 are ambiguous...
A better approach is to assign aliases to the dataframes, and then reference
the ouptut columns from the join operation using these aliases:
>>> df.join(
... df3,
... [df.name == df3.name, df.age == df3.age],
... 'outer'
... ).select(df.name, df3.age).show()
>>> df.alias("a").join(
... df.alias("b"), sf.col("a.name") == sf.col("b.name"), "outer"
... ).sort(sf.desc("a.name")).select("a.name", "b.age").show()
+-----+---+
| name|age|
+-----+---+
|Alice| 2|
| Bob| 5|
|Alice| 2|
+-----+---+
Outer join on a single column with implicit join condition using column name
When you provide the column name directly as the join condition, Spark will treat
both name columns as one, and will not produce separate columns for `df.name` and
`df2.name`. This avoids having duplicate columns in the output.
>>> df.join(df2, "name", "outer").sort(sf.desc("name")).show()
+-----+----+------+
| name| age|height|
+-----+----+------+
| Tom|NULL| 80|
| Bob| 5| 85|
|Alice| 2| NULL|
+-----+----+------+
Outer join on multiple columns
>>> df.join(df3, ["name", "age"], "outer").show()
+-----+----+------+
| name| age|height|
+-----+----+------+
| NULL|NULL| NULL|
|Alice| 2| NULL|
|Alice| 10| 80|
| Bob| 5| NULL|
| Tom|NULL| NULL|
+-----+----+------+
Left outer join on columns
>>> df.join(df2, "name", "left_outer").show()
+-----+---+------+
| name|age|height|
+-----+---+------+
|Alice| 2| NULL|
| Bob| 5| 85|
+-----+---+------+
Right outer join on columns
>>> df.join(df2, "name", "right_outer").show()
+----+----+------+
|name| age|height|
+----+----+------+
| Tom|NULL| 80|
| Bob| 5| 85|
+----+----+------+
Left semi join on columns
>>> df.join(df2, "name", "left_semi").show()
+----+---+
|name|age|
+----+---+
| Bob| 5|
+----+---+
Left anti join on columns
>>> df.join(df2, "name", "left_anti").show()
+-----+---+
| name|age|
+-----+---+
|Alice| 2|
+-----+---+
"""

Expand Down

0 comments on commit db16236

Please sign in to comment.