-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-26856][PYSPARK] Python support for from_avro and to_avro APIs #23797
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
0161e50
1733afe
ea67dca
331cfcd
7a8cc33
d524937
dcd7b07
89ff143
65f523b
74aaa6f
f3f0348
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,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. | ||
| # | ||
|
|
||
| __all__ = ['functions'] |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,134 @@ | ||
| # | ||
| # 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 avro functions | ||
| """ | ||
|
|
||
|
|
||
| from pyspark import since, SparkContext | ||
| from pyspark.rdd import ignore_unicode_prefix | ||
| from pyspark.sql.column import Column, _to_java_column | ||
| from pyspark.util import _print_missing_jar | ||
|
|
||
|
|
||
| @ignore_unicode_prefix | ||
| @since(3.0) | ||
| def from_avro(data, jsonFormatSchema, options={}): | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| """ | ||
| Converts a binary column of avro format into its corresponding catalyst value. The specified | ||
| schema must match the read data, otherwise the behavior is undefined: it may fail or return | ||
| arbitrary result. | ||
|
|
||
| Note: Avro is built-in but external data source module since Spark 2.4. Please deploy the | ||
|
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. Could we improve the wording here maybe? "built-in but external" might be a bit confusing. What do you think of something like it's a supported but optional data source that requires special deployment?
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 taken over from the original feature which introduced in 2.4. I think the users already got used to it. If you still think it worth I suggest to modify the original feature as well. |
||
| application as per the deployment section of "Apache Avro Data Source Guide". | ||
|
|
||
| :param data: the binary column. | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| :param jsonFormatSchema: the avro schema in JSON string format. | ||
| :param options: options to control how the Avro record is parsed. | ||
|
|
||
| >>> from pyspark.sql import Row | ||
| >>> from pyspark.sql.avro.functions import from_avro, to_avro | ||
| >>> data = [(1, Row(name='Alice', age=2))] | ||
| >>> df = spark.createDataFrame(data, ("key", "value")) | ||
| >>> avroDf = df.select(to_avro(df.value).alias("avro")) | ||
| >>> avroDf.collect() | ||
| [Row(avro=bytearray(b'\\x00\\x00\\x04\\x00\\nAlice'))] | ||
| >>> jsonFormatSchema = '''{"type":"record","name":"topLevelRecord","fields": | ||
| ... [{"name":"avro","type":[{"type":"record","name":"value","namespace":"topLevelRecord", | ||
| ... "fields":[{"name":"age","type":["long","null"]}, | ||
| ... {"name":"name","type":["string","null"]}]},"null"]}]}''' | ||
| >>> avroDf.select(from_avro(avroDf.avro, jsonFormatSchema).alias("value")).collect() | ||
| [Row(value=Row(avro=Row(age=2, name=u'Alice')))] | ||
| """ | ||
|
|
||
| sc = SparkContext._active_spark_context | ||
| try: | ||
| jc = sc._jvm.org.apache.spark.sql.avro.functions.from_avro( | ||
| _to_java_column(data), jsonFormatSchema, options) | ||
| except TypeError as e: | ||
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| if str(e) == "'JavaPackage' object is not callable": | ||
| _print_missing_jar("Avro", "avro", "avro", sc.version) | ||
| raise | ||
| return Column(jc) | ||
|
|
||
|
|
||
| @ignore_unicode_prefix | ||
| @since(3.0) | ||
| def to_avro(data): | ||
| """ | ||
| Converts a column into binary of avro format. | ||
|
|
||
| Note: Avro is built-in but external data source module since Spark 2.4. Please deploy the | ||
| application as per the deployment section of "Apache Avro Data Source Guide". | ||
|
|
||
| :param data: the data column. | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| >>> from pyspark.sql import Row | ||
| >>> from pyspark.sql.avro.functions import to_avro | ||
| >>> data = [(1, Row(name='Alice', age=2))] | ||
| >>> df = spark.createDataFrame(data, ("key", "value")) | ||
| >>> df.select(to_avro(df.value).alias("avro")).collect() | ||
| [Row(avro=bytearray(b'\\x00\\x00\\x04\\x00\\nAlice'))] | ||
| """ | ||
|
|
||
| sc = SparkContext._active_spark_context | ||
| try: | ||
| jc = sc._jvm.org.apache.spark.sql.avro.functions.to_avro(_to_java_column(data)) | ||
| except TypeError as e: | ||
| if str(e) == "'JavaPackage' object is not callable": | ||
| _print_missing_jar("Avro", "avro", "avro", sc.version) | ||
| raise | ||
| return Column(jc) | ||
|
|
||
|
|
||
| def _test(): | ||
| import os | ||
| import sys | ||
| from pyspark.testing.utils import search_jar | ||
| avro_jar = search_jar("external/avro", "spark-avro") | ||
| if avro_jar is None: | ||
| print( | ||
| "Skipping all Avro Python tests as the optional Avro project was " | ||
| "not compiled into a JAR. To run these tests, " | ||
| "you need to build Spark with 'build/sbt -Pavro package' or " | ||
| "'build/mvn -Pavro package' before running this test.") | ||
| sys.exit(0) | ||
| else: | ||
| existing_args = os.environ.get("PYSPARK_SUBMIT_ARGS", "pyspark-shell") | ||
| jars_args = "--jars %s" % avro_jar | ||
| os.environ["PYSPARK_SUBMIT_ARGS"] = " ".join([jars_args, existing_args]) | ||
|
|
||
| import doctest | ||
| from pyspark.sql import Row, SparkSession | ||
| import pyspark.sql.avro.functions | ||
| globs = pyspark.sql.avro.functions.__dict__.copy() | ||
| spark = SparkSession.builder\ | ||
| .master("local[4]")\ | ||
| .appName("sql.avro.functions tests")\ | ||
| .getOrCreate() | ||
| globs['spark'] = spark | ||
| (failure_count, test_count) = doctest.testmod( | ||
| pyspark.sql.avro.functions, globs=globs, | ||
| optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) | ||
| spark.stop() | ||
| if failure_count: | ||
| sys.exit(-1) | ||
|
|
||
|
|
||
| if __name__ == "__main__": | ||
| _test() | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -14,6 +14,7 @@ | |
| # See the License for the specific language governing permissions and | ||
| # limitations under the License. | ||
| # | ||
| import glob | ||
| import os | ||
| import struct | ||
| import sys | ||
|
|
@@ -100,3 +101,27 @@ def write(self, b): | |
|
|
||
| def close(self): | ||
| pass | ||
|
|
||
|
|
||
| def search_jar(project_relative_path, jar_name_prefix): | ||
| project_full_path = os.path.join( | ||
| os.environ["SPARK_HOME"], project_relative_path) | ||
|
|
||
| # We should ignore the following jars | ||
| ignored_jar_suffixes = ("javadoc.jar", "sources.jar", "test-sources.jar", "tests.jar") | ||
|
|
||
| # Search jar in the project dir using the jar name_prefix for both sbt build and maven | ||
| # build because the artifact jars are in different directories. | ||
| sbt_build = glob.glob(os.path.join( | ||
| project_full_path, "target/scala-*/%s*.jar" % jar_name_prefix)) | ||
|
Member
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. Hi, All. |
||
| maven_build = glob.glob(os.path.join( | ||
| project_full_path, "target/%s*.jar" % jar_name_prefix)) | ||
| jar_paths = sbt_build + maven_build | ||
| jars = [jar for jar in jar_paths if not jar.endswith(ignored_jar_suffixes)] | ||
|
|
||
| if not jars: | ||
| return None | ||
| elif len(jars) > 1: | ||
| raise Exception("Found multiple JARs: %s; please remove all but one" % (", ".join(jars))) | ||
| else: | ||
| return jars[0] | ||
Uh oh!
There was an error while loading. Please reload this page.