@@ -1763,9 +1763,27 @@ def coalesce(self, numPartitions: int) -> "DataFrame":
1763
1763
1764
1764
Examples
1765
1765
--------
1766
- >>> df = spark.range(10)
1767
- >>> df.coalesce(1).rdd.getNumPartitions()
1768
- 1
1766
+ >>> from pyspark.sql import functions as sf
1767
+ >>> spark.range(0, 10, 1, 3).select(
1768
+ ... sf.spark_partition_id().alias("partition")
1769
+ ... ).distinct().sort("partition").show()
1770
+ +---------+
1771
+ |partition|
1772
+ +---------+
1773
+ | 0|
1774
+ | 1|
1775
+ | 2|
1776
+ +---------+
1777
+
1778
+ >>> from pyspark.sql import functions as sf
1779
+ >>> spark.range(0, 10, 1, 3).coalesce(1).select(
1780
+ ... sf.spark_partition_id().alias("partition")
1781
+ ... ).distinct().sort("partition").show()
1782
+ +---------+
1783
+ |partition|
1784
+ +---------+
1785
+ | 0|
1786
+ +---------+
1769
1787
"""
1770
1788
return DataFrame (self ._jdf .coalesce (numPartitions ), self .sparkSession )
1771
1789
@@ -1809,23 +1827,78 @@ def repartition( # type: ignore[misc]
1809
1827
1810
1828
Examples
1811
1829
--------
1812
- >>> df = spark.createDataFrame(
1813
- ... [(14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"])
1830
+ >>> from pyspark.sql import functions as sf
1831
+ >>> df = spark.range(0, 64, 1, 9).withColumn(
1832
+ ... "name", sf.concat(sf.lit("name_"), sf.col("id").cast("string"))
1833
+ ... ).withColumn(
1834
+ ... "age", sf.col("id") - 32
1835
+ ... )
1836
+ >>> df.select(
1837
+ ... sf.spark_partition_id().alias("partition")
1838
+ ... ).distinct().sort("partition").show()
1839
+ +---------+
1840
+ |partition|
1841
+ +---------+
1842
+ | 0|
1843
+ | 1|
1844
+ | 2|
1845
+ | 3|
1846
+ | 4|
1847
+ | 5|
1848
+ | 6|
1849
+ | 7|
1850
+ | 8|
1851
+ +---------+
1814
1852
1815
1853
Repartition the data into 10 partitions.
1816
1854
1817
- >>> df.repartition(10).rdd.getNumPartitions()
1818
- 10
1855
+ >>> df.repartition(10).select(
1856
+ ... sf.spark_partition_id().alias("partition")
1857
+ ... ).distinct().sort("partition").show()
1858
+ +---------+
1859
+ |partition|
1860
+ +---------+
1861
+ | 0|
1862
+ | 1|
1863
+ | 2|
1864
+ | 3|
1865
+ | 4|
1866
+ | 5|
1867
+ | 6|
1868
+ | 7|
1869
+ | 8|
1870
+ | 9|
1871
+ +---------+
1819
1872
1820
1873
Repartition the data into 7 partitions by 'age' column.
1821
1874
1822
- >>> df.repartition(7, "age").rdd.getNumPartitions()
1823
- 7
1875
+ >>> df.repartition(7, "age").select(
1876
+ ... sf.spark_partition_id().alias("partition")
1877
+ ... ).distinct().sort("partition").show()
1878
+ +---------+
1879
+ |partition|
1880
+ +---------+
1881
+ | 0|
1882
+ | 1|
1883
+ | 2|
1884
+ | 3|
1885
+ | 4|
1886
+ | 5|
1887
+ | 6|
1888
+ +---------+
1824
1889
1825
1890
Repartition the data into 7 partitions by 'age' and 'name columns.
1826
1891
1827
- >>> df.repartition(3, "name", "age").rdd.getNumPartitions()
1828
- 3
1892
+ >>> df.repartition(3, "name", "age").select(
1893
+ ... sf.spark_partition_id().alias("partition")
1894
+ ... ).distinct().sort("partition").show()
1895
+ +---------+
1896
+ |partition|
1897
+ +---------+
1898
+ | 0|
1899
+ | 1|
1900
+ | 2|
1901
+ +---------+
1829
1902
"""
1830
1903
if isinstance (numPartitions , int ):
1831
1904
if len (cols ) == 0 :
@@ -1893,15 +1966,23 @@ def repartitionByRange( # type: ignore[misc]
1893
1966
1894
1967
Examples
1895
1968
--------
1896
- >>> df = spark.createDataFrame(
1897
- ... [(14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"])
1898
-
1899
1969
Repartition the data into 2 partitions by range in 'age' column.
1900
- For example, the first partition can have ``(14, "Tom")``, and the second
1901
- partition would have ``(16, "Bob")`` and ``(23, "Alice")``.
1970
+ For example, the first partition can have ``(14, "Tom")`` and ``(16, "Bob")``,
1971
+ and the second partition would have ``(23, "Alice")``.
1902
1972
1903
- >>> df.repartitionByRange(2, "age").rdd.getNumPartitions()
1904
- 2
1973
+ >>> from pyspark.sql import functions as sf
1974
+ >>> spark.createDataFrame(
1975
+ ... [(14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"]
1976
+ ... ).repartitionByRange(2, "age").select(
1977
+ ... "age", "name", sf.spark_partition_id()
1978
+ ... ).show()
1979
+ +---+-----+--------------------+
1980
+ |age| name|SPARK_PARTITION_ID()|
1981
+ +---+-----+--------------------+
1982
+ | 14| Tom| 0|
1983
+ | 16| Bob| 0|
1984
+ | 23|Alice| 1|
1985
+ +---+-----+--------------------+
1905
1986
"""
1906
1987
if isinstance (numPartitions , int ):
1907
1988
if len (cols ) == 0 :
0 commit comments