-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-54020] Support spark.sql(...) Python API inside query functions for Spark Declarative Pipeline
#53024
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
Changes from all commits
e515b85
c6b88d0
0641991
a5a3fbb
506dcdd
4a74125
56d5ece
5ffe3a3
de38572
6a7d66f
5c214d7
433b537
887ee0a
e483cb0
8da65ce
fe6f1cd
435ff6c
5b43ba0
7ab17fa
314d69e
b67b486
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,48 @@ | ||
| # | ||
| # Licensed to the Apache Software Foundation (ASF) under one or more | ||
| # contributor license agreements. See the NOTICE file distributed with | ||
| # this work for additional information regarding copyright ownership. | ||
| # The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| # (the "License"); you may not use this file except in compliance with | ||
| # the License. You may obtain a copy of the License at | ||
| # | ||
| # http://www.apache.org/licenses/LICENSE-2.0 | ||
| # | ||
| # Unless required by applicable law or agreed to in writing, software | ||
| # distributed under the License is distributed on an "AS IS" BASIS, | ||
| # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| # See the License for the specific language governing permissions and | ||
| # limitations under the License. | ||
| # | ||
| from contextlib import contextmanager | ||
| from typing import Generator, Optional | ||
| from pyspark.sql import SparkSession | ||
|
|
||
| from typing import Any, cast | ||
|
|
||
|
|
||
| @contextmanager | ||
| def add_pipeline_analysis_context( | ||
| spark: SparkSession, dataflow_graph_id: str, flow_name: Optional[str] | ||
| ) -> Generator[None, None, None]: | ||
| """ | ||
| Context manager that add PipelineAnalysisContext extension to the user context | ||
| used for pipeline specific analysis. | ||
| """ | ||
| extension_id = None | ||
| # Cast because mypy seems to think `spark` is a function, not an object. | ||
| # Likely related to SPARK-47544. | ||
| client = cast(Any, spark).client | ||
| try: | ||
| import pyspark.sql.connect.proto as pb2 | ||
| from google.protobuf import any_pb2 | ||
|
|
||
| analysis_context = pb2.PipelineAnalysisContext( | ||
| dataflow_graph_id=dataflow_graph_id, flow_name=flow_name | ||
| ) | ||
| extension = any_pb2.Any() | ||
| extension.Pack(analysis_context) | ||
| extension_id = client.add_threadlocal_user_context_extension(extension) | ||
| yield | ||
| finally: | ||
| client.remove_user_context_extension(extension_id) |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,6 +35,7 @@ | |
| from pyspark.sql.types import StructType | ||
| from typing import Any, cast | ||
| import pyspark.sql.connect.proto as pb2 | ||
| from pyspark.pipelines.add_pipeline_analysis_context import add_pipeline_analysis_context | ||
|
|
||
|
|
||
| class SparkConnectGraphElementRegistry(GraphElementRegistry): | ||
|
|
@@ -43,6 +44,7 @@ class SparkConnectGraphElementRegistry(GraphElementRegistry): | |
| def __init__(self, spark: SparkSession, dataflow_graph_id: str) -> None: | ||
| # Cast because mypy seems to think `spark`` is a function, not an object. Likely related to | ||
| # SPARK-47544. | ||
| self._spark = spark | ||
| self._client = cast(Any, spark).client | ||
| self._dataflow_graph_id = dataflow_graph_id | ||
|
|
||
|
|
@@ -110,8 +112,11 @@ def register_output(self, output: Output) -> None: | |
| self._client.execute_command(command) | ||
|
|
||
| def register_flow(self, flow: Flow) -> None: | ||
| with block_spark_connect_execution_and_analysis(): | ||
| df = flow.func() | ||
| with add_pipeline_analysis_context( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If I understand correctly, this will result in two PipelineAnalysisContexts added to the same request. And the server code knows to expect that.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes! Exactly, for spark.sql() outside query function, it would only have one extension associated with it, but two for spark.sql() inside query function |
||
| spark=self._spark, dataflow_graph_id=self._dataflow_graph_id, flow_name=flow.name | ||
| ): | ||
| with block_spark_connect_execution_and_analysis(): | ||
| df = flow.func() | ||
| relation = cast(ConnectDataFrame, df)._plan.plan(self._client) | ||
|
|
||
| relation_flow_details = pb2.PipelineCommand.DefineFlow.WriteRelationFlowDetails( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,100 @@ | ||
| # | ||
| # Licensed to the Apache Software Foundation (ASF) under one or more | ||
| # contributor license agreements. See the NOTICE file distributed with | ||
| # this work for additional information regarding copyright ownership. | ||
| # The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| # (the "License"); you may not use this file except in compliance with | ||
| # the License. You may obtain a copy of the License at | ||
| # | ||
| # http://www.apache.org/licenses/LICENSE-2.0 | ||
| # | ||
| # Unless required by applicable law or agreed to in writing, software | ||
| # distributed under the License is distributed on an "AS IS" BASIS, | ||
| # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| # See the License for the specific language governing permissions and | ||
| # limitations under the License. | ||
| # | ||
| import unittest | ||
|
|
||
| from pyspark.testing.connectutils import ( | ||
| ReusedConnectTestCase, | ||
| should_test_connect, | ||
| connect_requirement_message, | ||
| ) | ||
|
|
||
| if should_test_connect: | ||
| from pyspark.pipelines.add_pipeline_analysis_context import add_pipeline_analysis_context | ||
|
|
||
|
|
||
| @unittest.skipIf(not should_test_connect, connect_requirement_message) | ||
| class AddPipelineAnalysisContextTests(ReusedConnectTestCase): | ||
| def test_add_pipeline_analysis_context_with_flow_name(self): | ||
| with add_pipeline_analysis_context(self.spark, "test_dataflow_graph_id", "test_flow_name"): | ||
| import pyspark.sql.connect.proto as pb2 | ||
|
|
||
| thread_local_extensions = self.spark.client.thread_local.user_context_extensions | ||
| self.assertEqual(len(thread_local_extensions), 1) | ||
| # Extension is stored as (id, extension), unpack the extension | ||
| _extension_id, extension = thread_local_extensions[0] | ||
| context = pb2.PipelineAnalysisContext() | ||
| extension.Unpack(context) | ||
| self.assertEqual(context.dataflow_graph_id, "test_dataflow_graph_id") | ||
| self.assertEqual(context.flow_name, "test_flow_name") | ||
| thread_local_extensions_after = self.spark.client.thread_local.user_context_extensions | ||
| self.assertEqual(len(thread_local_extensions_after), 0) | ||
|
|
||
| def test_add_pipeline_analysis_context_without_flow_name(self): | ||
| with add_pipeline_analysis_context(self.spark, "test_dataflow_graph_id", None): | ||
| import pyspark.sql.connect.proto as pb2 | ||
|
|
||
| thread_local_extensions = self.spark.client.thread_local.user_context_extensions | ||
| self.assertEqual(len(thread_local_extensions), 1) | ||
| # Extension is stored as (id, extension), unpack the extension | ||
| _extension_id, extension = thread_local_extensions[0] | ||
| context = pb2.PipelineAnalysisContext() | ||
| extension.Unpack(context) | ||
| self.assertEqual(context.dataflow_graph_id, "test_dataflow_graph_id") | ||
| # Empty string means no flow name | ||
| self.assertEqual(context.flow_name, "") | ||
| thread_local_extensions_after = self.spark.client.thread_local.user_context_extensions | ||
| self.assertEqual(len(thread_local_extensions_after), 0) | ||
|
|
||
| def test_nested_add_pipeline_analysis_context(self): | ||
| import pyspark.sql.connect.proto as pb2 | ||
|
|
||
| with add_pipeline_analysis_context(self.spark, "test_dataflow_graph_id_1", flow_name=None): | ||
| with add_pipeline_analysis_context( | ||
| self.spark, "test_dataflow_graph_id_2", flow_name="test_flow_name" | ||
| ): | ||
| thread_local_extensions = self.spark.client.thread_local.user_context_extensions | ||
| self.assertEqual(len(thread_local_extensions), 2) | ||
| # Extension is stored as (id, extension), unpack the extensions | ||
| _, extension_1 = thread_local_extensions[0] | ||
| context_1 = pb2.PipelineAnalysisContext() | ||
| extension_1.Unpack(context_1) | ||
| self.assertEqual(context_1.dataflow_graph_id, "test_dataflow_graph_id_1") | ||
| self.assertEqual(context_1.flow_name, "") | ||
| _, extension_2 = thread_local_extensions[1] | ||
| context_2 = pb2.PipelineAnalysisContext() | ||
| extension_2.Unpack(context_2) | ||
| self.assertEqual(context_2.dataflow_graph_id, "test_dataflow_graph_id_2") | ||
| self.assertEqual(context_2.flow_name, "test_flow_name") | ||
| thread_local_extensions_after_1 = self.spark.client.thread_local.user_context_extensions | ||
| self.assertEqual(len(thread_local_extensions_after_1), 1) | ||
| _, extension_3 = thread_local_extensions_after_1[0] | ||
| context_3 = pb2.PipelineAnalysisContext() | ||
| extension_3.Unpack(context_3) | ||
| self.assertEqual(context_3.dataflow_graph_id, "test_dataflow_graph_id_1") | ||
| self.assertEqual(context_3.flow_name, "") | ||
| thread_local_extensions_after_2 = self.spark.client.thread_local.user_context_extensions | ||
| self.assertEqual(len(thread_local_extensions_after_2), 0) | ||
|
|
||
|
|
||
| if __name__ == "__main__": | ||
| try: | ||
| import xmlrunner # type: ignore | ||
|
|
||
| testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) | ||
| except ImportError: | ||
| testRunner = None | ||
| unittest.main(testRunner=testRunner, verbosity=2) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Now that we have the context on the server side, it might make more sense to block these operations there – then we don't need to replicate this weird monkeypatching logic across all the clients when we add support for other languages. Doesn't need to be part of this PR though.