You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Maciej Szymkiewicz (JIRA)" <ji...@apache.org> on 2017/01/10 20:43:58 UTC
[jira] [Created] (SPARK-19163) Lazy creation of the _judf
Maciej Szymkiewicz created SPARK-19163:
------------------------------------------
Summary: Lazy creation of the _judf
Key: SPARK-19163
URL: https://issues.apache.org/jira/browse/SPARK-19163
Project: Spark
Issue Type: Sub-task
Components: PySpark, SQL
Affects Versions: 2.0.0, 1.6.0, 1.5.0, 2.1.1
Reporter: Maciej Szymkiewicz
Current state
Right {{UserDefinedFunction}} eagerly creates {{_judf}} and initializes {{SparkSession}} (https://github.com/apache/spark/blob/master/python/pyspark/sql/functions.py#L1832) as a side effect. This behavior may have undesired results when {{udf}} is imported from a module:
{{myudfs.py}}
{code}
from pyspark.sql.functions import udf
from pyspark.sql.types import IntegerType
def _add_one(x):
"""Adds one"""
if x is not None:
return x + 1
add_one = udf(_add_one, IntegerType())
{code}
Example session:
{code}
In [1]: from pyspark.sql import SparkSession
In [2]: from myudfs import add_one
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
17/01/07 19:55:44 WARN Utils: Your hostname, xxx resolves to a loopback address: 127.0.1.1; using xxx instead (on interface eth0)
17/01/07 19:55:44 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
In [3]: spark = SparkSession.builder.appName("foo").getOrCreate()
In [4]: spark.sparkContext.appName
Out[4]: 'pyspark-shell'
{code}
Proposed
Delay {{_judf}} initialization until the first call.
{code}
In [1]: from pyspark.sql import SparkSession
In [2]: from myudfs import add_one
In [3]: spark = SparkSession.builder.appName("foo").getOrCreate()
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
17/01/07 19:58:38 WARN Utils: Your hostname, xxx resolves to a loopback address: 127.0.1.1; using xxx instead (on interface eth0)
17/01/07 19:58:38 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
In [4]: spark.sparkContext.appName
Out[4]: 'foo'
{code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org