Skip to content

Conversation

@zero323
Copy link
Member

@zero323 zero323 commented Jan 10, 2017

What changes were proposed in this pull request?

Defer UserDefinedFunction._judf initialization to the first call. This prevents unintended SparkSession initialization. This allows users to define and import UDF without creating a context / session as a side effect.

SPARK-19163

How was this patch tested?

Unit tests.

@SparkQA
Copy link

SparkQA commented Jan 10, 2017

Test build #71163 has finished for PR 16536 at commit 2ca2557.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 12, 2017

Test build #71254 has finished for PR 16536 at commit c936813.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zero323 zero323 changed the title [SPARK-19163][PYTHON][SQL][WIP] Delay _judf initialization to the __call__ [SPARK-19163][PYTHON][SQL]Delay _judf initialization to the __call__ Jan 12, 2017
@zero323 zero323 changed the title [SPARK-19163][PYTHON][SQL]Delay _judf initialization to the __call__ [SPARK-19163][PYTHON][SQL] Delay _judf initialization to the __call__ Jan 12, 2017
@SparkQA
Copy link

SparkQA commented Jan 13, 2017

Test build #71351 has finished for PR 16536 at commit 641ec1d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 13, 2017

Test build #71350 has finished for PR 16536 at commit b124bb4.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 19, 2017

Test build #71679 has finished for PR 16536 at commit 57735b2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 20, 2017

Test build #71688 has finished for PR 16536 at commit abb3726.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@rdblue
Copy link
Contributor

rdblue commented Jan 20, 2017

+1

Looks good to me.

Copy link
Contributor

@holdenk holdenk left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for working on this! I think this is going to be useful for Python UDF libraries. I've got a few questions - let me know what your thoughts are :)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe add a comment explaining the purposes of this, just for future readers of the code.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems like a good test but maybe a bit too focused on testing the implementation specifics?

Maybe it might more sense to also have a test which verifies creating a UDF doesn't create a SparkSession since that is the intended purposes (we don't really care about delaying the initialization of _judfy that much per-se but we do care about verifying that we don't eagerly create the SparkSession on import). What do you think?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I thought about it but I have this impression, maybe incorrect, that we avoid creating new contexts to keep total execution time manageable. If you think this justifies a separate TestCase I am more than fine with that (SPARK-19224 and [PYSPARK] Python tests organization , right?).

If not, we could mock this, and put assert on the number of calls.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think a seperate test case and would able to be pretty light weight since it doesn't need to create a SparkContext or anything which traditionally takes longer to set up. What do you think?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@holdenk Separate case it is. As long as implementation is correct an overhead is negligible.

Copy link
Member Author

@zero323 zero323 Jan 30, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's keep these tests, to make sure that _judf is initialized when necessary.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

there is a assertIsInstance function that could simplify this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So there isn't any lock around this - I suspect we aren't too likely to have concurrent calls to this - but just to be safe we should maybe think through what would happen if this is does happen (and then leave a comment about it)?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you elaborate a bit? I am not sure if I understand the issue.

