Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16536#discussion_r98801781
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -1826,23 +1826,35 @@ class UserDefinedFunction(object):
         def __init__(self, func, returnType, name=None):
             self.func = func
             self.returnType = returnType
    -        self._judf = self._create_judf(name)
    -
    -    def _create_judf(self, name):
    +        self._judf_placeholder = None
    +        self._name = name or (
    +            func.__name__ if hasattr(func, '__name__')
    +            else func.__class__.__name__)
    +
    +    @property
    +    def _judf(self):
    +        # It is possible that concurent access, to newly created UDF,
    +        # will initalize multiple UserDefinedPythonFunctions.
    +        # This is unlikely, doesn't affect correctness,
    +        # and should have a minimal performance impact.
    +        if self._judf_placeholder is None:
    +            self._judf_placeholder = self._create_judf()
    +        return self._judf_placeholder
    +
    +    def _create_judf(self):
             from pyspark.sql import SparkSession
    +
             sc = SparkContext.getOrCreate()
    -        wrapped_func = _wrap_function(sc, self.func, self.returnType)
             spark = SparkSession.builder.getOrCreate()
    +
    +        wrapped_func = _wrap_function(sc, self.func, self.returnType)
             jdt = spark._jsparkSession.parseDataType(self.returnType.json())
    -        if name is None:
    -            f = self.func
    -            name = f.__name__ if hasattr(f, '__name__') else 
f.__class__.__name__
             judf = 
sc._jvm.org.apache.spark.sql.execution.python.UserDefinedPythonFunction(
    -            name, wrapped_func, jdt)
    +            self._name, wrapped_func, jdt)
             return judf
     
         def __call__(self, *cols):
    -        sc = SparkContext._active_spark_context
    +        sc = SparkContext.getOrCreate()
    --- End diff --
    
    So by switching this to `getOrCreate` we put a lock acquisition in the path 
of `__call__` which is maybe not ideal. We could maybe fix this by getting 
`_judf` first (e.g. `judf = self._judf`)? (Although it should be a mostly 
uncontended lock so it shouldn't be that bad, but if we ended up having a 
multi-threaded PySpark DataFrame UDF application this could maybe degrade a 
little bit).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to