-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-14131][SQL[STREAMING] Improved fix for avoiding potential deadlocks in HDFSMetadataLog #14292
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-14131][SQL[STREAMING] Improved fix for avoiding potential deadlocks in HDFSMetadataLog #14292
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ import java.util.UUID | |
|
|
||
| import scala.language.implicitConversions | ||
| import scala.util.Try | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.scalatest.BeforeAndAfterAll | ||
|
|
@@ -34,7 +35,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier | |
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.util._ | ||
| import org.apache.spark.sql.execution.FilterExec | ||
| import org.apache.spark.util.Utils | ||
| import org.apache.spark.util.{UninterruptibleThread, Utils} | ||
|
|
||
| /** | ||
| * Helper trait that should be extended by all SQL test suites. | ||
|
|
@@ -247,6 +248,46 @@ private[sql] trait SQLTestUtils | |
| } | ||
| } | ||
| } | ||
|
|
||
| /** Run a test on a separate [[UninterruptibleThread]]. */ | ||
| protected def testWithUninterruptibleThread(name: String, quietly: Boolean = false) | ||
| (body: => Unit): Unit = { | ||
| val timeoutMillis = 10000 | ||
| @transient var ex: Throwable = null | ||
|
|
||
| def runOnThread(): Unit = { | ||
| val thread = new UninterruptibleThread(s"Testing thread for test $name") { | ||
| override def run(): Unit = { | ||
| try { | ||
| body | ||
| } catch { | ||
| case NonFatal(e) => | ||
| ex = e | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will it work?! You're on another thread here and closing over
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. my bad. ex needs to be transient. |
||
| } | ||
| } | ||
| } | ||
| thread.setDaemon(true) | ||
| thread.start() | ||
| thread.join(timeoutMillis) | ||
| if (thread.isAlive) { | ||
| thread.interrupt() | ||
| // If this interrupt does not work, then this thread is most likely running something that | ||
| // is not interruptible. There is not much point to wait for the thread to termniate, and | ||
| // we rather let the JVM terminate the thread on exit. | ||
| fail( | ||
| s"Test '$name' running on o.a.s.util.UninterruptibleThread timed out after" + | ||
| s" $timeoutMillis ms") | ||
| } else if (ex != null) { | ||
| throw ex | ||
| } | ||
| } | ||
|
|
||
| if (quietly) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'd appreciate your comment on the following alternative: ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Its more scala-ish, but slightly non-intuitive to read. Maybe rename f to testingFunc
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it does not compile that easily because test and testQuietly have different param signature, and the code gets complicated trying to make it work. This is hard to understand. So I am keeping it as is.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What about this? |
||
| testQuietly(name) { runOnThread() } | ||
| } else { | ||
| test(name) { runOnThread() } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private[sql] object SQLTestUtils { | ||
|
|
||
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.
Just a single line but takes a while to figure out what it does. I'd rewrite it to:
and would do more transformation depending on the types (didn't check in IDE) Just an idea to untangle the knots :)
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.
You snippet coverts it
Seq[(Source, Option[Offset])]. I find it more tedious to extractSeq[Source, Offset)]from it.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.
Gave it a longer thought. I'm not using for comprehension very often, but when I do...What do you think about this?