Assignment is atomic (so we don't have to worry about corruption), for any practical purpose operation is idempotent (we can return expressions using different Java objects but as far as I am concerned this is just a detail of implementation), access to Py4J is thread safe and as far as I remember function registries are synchronized. I there any issue i missed here?

Thanks for looking into this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think @holdenk's concern is that this would allow concurrent calls to _create_udf. That would create two UserDefinedPythonFunction objects, but I don't see anything on the Scala side that is concerning about that.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@rdblue I get this part, and this is a possible scenario. Question is if this justifies preventive lock. As far as I am aware there should be no correctness issues here. SparkSession already locks during initialization so we are safe there.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I don't think it should require a lock. I think concurrent calls are very unlikely and safe.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I stress tested this a bit and I haven't found any abnormalities but I found a small problem with __call__ on the way. Fixed now.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, I'd maybe just leave a comment saying that we've left out the lock since double creation is both unlikely and OK.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@holdenk Done.

@SparkQA
Copy link

SparkQA commented Jan 26, 2017

Test build #72021 has finished for PR 16536 at commit 16245e4.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 26, 2017

Test build #72034 has finished for PR 16536 at commit a5567b2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 26, 2017

Test build #72039 has finished for PR 16536 at commit 29ffc57.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@holdenk I believe that for a full test udf would have to create SparkContext. But mock is cheap.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can create a testcase without the Spark base and verify that creating a UDF doesn't create a SparkContext. This does not require making a SparkContext.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@holdenk Do you mean something like SparkContext._active_spark_context is None? Then we need to make sure it is tear down if it was initialized after all, right? Isn't mocking cleaner?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

class UDFInitializationTestCase(unittest.TestCase):
    def tearDown(self):
        if SparkSession._instantiatedSession is not None:
            SparkSession._instantiatedSession.stop()

        if SparkContext._active_spark_context is not None:
            SparkContext._active_spark_contex.stop()

    def test_udf_context_access(self):
        from pyspark.sql.functions import UserDefinedFunction
        f = UserDefinedFunction(lambda x: x, StringType())
        self.assertIsNone(SparkContext._active_spark_context)
        self.assertIsNone(SparkSession._instantiatedSession)
            

Copy link
Member Author

@zero323 zero323 Jan 30, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And add a separate test case checking SparkContext and SparkSession state.

@SparkQA
Copy link

SparkQA commented Jan 30, 2017

Test build #72170 has finished for PR 16536 at commit ee953a9.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class UDFInitializationTests(unittest.TestCase):

@SparkQA
Copy link

SparkQA commented Jan 30, 2017

Test build #72172 has finished for PR 16536 at commit 489ef54.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class UDFInitializationTests(unittest.TestCase):

@SparkQA
Copy link

SparkQA commented Jan 30, 2017

Test build #72173 has finished for PR 16536 at commit 1a8280d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@holdenk
Copy link
Contributor

holdenk commented Jan 30, 2017

The changes look good to me, I'll take a quick pass at the formatting to make sure - but otherwise I'll try and merge this tomorrow :)

Copy link
Contributor

@holdenk holdenk left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok it looks good, just one minor comment about having getOrCreate which acquires a lock in the hot path for __call__. Thanks for adding the tests :)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@holdenk Sounds reasonable.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Though I am not sure if it really matters here. If _instantiatedContext is not None we'll do the same thing, otherwise we fall back to initialization in _judf.

@zero323
Copy link
Member Author

zero323 commented Feb 1, 2017

@holdenk I have one more suggestion. Shouldn't we replace

    def _create_judf(self):
        from pyspark.sql import SparkSession

        sc = SparkContext.getOrCreate()
        spark = SparkSession.builder.getOrCreate()

with

    def _create_judf(self):
        from pyspark.sql import SparkSession

        spark = SparkSession.builder.getOrCreate()
        sc = spark.sparkContext

I left it as is but I think it could be cleaner.

@SparkQA
Copy link

SparkQA commented Feb 1, 2017

Test build #72218 has finished for PR 16536 at commit cb496f3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 1, 2017

Test build #72219 has finished for PR 16536 at commit 9332da3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@holdenk
Copy link
Contributor

holdenk commented Feb 1, 2017

@zero323 that sounds like a good improvement.

@zero323
Copy link
Member Author

zero323 commented Feb 1, 2017

@holdenk Done :)

@holdenk
Copy link
Contributor

holdenk commented Feb 1, 2017

Great, I'll wait for jenkins then :)

@SparkQA
Copy link

SparkQA commented Feb 1, 2017

Test build #72222 has finished for PR 16536 at commit 923b88d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@holdenk
Copy link
Contributor

holdenk commented Feb 1, 2017

Going to go ahead and merge. Still need to sort out the JIRA permissions so will take a bit for me to get that updated for you.

@asfgit asfgit closed this in 9063835 Feb 1, 2017
@zero323
Copy link
Member Author

zero323 commented Feb 1, 2017

Thanks a bunch @holdenk

@zero323 zero323 deleted the SPARK-19163 branch February 1, 2017 11:17
cmonkey pushed a commit to cmonkey/spark that referenced this pull request Feb 15, 2017
## What changes were proposed in this pull request?

Defer `UserDefinedFunction._judf` initialization to the first call. This prevents unintended `SparkSession` initialization.  This allows users to define and import UDF without creating a context / session as a side effect.

[SPARK-19163](https://issues.apache.org/jira/browse/SPARK-19163)

## How was this patch tested?

Unit tests.

Author: zero323 <[email protected]>

Closes apache#16536 from zero323/SPARK-19163.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants