Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
73 changes: 70 additions & 3 deletions python/docs/source/development/debugging.rst
Original file line number Diff line number Diff line change
Expand Up @@ -215,8 +215,14 @@ Python/Pandas UDF
~~~~~~~~~~~~~~~~~

PySpark provides remote `memory_profiler <https://github.com/pythonprofilers/memory_profiler>`_ for
Python/Pandas UDFs, which can be enabled by setting ``spark.python.profile.memory`` configuration to ``true``. That
can be used on editors with line numbers such as Jupyter notebooks. An example on a Jupyter notebook is as shown below.
Python/Pandas UDFs.

SparkContext-based
^^^^^^^^^^^^^^^^^^

SparkContext-based memory profiler can be enabled by setting ``spark.python.profile.memory`` configuration to ``true``.
That can be used on editors with line numbers such as Jupyter notebooks. An example on a Jupyter notebook is as shown
below.

.. code-block:: bash

Expand Down Expand Up @@ -268,6 +274,31 @@ The UDF IDs can be seen in the query plan, for example, ``add1(...)#2L`` in ``Ar

This feature is not supported with registered UDFs or UDFs with iterators as inputs/outputs.

SparkSession-based
^^^^^^^^^^^^^^^^^^

SparkSession-based memory profiler can be enabled by setting the `Runtime SQL configuration <https://spark.apache.org/docs/latest/configuration.html#runtime-sql-configuration>`_
``spark.sql.pyspark.udf.profiler`` to ``memory``. That can be used on editors with line numbers such as Jupyter notebooks.
The above example is modified to use the SparkSession-based memory profiler as shown below.

.. code-block:: python

from pyspark.sql.functions import pandas_udf
df = spark.range(10)

@pandas_udf("long")
def add1(x):
return x + 1

spark.conf.set("spark.sql.pyspark.udf.profiler", "memory")

added = df.select(add1("id"))
added.show()
spark.profile.show(type="memory")

The result profile is the same as that profiled by the SparkContext-based memory profiler. The UDF IDs can also be seen in the query plan.

This feature is supported on both Spark Connect and non-Spark-Connect, including registered UDFs.

Identifying Hot Loops (Python Profilers)
----------------------------------------
Expand Down Expand Up @@ -341,7 +372,12 @@ Python/Pandas UDF
~~~~~~~~~~~~~~~~~

To use this on Python/Pandas UDFs, PySpark provides remote `Python Profilers <https://docs.python.org/3/library/profile.html>`_ for
Python/Pandas UDFs, which can be enabled by setting ``spark.python.profile`` configuration to ``true``.
Python/Pandas UDFs.

SparkContext-based
Copy link
Member

@HyukjinKwon HyukjinKwon Feb 29, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you can just remove this, and just add one additional section called runtime profiler

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cc @ueshin do you have other thoughts?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about put the new doc to the first place?

  • Identifying Hot Loops (Python Profilers)
    • Driver Side
      ...
    • Executor Side
      • Python/Pandas UDF
        Show the new profiler usage
      • Legacy (for RDD or non-Spark Connect)
        Put the current doc here

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe there are many existing users of SparkContext-based profilers. Shall we keep it in the debugging guide until SparkSession-based profilers gain more adoption and positive feedbacks? I'll adjust the order to show SparkSession-based profilers first as @ueshin suggested. What do you think @HyukjinKwon?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We will remove "legacy" profilers for readability and clarity and start preparing migration guide.

^^^^^^^^^^^^^^^^^^

SparkContext-based performance profiler can be enabled by setting ``spark.python.profile`` configuration to ``true``.

.. code-block:: bash

Expand Down Expand Up @@ -393,6 +429,37 @@ The UDF IDs can be seen in the query plan, for example, ``add1(...)#2L`` in ``Ar

This feature is not supported with registered UDFs.

SparkSession-based
^^^^^^^^^^^^^^^^^^

SparkSession-based performance profiler can be enabled by setting the `Runtime SQL configuration <https://spark.apache.org/docs/latest/configuration.html#runtime-sql-configuration>`_
``spark.sql.pyspark.udf.profiler`` to ``perf``. The above example is modified to use the SparkSession-based performance profiler as shown below.

.. code-block:: python

>>> from pyspark.sql.functions import pandas_udf
>>> df = spark.range(10)
>>> @pandas_udf("long")
... def add1(x):
... return x + 1
...
>>> added = df.select(add1("id"))

>>> spark.conf.set("spark.sql.pyspark.udf.profiler", "perf")
>>> added.show()
+--------+
|add1(id)|
+--------+
...
+--------+

>>> spark.profile.show(type="perf")
...

The result profile is the same as that profiled by the SparkContext-based memory profiler. The UDF IDs can also be seen in the query plan.

This feature is supported on both Spark Connect and non-Spark-Connect, including registered UDFs.

Common Exceptions / Errors
--------------------------

Expand Down
1 change: 1 addition & 0 deletions python/docs/source/reference/pyspark.sql/spark_session.rst
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ See also :class:`SparkSession`.
SparkSession.createDataFrame
SparkSession.getActiveSession
SparkSession.newSession
SparkSession.profile
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should also have a dedicated section for profile.show, profile.dump.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I hit

[autosummary] failed to import pyspark.sql.SparkSession.profile.dump.
Possible hints:
* AttributeError: 'property' object has no attribute 'dump'
* ImportError: 
* ModuleNotFoundError: No module named 'pyspark.sql.SparkSession'

The profile property returns a Profile class instance, Sphinx might have difficulty accessing it. Do you happen to know the best way to resolve that?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Need

:template: autosummary/accessor_method.rst

?

See #44012 (comment)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm I was thinking the same but it kept failing with the error message..

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think SparkSession.builder works because it is a classproperty whereas profile is a property of SparkSession.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have a workaround 76e7387 by using autoclass, but it doesn't look consistent with the rest of the page, as shown below.

image

I'm wondering if we should have a follow-up designated for that part.

SparkSession.range
SparkSession.read
SparkSession.readStream
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/sql/connect/session.py
Original file line number Diff line number Diff line change
Expand Up @@ -946,6 +946,8 @@ def _profiler_collector(self) -> ProfilerCollector:
def profile(self) -> Profile:
return Profile(self._client._profiler_collector)

profile.__doc__ = PySparkSession.profile.__doc__


SparkSession.__doc__ = PySparkSession.__doc__

Expand Down
12 changes: 12 additions & 0 deletions python/pyspark/sql/session.py
Original file line number Diff line number Diff line change
Expand Up @@ -908,6 +908,18 @@ def dataSource(self) -> "DataSourceRegistration":

@property
def profile(self) -> Profile:
"""Returns a :class:`Profile` for performance/memory profiling.

.. versionadded:: 4.0.0

Returns
-------
:class:`Profile`

Notes
-----
Supports Spark Connect.
"""
return Profile(self._profiler_collector)

def range(
Expand Down