-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-4813][Streaming] Fix the issue that ContextWaiter didn't handle 'spurious wakeup' #3661
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
Changes from 1 commit
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 |
|---|---|---|
|
|
@@ -17,30 +17,74 @@ | |
|
|
||
| package org.apache.spark.streaming | ||
|
|
||
| import java.util.concurrent.{TimeoutException, TimeUnit} | ||
| import java.util.concurrent.locks.ReentrantLock | ||
| import javax.annotation.concurrent.GuardedBy | ||
|
|
||
| private[streaming] class ContextWaiter { | ||
|
|
||
| private val lock = new ReentrantLock() | ||
| private val condition = lock.newCondition() | ||
|
|
||
| @GuardedBy("lock") | ||
| private var error: Throwable = null | ||
|
|
||
| @GuardedBy("lock") | ||
| private var stopped: Boolean = false | ||
|
|
||
| def notifyError(e: Throwable) = synchronized { | ||
| error = e | ||
| notifyAll() | ||
| def notifyError(e: Throwable) = { | ||
|
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. Minor nit, but mind adding |
||
| lock.lock() | ||
| try { | ||
| error = e | ||
| condition.signalAll() | ||
| } finally { | ||
| lock.unlock() | ||
| } | ||
| } | ||
|
|
||
| def notifyStop() = synchronized { | ||
| stopped = true | ||
| notifyAll() | ||
| def notifyStop() = { | ||
|
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. Same here: could this have an explicit |
||
| lock.lock() | ||
| try { | ||
| stopped = true | ||
| condition.signalAll() | ||
| } finally { | ||
| lock.unlock() | ||
| } | ||
| } | ||
|
|
||
| def waitForStopOrError(timeout: Long = -1) = synchronized { | ||
| // If already had error, then throw it | ||
| if (error != null) { | ||
| throw error | ||
| } | ||
| /** | ||
| * Return `true` if it's stopped; or throw the reported error if `notifyError` has been called; or | ||
| * `false` if the waiting time detectably elapsed before return from the method. | ||
| */ | ||
| def waitForStopOrError(timeout: Long = -1): Boolean = { | ||
|
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 considered whether this should throw Also, it looks like the usages of this in /**
* Wait for the execution to stop. Any exceptions that occurs during the execution
* will be thrown in this thread.
*/
def awaitTermination() {
waiter.waitForStopOrError()
}
/**
* Wait for the execution to stop. Any exceptions that occurs during the execution
* will be thrown in this thread.
* @param timeout time to wait in milliseconds
*/
def awaitTermination(timeout: Long) {
waiter.waitForStopOrError(timeout)
}
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 suppose the documentation for the
Member
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. Do you think if it's better to return
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. Another option would be to just add a new
Member
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. Agreed. Done.
Member
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. How about making
Member
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. @tdas what do you think?
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 think that will break a lot of existing programs for people. A lot of time (even in our own test cases)
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. In hindsight, instead of modeling
Member
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.
@tdas, Sorry that I forgot to reply you. You said you designed it just like Akka /**
* Block current thread until the system has been shutdown, or the specified
* timeout has elapsed. This will block until after all on termination
* callbacks have been run.
*
* @throws TimeoutException in case of timeout
*/
def awaitTermination(timeout: Duration): Unit |
||
| lock.lock() | ||
| try { | ||
| if (timeout < 0) { | ||
| while (true) { | ||
|
Member
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. Maybe it's just me but it feels like these loops would be simpler just testing
Member
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's cleaner now. |
||
| // If already stopped, then exit | ||
| if (stopped) return true | ||
| // If already had error, then throw it | ||
| if (error != null) throw error | ||
|
|
||
| condition.await() | ||
| } | ||
| } else { | ||
| var nanos = TimeUnit.MILLISECONDS.toNanos(timeout) | ||
| while (true) { | ||
| // If already stopped, then exit | ||
| if (stopped) return true | ||
| // If already had error, then throw it | ||
| if (error != null) throw error | ||
| // If no time remains, then exit | ||
| if (nanos <= 0) return false | ||
|
|
||
| // If not already stopped, then wait | ||
| if (!stopped) { | ||
| if (timeout < 0) wait() else wait(timeout) | ||
| if (error != null) throw error | ||
| nanos = condition.awaitNanos(nanos) | ||
| } | ||
| } | ||
| // Never reached. Make the compiler happy. | ||
| true | ||
| } finally { | ||
| lock.unlock() | ||
| } | ||
| } | ||
| } | ||
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.
Minor point - these are not in the JDK but in a Findbugs library for JSR-305. It's not used in Spark, and happens to be a dependency now. Maybe not worth using it just 1 place?
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.
So which one do you prefer?
GuardedByfrom now on.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.
In addition, now Findbugs does not recognize
GuardedByin Scala codes.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.
BTW, I turned to
GuardedBybecause @aarondav asked me to do it in #3634There 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.
Yes, that's why I brought it up. It's not actually a standard Java annotation (unless someone tells me it just turned up in 8 or something) but part of JSR-305. This is a dependency of Spark core at the moment, but none of the annotations are used. I think we should just not use them instead of using this lib in 1 place.
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.
OK. I changed them to comments.
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.
In general, I'm a fan of the FindBugs annotations. I've had trouble getting the various analysis tools to work well with them in Scala, though. +1 to this commenting convention, though; this is very helpful.