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
SPARK-18368: Fix regexp_replace with task serialization. #15816
Conversation
Test build #68373 has finished for PR 15816 at commit
|
Wow I am actually surprised that this wasn't caught before. Integration testing should have uncovered this. Could you add one? |
92980fc
to
3536f6a
Compare
Sorry I missed your update. |
@@ -191,4 +192,17 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { | |||
checkEvaluation(StringSplit(s1, s2), null, row3) | |||
} | |||
|
|||
test("RegExpReplace serialization") { | |||
val serializer = new JavaSerializer(new SparkConf()).newInstance |
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.
Maybe we should add a similar test to ExpressionEvalHelper
.
@hvanhovell, I updated the test so that all expressions passed through |
Test build #68369 has finished for PR 15816 at commit
|
Test build #68377 has finished for PR 15816 at commit
|
Test build #68375 has finished for PR 15816 at commit
|
## What changes were proposed in this pull request? This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized. ## How was this patch tested? * Verified that this patch fixed the query that found the bug. * Added a test case that fails without the fix. Author: Ryan Blue <blue@apache.org> Closes #15816 from rdblue/SPARK-18368-fix-regexp-replace. (cherry picked from commit b9192bb) Signed-off-by: Reynold Xin <rxin@databricks.com>
Merging in master/branch-2.1/branch-2.0. |
## What changes were proposed in this pull request? This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized. ## How was this patch tested? * Verified that this patch fixed the query that found the bug. * Added a test case that fails without the fix. Author: Ryan Blue <blue@apache.org> Closes #15816 from rdblue/SPARK-18368-fix-regexp-replace. (cherry picked from commit b9192bb) Signed-off-by: Reynold Xin <rxin@databricks.com>
I'm surprised too that we haven't caught this earlier ... |
Thanks @rxin and @hvanhovell! I appreciate the quick reviews. |
I am wondering if it breaks some tests?
https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.6/1952/ |
Reverted from master/branch-2.1/branch-2.0. |
@rdblue Can you send a new pr with the fix? Thanks! |
|
## What changes were proposed in this pull request? This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized. ## How was this patch tested? * Verified that this patch fixed the query that found the bug. * Added a test case that fails without the fix. Author: Ryan Blue <blue@apache.org> Closes apache#15816 from rdblue/SPARK-18368-fix-regexp-replace.
What changes were proposed in this pull request?
This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized,
result: StringBuffer
will be correctly initialized.How was this patch tested?