Skip to content
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

[SPARK-48560][SS][PYTHON] Make StreamingQueryListener.spark settable #46909

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from all 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
15 changes: 9 additions & 6 deletions python/pyspark/sql/streaming/listener.py
Original file line number Diff line number Diff line change
Expand Up @@ -64,16 +64,19 @@ class StreamingQueryListener(ABC):
"""

def _set_spark_session(
self, spark: "SparkSession" # type: ignore[name-defined] # noqa: F821
self, session: "SparkSession" # type: ignore[name-defined] # noqa: F821
) -> None:
self._sparkSession = spark
if self.spark is None:
self.spark = session

@property
def spark(self) -> Optional["SparkSession"]: # type: ignore[name-defined] # noqa: F821
if hasattr(self, "_sparkSession"):
return self._sparkSession
else:
return None
return getattr(self, "_sparkSession", None)

@spark.setter
def spark(self, session: "SparkSession") -> None: # type: ignore[name-defined] # noqa: F821
# For backward compatibility
self._sparkSession = session

def _init_listener_id(self) -> None:
self._id = str(uuid.uuid4())
Expand Down
17 changes: 17 additions & 0 deletions python/pyspark/sql/tests/streaming/test_streaming_listener.py
Original file line number Diff line number Diff line change
Expand Up @@ -592,6 +592,23 @@ def test_streaming_query_progress_fromJson(self):
self.assertEqual(sink.numOutputRows, -1)
self.assertEqual(sink.metrics, {})

def test_spark_property_in_listener(self):
# SPARK-48560: Make StreamingQueryListener.spark settable
class TestListener(StreamingQueryListener):
def __init__(self, session):
self.spark = session

def onQueryStarted(self, event):
pass

def onQueryProgress(self, event):
pass

def onQueryTerminated(self, event):
pass

self.assertEqual(TestListener(self.spark).spark, self.spark)


if __name__ == "__main__":
import unittest
Expand Down