[ https://issues.apache.org/jira/browse/SPARK-19163?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15816318#comment-15816318 ]
Apache Spark commented on SPARK-19163: -------------------------------------- User 'zero323' has created a pull request for this issue: https://github.com/apache/spark/pull/16536 > 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: 1.6.0, 2.0.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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org