Skip to content
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-9393] [SQL] Fix several error-handling bugs in ScriptTransform operator #7710

Closed
wants to merge 12 commits into from

Conversation

JoshRosen
Copy link
Contributor

SparkSQL's ScriptTransform operator has several serious bugs which make debugging fairly difficult:

  • If exceptions are thrown in the writing thread then the child process will not be killed, leading to a deadlock because the reader thread will block while waiting for input that will never arrive.
  • TaskContext is not propagated to the writer thread, which may cause errors in upstream pipelined operators.
  • Exceptions which occur in the writer thread are not propagated to the main reader thread, which may cause upstream errors to be silently ignored instead of killing the job. This can lead to silently incorrect query results.
  • The writer thread is not a daemon thread, but it should be.

In addition, the code in this file is extremely messy:

  • Lots of fields are nullable but the nullability isn't clearly explained.
  • Many confusing variable names: for instance, there are variables named ite and iterator that are defined in the same scope.
  • Some code was misindented.
  • The *serdeClass variables are actually expected to be single-quoted strings, which is really confusing: I feel that this parsing / extraction should be performed in the analyzer, not in the operator itself.
  • There were no unit tests for the operator itself, only end-to-end tests.

This pull request addresses these issues, borrowing some error-handling techniques from PySpark's PythonRDD.

@JoshRosen
Copy link
Contributor Author

/cc @viirya, it would be good to have you review this, given that you wrote a lot of this code. Also, ping @marmbrus who committed much of the original code.

