-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-44877][CONNECT][PYTHON] Support python protobuf functions for Spark Connect #42563
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 1 commit
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 |
|---|---|---|
|
|
@@ -271,27 +271,27 @@ def __hash__(self): | |
| ], | ||
| ) | ||
|
|
||
| connect = Module( | ||
|
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. Did you swap the order of
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. yep, I swapped the order to let |
||
| name="connect", | ||
| dependencies=[hive, avro], | ||
| protobuf = Module( | ||
| name="protobuf", | ||
| dependencies=[sql], | ||
| source_file_regexes=[ | ||
| "connector/connect", | ||
| "connector/protobuf", | ||
| ], | ||
| build_profile_flags=["-Pconnect"], | ||
| sbt_test_goals=[ | ||
| "connect/test", | ||
| "connect-client-jvm/test", | ||
| "protobuf/test", | ||
| ], | ||
| ) | ||
|
|
||
| protobuf = Module( | ||
| name="protobuf", | ||
| dependencies=[sql], | ||
| connect = Module( | ||
| name="connect", | ||
| dependencies=[hive, avro, protobuf], | ||
| source_file_regexes=[ | ||
| "connector/protobuf", | ||
| "connector/connect", | ||
| ], | ||
| build_profile_flags=["-Pconnect"], | ||
| sbt_test_goals=[ | ||
| "protobuf/test", | ||
| "connect/test", | ||
| "connect-client-jvm/test", | ||
| ], | ||
| ) | ||
|
|
||
|
|
@@ -832,6 +832,7 @@ def __hash__(self): | |
| "pyspark.sql.connect.dataframe", | ||
| "pyspark.sql.connect.functions", | ||
| "pyspark.sql.connect.avro.functions", | ||
| "pyspark.sql.connect.protobuf.functions", | ||
| "pyspark.sql.connect.streaming.readwriter", | ||
| "pyspark.sql.connect.streaming.query", | ||
| # sql unittests | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,18 @@ | ||
| # | ||
| # 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. | ||
| # | ||
|
|
||
| """Spark Connect Python Client - Protobuf Functions""" |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,176 @@ | ||
| # | ||
| # 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. | ||
| # | ||
|
|
||
| """ | ||
| A collections of builtin protobuf functions | ||
| """ | ||
|
|
||
| from pyspark.sql.connect.utils import check_dependencies | ||
|
|
||
| check_dependencies(__name__) | ||
|
|
||
| from typing import Dict, Optional, TYPE_CHECKING | ||
|
|
||
| from pyspark.sql.protobuf import functions as PyProtobufFunctions | ||
|
|
||
| from pyspark.sql.connect.column import Column | ||
| from pyspark.sql.connect.functions import _invoke_function, _to_col, _options_to_col, lit | ||
|
|
||
| if TYPE_CHECKING: | ||
| from pyspark.sql.connect._typing import ColumnOrName | ||
|
|
||
|
|
||
| def from_protobuf( | ||
| data: "ColumnOrName", | ||
| messageName: str, | ||
| descFilePath: Optional[str] = None, | ||
| options: Optional[Dict[str, str]] = None, | ||
| binaryDescriptorSet: Optional[bytes] = None, | ||
| ) -> Column: | ||
| binary_proto = None | ||
| if binaryDescriptorSet is not None: | ||
| binary_proto = binaryDescriptorSet | ||
| elif descFilePath is not None: | ||
| binary_proto = _read_descriptor_set_file(descFilePath) | ||
|
|
||
| if binary_proto is not None: | ||
| if options is None: | ||
| return _invoke_function( | ||
| "from_protobuf", | ||
| _to_col(data), | ||
| lit(messageName), | ||
| lit(binary_proto) | ||
| ) | ||
| else: | ||
| return _invoke_function( | ||
|
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. So many calls to
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.
|
||
| "from_protobuf", | ||
| _to_col(data), | ||
| lit(messageName), | ||
| lit(binary_proto), | ||
| _options_to_col(options) | ||
| ) | ||
| else: | ||
| if options is None: | ||
| return _invoke_function("from_protobuf", _to_col(data), lit(messageName)) | ||
| else: | ||
| return _invoke_function( | ||
| "from_protobuf", | ||
| _to_col(data), | ||
| lit(messageName), | ||
| _options_to_col(options) | ||
| ) | ||
|
|
||
|
|
||
| from_protobuf.__doc__ = PyProtobufFunctions.from_protobuf.__doc__ | ||
|
|
||
|
|
||
| def to_protobuf( | ||
| data: "ColumnOrName", | ||
| messageName: str, | ||
| descFilePath: Optional[str] = None, | ||
| options: Optional[Dict[str, str]] = None, | ||
| binaryDescriptorSet: Optional[bytes] = None, | ||
| ) -> Column: | ||
| binary_proto = None | ||
| if binaryDescriptorSet is not None: | ||
| binary_proto = binaryDescriptorSet | ||
| elif descFilePath is not None: | ||
| binary_proto = _read_descriptor_set_file(descFilePath) | ||
|
|
||
| if binary_proto is not None: | ||
| if options is None: | ||
| return _invoke_function( | ||
| "to_protobuf", | ||
| _to_col(data), | ||
| lit(messageName), | ||
| lit(binary_proto) | ||
| ) | ||
| else: | ||
| return _invoke_function( | ||
| "to_protobuf", | ||
| _to_col(data), | ||
| lit(messageName), | ||
| lit(binary_proto), | ||
| _options_to_col(options) | ||
| ) | ||
| else: | ||
| if options is None: | ||
| return _invoke_function("to_protobuf", _to_col(data), lit(messageName)) | ||
| else: | ||
| return _invoke_function( | ||
| "to_protobuf", | ||
| _to_col(data), | ||
| lit(messageName), | ||
| _options_to_col(options) | ||
| ) | ||
|
|
||
|
|
||
| to_protobuf.__doc__ = PyProtobufFunctions.to_protobuf.__doc__ | ||
|
|
||
|
|
||
| def _read_descriptor_set_file(filePath: str) -> bytes: | ||
| with open(filePath, "rb") as f: | ||
| return f.read() | ||
|
|
||
|
|
||
| def _test() -> None: | ||
| import os | ||
| import sys | ||
| from pyspark.testing.utils import search_jar | ||
|
|
||
| protobuf_jar = search_jar("connector/protobuf", "spark-protobuf-assembly-", "spark-protobuf") | ||
| if protobuf_jar is None: | ||
| print( | ||
| "Skipping all Protobuf Python tests as the optional Protobuf project was " | ||
| "not compiled into a JAR. To run these tests, " | ||
| "you need to build Spark with 'build/sbt package' or " | ||
| "'build/mvn package' before running this test." | ||
| ) | ||
| sys.exit(0) | ||
|
Comment on lines
+125
to
+133
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. This should not fail like this right?
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. This part is just copied from the original protobuf functions. Should we do it differently here? |
||
| else: | ||
| existing_args = os.environ.get("PYSPARK_SUBMIT_ARGS", "pyspark-shell") | ||
| jars_args = "--jars %s" % protobuf_jar | ||
| os.environ["PYSPARK_SUBMIT_ARGS"] = " ".join([jars_args, existing_args]) | ||
|
|
||
| import doctest | ||
| from pyspark.sql import SparkSession as PySparkSession | ||
| import pyspark.sql.connect.protobuf.functions | ||
|
|
||
| globs = pyspark.sql.connect.protobuf.functions.__dict__.copy() | ||
|
|
||
| globs["spark"] = ( | ||
| PySparkSession.builder.appName("sql.protobuf.functions tests") | ||
| .remote("local[2]") | ||
| .getOrCreate() | ||
| ) | ||
|
|
||
| (failure_count, test_count) = doctest.testmod( | ||
| pyspark.sql.connect.protobuf.functions, | ||
| globs=globs, | ||
| optionflags=doctest.ELLIPSIS | ||
| | doctest.NORMALIZE_WHITESPACE | ||
| | doctest.IGNORE_EXCEPTION_DETAIL, | ||
| ) | ||
|
|
||
| globs["spark"].stop() | ||
|
|
||
| if failure_count: | ||
| sys.exit(-1) | ||
|
|
||
|
|
||
| if __name__ == "__main__": | ||
| _test() | ||
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.
This is needed to avoid
java.lang.NoClassDefFoundError: org/apache/spark/sql/protobuf/CatalystDataToProtobuf. Similar issue was also mentioned in Avro support PR.