Github user zsxwing commented on a diff in the pull request:
https://github.com/apache/spark/pull/3661#discussion_r23372367
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala ---
@@ -17,30 +17,63 @@
package org.apache.spark.streaming
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.locks.ReentrantLock
+
private[streaming] class ContextWaiter {
+
+ private val lock = new ReentrantLock()
+ private val condition = lock.newCondition()
+
+ // Guarded by "lock"
private var error: Throwable = null
- private var stopped: Boolean = false
- def notifyError(e: Throwable) = synchronized {
- error = e
- notifyAll()
- }
+ // Guarded by "lock"
+ private var stopped: Boolean = false
- def notifyStop() = synchronized {
- stopped = true
- notifyAll()
+ def notifyError(e: Throwable): Unit = {
+ lock.lock()
+ try {
+ error = e
+ condition.signalAll()
+ } finally {
+ lock.unlock()
+ }
}
- def waitForStopOrError(timeout: Long = -1) = synchronized {
- // If already had error, then throw it
- if (error != null) {
- throw error
+ def notifyStop(): Unit = {
+ lock.lock()
+ try {
+ stopped = true
+ condition.signalAll()
+ } finally {
+ lock.unlock()
}
+ }
- // If not already stopped, then wait
- if (!stopped) {
- if (timeout < 0) wait() else wait(timeout)
+ /**
+ * 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 = {
--- End diff --
> In hindsight, instead of modeling awaitTermination against Akka
ActorSystem's awaitTermination (which return Unit) , I should have modeled it
like Java ExecutorService's awaitTermination which returns a Boolean. Now its
not possible to change the API without breaking compatiblity. :(
@tdas, Sorry that I forgot to reply you. You said you designed it just like
Akka `ActorSystem.awaitTermination`. But
[ActorSystem.awaitTermination](https://github.com/akka/akka/blob/master/akka-actor/src/main/scala/akka/actor/ActorSystem.scala#L394)
will throw a TimeoutException in case of timeout.
```Scala
/**
* 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
```
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]