-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19163][PYTHON][SQL] Delay _judf initialization to the __call__ #16536
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
22b967c
Delay _judf initialization to the __call__
zero323 7b252c6
Move name resolution logic from _create_judf to __init__
zero323 fb009e0
Use getOrCreate instead of _active_spark_context
zero323 489ef54
Add UDFInitializationTests
zero323 7333f29
Add comment describing multithreaded initalization
zero323 cb496f3
Use _active_spark_context in place of getOrCreate
zero323 9332da3
Add a comment explaining the purpose of _judf_placeholder
zero323 923b88d
Use spark.sparkContext instead of separate _judf
zero323 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -468,6 +468,27 @@ def filename(path): | |
| row2 = df2.select(sameText(df2['file'])).first() | ||
| self.assertTrue(row2[0].find("people.json") != -1) | ||
|
|
||
| def test_udf_defers_judf_initalization(self): | ||
| # This is separate of UDFInitializationTests | ||
| # to avoid context initialization | ||
| # when udf is called | ||
|
|
||
| from pyspark.sql.functions import UserDefinedFunction | ||
|
|
||
| f = UserDefinedFunction(lambda x: x, StringType()) | ||
|
|
||
| self.assertIsNone( | ||
| f._judf_placeholder, | ||
| "judf should not be initialized before the first call." | ||
| ) | ||
|
|
||
| self.assertIsInstance(f("foo"), Column, "UDF call should return a Column.") | ||
|
|
||
| self.assertIsNotNone( | ||
| f._judf_placeholder, | ||
| "judf should be initialized after UDF has been called." | ||
| ) | ||
|
|
||
| def test_basic_functions(self): | ||
| rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) | ||
| df = self.spark.read.json(rdd) | ||
|
|
@@ -1947,6 +1968,29 @@ def test_sparksession_with_stopped_sparkcontext(self): | |
| df.collect() | ||
|
|
||
|
|
||
| class UDFInitializationTests(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_init_shouldnt_initalize_context(self): | ||
| from pyspark.sql.functions import UserDefinedFunction | ||
|
|
||
| UserDefinedFunction(lambda x: x, StringType()) | ||
|
|
||
| self.assertIsNone( | ||
| SparkContext._active_spark_context, | ||
| "SparkContext shouldn't be initialized when UserDefinedFunction is created." | ||
| ) | ||
| self.assertIsNone( | ||
| SparkSession._instantiatedSession, | ||
| "SparkSession shouldn't be initialized when UserDefinedFunction is created." | ||
| ) | ||
|
|
||
|
|
||
| class HiveContextSQLTests(ReusedPySparkTestCase): | ||
|
|
||
| @classmethod | ||
|
|
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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
TestCaseI 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.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
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
_judfis initialized when necessary.