This is an automated email from the ASF dual-hosted git repository.
gurwls223 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 7a5bb5d83551 [SPARK-47078][DOCS][PYTHON] Documentation for
SparkSession-based Profilers
7a5bb5d83551 is described below
commit 7a5bb5d83551b749a64f38dc937e9b488616cf8d
Author: Xinrong Meng <[email protected]>
AuthorDate: Fri Mar 8 09:42:40 2024 +0900
[SPARK-47078][DOCS][PYTHON] Documentation for SparkSession-based Profilers
### What changes were proposed in this pull request?
Documentation for SparkSession-based Profilers.
### Why are the changes needed?
For easier user onboarding and better usability.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test. Screenshots of built htmls are as shown below.

### Was this patch authored or co-authored using generative AI tooling?
No.
Closes #45269 from xinrong-meng/profiler_doc.
Authored-by: Xinrong Meng <[email protected]>
Signed-off-by: Hyukjin Kwon <[email protected]>
---
python/docs/source/development/debugging.rst | 69 +++++++---------------
.../source/reference/pyspark.sql/spark_session.rst | 1 +
python/pyspark/sql/connect/session.py | 2 +
python/pyspark/sql/session.py | 12 ++++
4 files changed, 35 insertions(+), 49 deletions(-)
diff --git a/python/docs/source/development/debugging.rst
b/python/docs/source/development/debugging.rst
index ef848a6e961e..b0b2c4837ded 100644
--- a/python/docs/source/development/debugging.rst
+++ b/python/docs/source/development/debugging.rst
@@ -215,13 +215,10 @@ 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.
-
-.. code-block:: bash
-
- pyspark --conf spark.python.profile.memory=true
+Python/Pandas UDFs. That can be used on editors with line numbers such as
Jupyter notebooks. UDFs with iterators as inputs/outputs are not supported.
+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``. An example on a Jupyter
notebook is as shown below.
.. code-block:: python
@@ -232,10 +229,11 @@ can be used on editors with line numbers such as Jupyter
notebooks. An example o
def add1(x):
return x + 1
+ spark.conf.set("spark.sql.pyspark.udf.profiler", "memory")
+
added = df.select(add1("id"))
added.show()
- sc.show_profiles()
-
+ spark.profile.show(type="memory")
The result profile is as shown below.
@@ -258,7 +256,6 @@ The UDF IDs can be seen in the query plan, for example,
``add1(...)#2L`` in ``Ar
added.explain()
-
.. code-block:: text
== Physical Plan ==
@@ -266,8 +263,11 @@ The UDF IDs can be seen in the query plan, for example,
``add1(...)#2L`` in ``Ar
+- ArrowEvalPython [add1(id#0L)#2L], [pythonUDF0#11L], 200
+- *(1) Range (0, 10, step=1, splits=16)
-This feature is not supported with registered UDFs or UDFs with iterators as
inputs/outputs.
+We can clear the result memory profile as shown below.
+
+.. code-block:: python
+ spark.profile.clear(id=2, type="memory")
Identifying Hot Loops (Python Profilers)
----------------------------------------
@@ -306,47 +306,14 @@ regular Python process unless you are running your driver
program in another mac
276 0.000 0.000 0.002 0.000 <frozen
importlib._bootstrap>:147(__enter__)
...
-Executor Side
-~~~~~~~~~~~~~
-
-To use this on executor side, PySpark provides remote `Python Profilers
<https://docs.python.org/3/library/profile.html>`_ for
-executor side, which can be enabled by setting ``spark.python.profile``
configuration to ``true``.
-
-.. code-block:: bash
-
- pyspark --conf spark.python.profile=true
-
-
-.. code-block:: python
-
- >>> rdd = sc.parallelize(range(100)).map(str)
- >>> rdd.count()
- 100
- >>> sc.show_profiles()
- ============================================================
- Profile of RDD<id=1>
- ============================================================
- 728 function calls (692 primitive calls) in 0.004 seconds
-
- Ordered by: internal time, cumulative time
-
- ncalls tottime percall cumtime percall filename:lineno(function)
- 12 0.001 0.000 0.001 0.000
serializers.py:210(load_stream)
- 12 0.000 0.000 0.000 0.000 {built-in method
_pickle.dumps}
- 12 0.000 0.000 0.001 0.000
serializers.py:252(dump_stream)
- 12 0.000 0.000 0.001 0.000 context.py:506(f)
- ...
-
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``.
-
-.. code-block:: bash
-
- pyspark --conf spark.python.profile=true
+PySpark provides remote `Python Profilers
<https://docs.python.org/3/library/profile.html>`_ for
+Python/Pandas UDFs. UDFs with iterators as inputs/outputs are not supported.
+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``. An example is as shown below.
.. code-block:: python
@@ -358,6 +325,7 @@ Python/Pandas UDFs, which can be enabled by setting
``spark.python.profile`` con
...
>>> added = df.select(add1("id"))
+ >>> spark.conf.set("spark.sql.pyspark.udf.profiler", "perf")
>>> added.show()
+--------+
|add1(id)|
@@ -365,7 +333,7 @@ Python/Pandas UDFs, which can be enabled by setting
``spark.python.profile`` con
...
+--------+
- >>> sc.show_profiles()
+ >>> spark.profile.show(type="perf")
============================================================
Profile of UDF<id=2>
============================================================
@@ -390,8 +358,11 @@ The UDF IDs can be seen in the query plan, for example,
``add1(...)#2L`` in ``Ar
+- ArrowEvalPython [add1(id#0L)#2L], [pythonUDF0#11L], 200
+- *(1) Range (0, 10, step=1, splits=16)
+We can clear the result performance profile as shown below.
+
+.. code-block:: python
-This feature is not supported with registered UDFs.
+ >>> spark.profile.clear(id=2, type="perf")
Common Exceptions / Errors
--------------------------
diff --git a/python/docs/source/reference/pyspark.sql/spark_session.rst
b/python/docs/source/reference/pyspark.sql/spark_session.rst
index f242e4439cf4..4be343c52140 100644
--- a/python/docs/source/reference/pyspark.sql/spark_session.rst
+++ b/python/docs/source/reference/pyspark.sql/spark_session.rst
@@ -49,6 +49,7 @@ See also :class:`SparkSession`.
SparkSession.createDataFrame
SparkSession.getActiveSession
SparkSession.newSession
+ SparkSession.profile
SparkSession.range
SparkSession.read
SparkSession.readStream
diff --git a/python/pyspark/sql/connect/session.py
b/python/pyspark/sql/connect/session.py
index 6c4a3064edf4..0e05f2743a0d 100644
--- a/python/pyspark/sql/connect/session.py
+++ b/python/pyspark/sql/connect/session.py
@@ -946,6 +946,8 @@ class SparkSession:
def profile(self) -> Profile:
return Profile(self._client._profiler_collector)
+ profile.__doc__ = PySparkSession.profile.__doc__
+
SparkSession.__doc__ = PySparkSession.__doc__
diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index f12a5365d86c..6c80b7f42da4 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -908,6 +908,18 @@ class SparkSession(SparkConversionMixin):
@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(
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]