@@ -33,11 +33,13 @@ import scala.util.control.NonFatal
*/
class SparkPlanTest extends SparkFunSuite {

protected def sqlContext: SQLContext = TestSQLContext
Copy link
Contributor Author

Choose a reason for hiding this comment

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

The changes here in SparkPlanTest are to allow it to be used with TestHive without leading to the dreaded "multiple active SparkContexts in the same JVM" error.

@SparkQA
Copy link

SparkQA commented Jul 28, 2015

Test build #38633 has finished for PR 7710 at commit 494cde0.

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

@SparkQA
Copy link

SparkQA commented Jul 28, 2015

Test build #38639 has finished for PR 7710 at commit 6a06a8c.

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

@JoshRosen
Copy link
Contributor Author

Note that making ScriptTransform work well wasn't super high on my list of priorities; I only worked on this because the error-handling problems here made it really hard to debug some failing tests.

@JoshRosen
Copy link
Contributor Author

@davies, it might be good to have you take a look at this given the similarity of this code to PySpark's similar error-handling techniques.

@@ -98,7 +100,7 @@ class SparkPlanTest extends SparkFunSuite {
planFunction: Seq[SparkPlan] => SparkPlan,
expectedAnswer: Seq[Row],
sortAnswers: Boolean = true): Unit = {
SparkPlanTest.checkAnswer(input, planFunction, expectedAnswer, sortAnswers) match {
SparkPlanTest.checkAnswer(input, planFunction, expectedAnswer, sortAnswers, sqlContext) match {
Copy link
Contributor

Choose a reason for hiding this comment

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

If sqlContext is already a member of SparkPlanTest, is it possible that we don't need to pass it to checkAnswer?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is a call on the SparkPlanTest companion object, which doesn't have that field.

Copy link
Contributor

Choose a reason for hiding this comment

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

This is minor, and does not have to be addressed in this PR, but why did we make that an object? It seems like this just complicates function calls since now we can't access fields like the sqlContext.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think that the idea was to allow re-use of some of the SparkPlanTest functions without having to mix them into the test suite; I agree that this could maybe be simplified by making both SparkPlanTest and QueryTest into traits / mixins so that you can easily use both in the same test suite.

@marmbrus
Copy link
Contributor

Thank you for cleaning this up. This was obviously written very quickly in the early days of Spark SQL and it looks a lot cleaner now :)

@SparkQA
Copy link

SparkQA commented Jul 28, 2015

Test build #38714 has finished for PR 7710 at commit 983f200.

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

// Javadoc this call will not throw an exception:
_exception = e
proc.destroy()
throw e
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we check that the task is interrupted or not? Like what we do for PythonRDD.

@davies
Copy link
Contributor

davies commented Jul 28, 2015

This looks good to me. We can merge it now if it blocks you from others, other improvements could be done as follow up PR.

@JoshRosen
Copy link
Contributor Author

@davies, thanks for the review. I'm going to take one final pass to address your latest comments + to update the out-of-date TODOs / copied comments, then will merge to master pending Jenkins.

@JoshRosen
Copy link
Contributor Author

Also, ping @zhichao-li and @chenghao-intel, since it looks like you both attempted to fix this bug in the past (https://issues.apache.org/jira/browse/SPARK-7862). (Just wanted to give you a head's up; not expecting review).

@JoshRosen
Copy link
Contributor Author

@davies, I decided to update the comments but am going to defer addressing your other comments since I think those are longstanding issues in ScriptTransform which are not blockers for me.

@SparkQA
Copy link

SparkQA commented Jul 28, 2015

Test build #38763 has finished for PR 7710 at commit 16c44e2.

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

@JoshRosen
Copy link
Contributor Author

Alright, going to merge this now and will file followup JIRAs so that someone else can investigate the LIMIT corner-case.

@asfgit asfgit closed this in 59b92ad Jul 28, 2015
asfgit pushed a commit that referenced this pull request Jul 30, 2015
This pull request enables Unsafe mode by default in Spark SQL. In order to do this, we had to fix a number of small issues:

**List of fixed blockers**:

- [x] Make some default buffer sizes configurable so that HiveCompatibilitySuite can run properly (#7741).
- [x] Memory leak on grouped aggregation of empty input (fixed by #7560 to fix this)
- [x] Update planner to also check whether codegen is enabled before planning unsafe operators.
- [x] Investigate failing HiveThriftBinaryServerSuite test.  This turns out to be caused by a ClassCastException that occurs when Exchange tries to apply an interpreted RowOrdering to an UnsafeRow when range partitioning an RDD.  This could be fixed by #7408, but a shorter-term fix is to just skip the Unsafe exchange path when RangePartitioner is used.
- [x] Memory leak exceptions masking exceptions that actually caused tasks to fail (will be fixed by #7603).
- [x]  ~~https://issues.apache.org/jira/browse/SPARK-9162, to implement code generation for ScalaUDF.  This is necessary for `UDFSuite` to pass.  For now, I've just ignored this test in order to try to find other problems while we wait for a fix.~~ This is no longer necessary as of #7682.
- [x] Memory leaks from Limit after UnsafeExternalSort cause the memory leak detector to fail tests. This is a huge problem in the HiveCompatibilitySuite (fixed by f4ac642a4e5b2a7931c5e04e086bb10e263b1db6).
- [x] Tests in `AggregationQuerySuite` are failing due to NaN-handling issues in UnsafeRow, which were fixed in #7736.
- [x] `org.apache.spark.sql.ColumnExpressionSuite.rand` needs to be updated so that the planner check also matches `TungstenProject`.
- [x] After having lowered the buffer sizes to 4MB so that most of HiveCompatibilitySuite runs:
  - [x] Wrong answer in `join_1to1` (fixed by #7680)
  - [x] Wrong answer in `join_nulls` (fixed by #7680)
  - [x] Managed memory OOM / leak in `lateral_view`
  - [x] Seems to hang indefinitely in `partcols1`.  This might be a deadlock in script transformation or a bug in error-handling code? The hang was fixed by #7710.
  - [x] Error while freeing memory in `partcols1`: will be fixed by #7734.
- [x] After fixing the `partcols1` hang, it appears that a number of later tests have issues as well.
- [x] Fix thread-safety bug in codegen fallback expression evaluation (#7759).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7564 from JoshRosen/unsafe-by-default and squashes the following commits:

83c0c56 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-by-default
f4cc859 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-by-default
963f567 [Josh Rosen] Reduce buffer size for R tests
d6986de [Josh Rosen] Lower page size in PySpark tests
013b9da [Josh Rosen] Also match TungstenProject in checkNumProjects
5d0b2d3 [Josh Rosen] Add task completion callback to avoid leak in limit after sort
ea250da [Josh Rosen] Disable unsafe Exchange path when RangePartitioning is used
715517b [Josh Rosen] Enable Unsafe by default
@JoshRosen JoshRosen deleted the script-transform branch August 29, 2016 19:28
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.

5 participants