You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "ueshin (via GitHub)" <gi...@apache.org> on 2024/02/16 01:52:57 UTC

Re: [PR] [SPARK-47069][PYTHON] Introduce `spark.profile.show/dump` for SparkSession-based profiling [spark]

ueshin commented on code in PR #45129:
URL: https://github.com/apache/spark/pull/45129#discussion_r1491866174


##########
python/pyspark/sql/profiler.py:
##########
@@ -239,3 +241,72 @@ def _profile_results(self) -> "ProfileResults":
         with self._lock:
             value = self._accumulator.value
             return value if value is not None else {}
+
+
+class Profile:
+    """User-facing profile API. This instance can be accessed by
+    :attr:`spark.profile`.
+
+    .. versionadded: 4.0.0
+    """
+
+    def __init__(self, sparkSession: "SparkSession"):
+        self.sparkSession = sparkSession
+
+    def show(self, *, type: Optional[str] = None, id: Optional[int] = None) -> None:
+        """
+        Show the profile results.
+
+        .. versionadded:: 4.0.0
+
+        Parameters
+        ----------
+        type : str, optional
+            The profiler type, which can be either "perf" or "memory".
+        id : int, optional
+            A UDF ID to be shown. If not specified, all the results will be shown.
+        """
+        if type == "memory":
+            self.sparkSession.showMemoryProfiles(id)

Review Comment:
   Shall we remove the old APIs? I think the new APIs are enough to have.



##########
python/pyspark/sql/profiler.py:
##########
@@ -239,3 +241,72 @@ def _profile_results(self) -> "ProfileResults":
         with self._lock:
             value = self._accumulator.value
             return value if value is not None else {}
+
+
+class Profile:
+    """User-facing profile API. This instance can be accessed by
+    :attr:`spark.profile`.
+
+    .. versionadded: 4.0.0
+    """
+
+    def __init__(self, sparkSession: "SparkSession"):
+        self.sparkSession = sparkSession

Review Comment:
   It should take `ProfilerCollector`?



##########
python/pyspark/sql/profiler.py:
##########
@@ -239,3 +241,72 @@ def _profile_results(self) -> "ProfileResults":
         with self._lock:
             value = self._accumulator.value
             return value if value is not None else {}
+
+
+class Profile:
+    """User-facing profile API. This instance can be accessed by
+    :attr:`spark.profile`.
+
+    .. versionadded: 4.0.0
+    """
+
+    def __init__(self, sparkSession: "SparkSession"):
+        self.sparkSession = sparkSession
+
+    def show(self, *, type: Optional[str] = None, id: Optional[int] = None) -> None:
+        """
+        Show the profile results.
+
+        .. versionadded:: 4.0.0
+
+        Parameters
+        ----------
+        type : str, optional
+            The profiler type, which can be either "perf" or "memory".
+        id : int, optional
+            A UDF ID to be shown. If not specified, all the results will be shown.
+        """
+        if type == "memory":
+            self.sparkSession.showMemoryProfiles(id)
+        elif type == "perf" or type is None:
+            self.sparkSession.showPerfProfiles(id)
+            if type is None:  # Show both perf and memory profiles
+                self.sparkSession.showMemoryProfiles(id)
+        else:
+            raise PySparkValueError(
+                error_class="VALUE_NOT_ALLOWED",
+                message_parameters={
+                    "arg_name": "type",
+                    "allowed_values": str(["perf", "memory"]),
+                },
+            )
+
+    def dump(self, path: str, *, type: Optional[str] = None, id: Optional[int] = None) -> None:

Review Comment:
   ditto.
   ```suggestion
       def dump(self, path: str, id: Optional[int] = None, *, type: Optional[str] = None) -> None:
   ```



##########
python/pyspark/sql/profiler.py:
##########
@@ -239,3 +241,72 @@ def _profile_results(self) -> "ProfileResults":
         with self._lock:
             value = self._accumulator.value
             return value if value is not None else {}
+
+
+class Profile:
+    """User-facing profile API. This instance can be accessed by
+    :attr:`spark.profile`.
+
+    .. versionadded: 4.0.0
+    """
+
+    def __init__(self, sparkSession: "SparkSession"):
+        self.sparkSession = sparkSession
+
+    def show(self, *, type: Optional[str] = None, id: Optional[int] = None) -> None:

Review Comment:
   I prefer:
   ```suggestion
       def show(self, id: Optional[int] = None, *, type: Optional[str] = None) -> None:
   ```
   
   ```py
   spark.profile.show()  # show all profile results
   spark.profile.show(1)  # show the profile results for ID = 1
   spark.profile.show(1, type="memory")  # show the memory profile results for ID = 1
   spark.profile.show(type="memory")  # show all memory profile results
   ```



##########
python/pyspark/sql/session.py:
##########
@@ -906,6 +907,12 @@ def dataSource(self) -> "DataSourceRegistration":
 
         return DataSourceRegistration(self)
 
+    @property
+    def profile(self) -> "Profile":

Review Comment:
   Need this for connect, too?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org