-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-40012][PYTHON][DOCS] Make pyspark.sql.dataframe examples self-contained (Part 1) #37444
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
Changes from all commits
380149f
1fed3ca
24c4769
60605d3
1ce9604
096ab81
dfa5726
aadf3d0
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 |
|---|---|---|
|
|
@@ -477,6 +477,19 @@ def schema(self) -> StructType: | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df = spark.createDataFrame( | ||
| ... [(14, "Tom"), (23, "Alice"),(16, "Bob")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| | 16| Bob| | ||
| +---+-----+ | ||
|
|
||
| Retrieve the schema of the current DataFrame. | ||
|
|
||
| >>> df.schema | ||
Transurgeon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| StructType([StructField('age', IntegerType(), True), | ||
| StructField('name', StringType(), True)]) | ||
|
|
@@ -731,29 +744,51 @@ def show(self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df | ||
| DataFrame[age: int, name: string] | ||
| >>> df = spark.createDataFrame([(14, "Tom"), (23, "Alice"), | ||
| ... (16, "Bob")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 2|Alice| | ||
| | 5| Bob| | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| | 16| Bob| | ||
| +---+-----+ | ||
|
|
||
| Show only top 2 rows. | ||
|
|
||
| >>> df.show(2) | ||
Transurgeon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| +---+-----+ | ||
| only showing top 2 rows | ||
|
|
||
| Show :class:`DataFrame` where the maximum number of characters is 3. | ||
|
|
||
| >>> df.show(truncate=3) | ||
| +---+----+ | ||
| |age|name| | ||
| +---+----+ | ||
| | 2| Ali| | ||
| | 5| Bob| | ||
| | 14| Tom| | ||
| | 23| Ali| | ||
| | 16| Bob| | ||
| +---+----+ | ||
|
|
||
| Show :class:`DataFrame` vertically. | ||
|
|
||
| >>> df.show(vertical=True) | ||
| -RECORD 0----- | ||
| age | 2 | ||
| name | Alice | ||
| age | 14 | ||
| name | Tom | ||
| -RECORD 1----- | ||
| age | 5 | ||
| name | Bob | ||
| age | 23 | ||
| name | Alice | ||
| -RECORD 2----- | ||
| age | 16 | ||
| name | Bob | ||
| """ | ||
|
|
||
| if not isinstance(n, int) or isinstance(n, bool): | ||
|
|
@@ -992,8 +1027,21 @@ def count(self) -> int: | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df = spark.createDataFrame([(14, "Tom"), (23, "Alice"), | ||
| ... (16, "Bob")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| | 16| Bob| | ||
| +---+-----+ | ||
|
|
||
| Return the number of rows in the :class:`DataFrame`. | ||
|
|
||
| >>> df.count() | ||
Transurgeon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 2 | ||
| 3 | ||
| """ | ||
| return int(self._jdf.count()) | ||
|
|
||
|
|
@@ -1088,8 +1136,21 @@ def take(self, num: int) -> List[Row]: | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df = spark.createDataFrame([(14, "Tom"), (23, "Alice"), | ||
| ... (16, "Bob")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| | 16| Bob| | ||
| +---+-----+ | ||
|
|
||
| Return the first 2 rows of the :class:`DataFrame`. | ||
|
|
||
| >>> df.take(2) | ||
Transurgeon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| [Row(age=2, name='Alice'), Row(age=5, name='Bob')] | ||
| [Row(age=14, name='Tom'), Row(age=23, name='Alice')] | ||
| """ | ||
| return self.limit(num).collect() | ||
|
|
||
|
|
@@ -1115,8 +1176,18 @@ def tail(self, num: int) -> List[Row]: | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df.tail(1) | ||
| [Row(age=5, name='Bob')] | ||
| >>> df = spark.createDataFrame([(14, "Tom"), (23, "Alice"), | ||
| ... (16, "Bob")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| | 16| Bob| | ||
| +---+-----+ | ||
| >>> df.tail(2) | ||
| [Row(age=23, name='Alice'), Row(age=16, name='Bob')] | ||
| """ | ||
| with SCCallSiteSync(self._sc): | ||
| sock_info = self._jdf.tailToPython(num) | ||
|
|
@@ -1511,6 +1582,19 @@ def distinct(self) -> "DataFrame": | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df = spark.createDataFrame([(14, "Tom"), (23, "Alice"), | ||
| ... (23, "Alice")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| | 23|Alice| | ||
| +---+-----+ | ||
|
|
||
| Return the number of distinct rows in the :class:`DataFrame` | ||
|
|
||
| >>> df.distinct().count() | ||
Transurgeon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 2 | ||
| """ | ||
|
|
@@ -1722,8 +1806,20 @@ def dtypes(self) -> List[Tuple[str, str]]: | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df = spark.createDataFrame( | ||
| ... [(14, "Tom"), (23, "Alice"),(16, "Bob")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| +---+-----+ | ||
|
|
||
| Return the name of each column along with their respective data types | ||
|
|
||
| >>> df.dtypes | ||
| [('age', 'int'), ('name', 'string')] | ||
| [('age', 'bigint'), ('name', 'string')] | ||
| """ | ||
| return [(str(f.name), f.dataType.simpleString()) for f in self.schema.fields] | ||
|
|
||
|
|
@@ -3358,7 +3454,21 @@ def fillna( | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df4.na.fill(50).show() | ||
| >>> df = spark.createDataFrame([(10, 80, "Alice"), (5, None, "Bob"), | ||
| ... (None, None, "Tom"), (None, None, None)], ["age", "height", "name"]) | ||
| >>> df.show() | ||
| +----+------+-----+ | ||
| | age|height| name| | ||
| +----+------+-----+ | ||
| | 10| 80|Alice| | ||
| | 5| null| Bob| | ||
| |null| null| Tom| | ||
| |null| null| null| | ||
| +----+------+-----+ | ||
|
|
||
| Fill all null values with 50 when the data type of the column is an Integer | ||
|
|
||
| >>> df.na.fill(50).show() | ||
| +---+------+-----+ | ||
| |age|height| name| | ||
| +---+------+-----+ | ||
|
|
@@ -3368,7 +3478,20 @@ def fillna( | |
| | 50| 50| null| | ||
| +---+------+-----+ | ||
|
|
||
| >>> df5.na.fill(False).show() | ||
| >>> df = spark.createDataFrame([(10, "Alice", None), (5, "Bob", None), | ||
| ... (None, "Mallory", True)], ["age", "name", "spy"]) | ||
| >>> df.show() | ||
| +----+-------+----+ | ||
| | age| name| spy| | ||
| +----+-------+----+ | ||
| | 10| Alice|null| | ||
| | 5| Bob|null| | ||
| |null|Mallory|true| | ||
| +----+-------+----+ | ||
|
|
||
| Fill all null values with ``False`` when the data type of the column is a boolean | ||
|
|
||
| >>> df.na.fill(False).show() | ||
| +----+-------+-----+ | ||
| | age| name| spy| | ||
| +----+-------+-----+ | ||
|
|
@@ -3377,7 +3500,21 @@ def fillna( | |
| |null|Mallory| true| | ||
| +----+-------+-----+ | ||
|
|
||
| >>> df4.na.fill({'age': 50, 'name': 'unknown'}).show() | ||
| >>> df = spark.createDataFrame([(10, 80, "Alice"), (5, None, "Bob"), | ||
| ... (None, None, "Tom"), (None, None, None)], ["age", "height", "name"]) | ||
| >>> df.show() | ||
| +----+------+-----+ | ||
| | age|height| name| | ||
| +----+------+-----+ | ||
| | 10| 80|Alice| | ||
| | 5| null| Bob| | ||
| |null| null| Tom| | ||
| |null| null| null| | ||
| +----+------+-----+ | ||
|
|
||
| Fill all null values in the 'age' column to 50 and "unknown" in the 'name' column | ||
|
|
||
| >>> df.na.fill({'age': 50, 'name': 'unknown'}).show() | ||
| +---+------+-------+ | ||
| |age|height| name| | ||
| +---+------+-------+ | ||
|
|
@@ -3489,7 +3626,21 @@ def replace( # type: ignore[misc] | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df4.na.replace(10, 20).show() | ||
| >>> df = spark.createDataFrame([(10, 80, "Alice"), (5, None, "Bob"), | ||
| ... (None, None, "Tom"), (None, None, None)], ["age", "height", "name"]) | ||
| >>> df.show() | ||
| +----+------+-----+ | ||
| | age|height| name| | ||
| +----+------+-----+ | ||
| | 10| 80|Alice| | ||
| | 5| null| Bob| | ||
| |null| null| Tom| | ||
| |null| null| null| | ||
| +----+------+-----+ | ||
|
|
||
| Replace all instances of the value 10 to the value 20 | ||
|
|
||
| >>> df.na.replace(10, 20).show() | ||
| +----+------+-----+ | ||
| | age|height| name| | ||
| +----+------+-----+ | ||
|
|
@@ -3499,17 +3650,9 @@ def replace( # type: ignore[misc] | |
| |null| null| null| | ||
| +----+------+-----+ | ||
|
|
||
| >>> df4.na.replace('Alice', None).show() | ||
| +----+------+----+ | ||
| | age|height|name| | ||
| +----+------+----+ | ||
| | 10| 80|null| | ||
| | 5| null| Bob| | ||
| |null| null| Tom| | ||
| |null| null|null| | ||
| +----+------+----+ | ||
| Replace all instances of Alice to null | ||
|
|
||
| >>> df4.na.replace({'Alice': None}).show() | ||
| >>> df.na.replace('Alice', None).show() | ||
| +----+------+----+ | ||
| | age|height|name| | ||
| +----+------+----+ | ||
|
|
@@ -3519,7 +3662,9 @@ def replace( # type: ignore[misc] | |
| |null| null|null| | ||
| +----+------+----+ | ||
|
|
||
| >>> df4.na.replace(['Alice', 'Bob'], ['A', 'B'], 'name').show() | ||
| Replace all instances of Alice to 'A' and Bob to 'B' under the name column | ||
|
|
||
| >>> df.na.replace(['Alice', 'Bob'], ['A', 'B'], 'name').show() | ||
| +----+------+----+ | ||
| | age|height|name| | ||
| +----+------+----+ | ||
|
|
@@ -4064,12 +4209,32 @@ def drop(self, *cols: "ColumnOrName") -> "DataFrame": # type: ignore[misc] | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df.drop('age').collect() | ||
| [Row(name='Alice'), Row(name='Bob')] | ||
|
|
||
| >>> df.drop(df.age).collect() | ||
| [Row(name='Alice'), Row(name='Bob')] | ||
|
|
||
| >>> df = spark.createDataFrame([(14, "Tom"), (23, "Alice"), | ||
| ... (16, "Bob")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| | 16| Bob| | ||
| +---+-----+ | ||
| >>> df.drop('age').show() | ||
| +-----+ | ||
| | name| | ||
| +-----+ | ||
| | Tom| | ||
| |Alice| | ||
| | Bob| | ||
| +-----+ | ||
| >>> df.drop(df.age).show() | ||
| +-----+ | ||
| | name| | ||
| +-----+ | ||
| | Tom| | ||
| |Alice| | ||
| | Bob| | ||
| +-----+ | ||
| >>> df.join(df2, df.name == df2.name, 'inner').drop(df.name).collect() | ||
|
Contributor
Author
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 am not sure what these 3 inner joins do exactly. I dont see anywhere an instantiation of df2.. What should I do with these 3 examples?
Member
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's showing a common example that join and drop the join key. |
||
| [Row(age=5, height=85, name='Bob')] | ||
|
|
||
|
|
@@ -4100,7 +4265,8 @@ def toDF(self, *cols: "ColumnOrName") -> "DataFrame": | |
| Parameters | ||
| ---------- | ||
| cols : str | ||
| new column names | ||
| new column names. The length of the list needs to be the same as the number | ||
| of columns in the initial :class:`DataFrame` | ||
|
|
||
| Returns | ||
| ------- | ||
|
|
@@ -4109,8 +4275,18 @@ def toDF(self, *cols: "ColumnOrName") -> "DataFrame": | |
|
|
||
| Examples | ||
| -------- | ||
| >>> df = spark.createDataFrame([(14, "Tom"), (23, "Alice"), | ||
| ... (16, "Bob")], ["age", "name"]) | ||
| >>> df.show() | ||
| +---+-----+ | ||
| |age| name| | ||
| +---+-----+ | ||
| | 14| Tom| | ||
| | 23|Alice| | ||
| | 16| Bob| | ||
| +---+-----+ | ||
| >>> df.toDF('f1', 'f2').collect() | ||
| [Row(f1=2, f2='Alice'), Row(f1=5, f2='Bob')] | ||
| [Row(f1=14, f2='Tom'), Row(f1=23, f2='Alice'), Row(f1=16, f2='Bob')] | ||
| """ | ||
| jdf = self._jdf.toDF(self._jseq(cols)) | ||
| return DataFrame(jdf, self.sparkSession) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.