XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.6.0, 2.0.0, 2.1.1
    • 2.2.0
    • PySpark, SQL
    • None

    Description

      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

      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())
      

      Example session:

      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'
      

      Proposed

      Delay _judf initialization until the first call.

      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'
      

      Attachments

        Activity

          People

            zero323 Maciej Szymkiewicz
            zero323 Maciej Szymkiewicz
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: