Skip to content

Commit

Permalink
Fix the issue that ContextWaiter didn't handle 'spurious wakeup'
Browse files Browse the repository at this point in the history
  • Loading branch information
zsxwing committed Dec 10, 2014
1 parent 9bd9334 commit e06bd4f
Showing 1 changed file with 59 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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) = {
lock.lock()
try {
error = e
condition.signalAll()
} finally {
lock.unlock()
}
}

def notifyStop() = synchronized {
stopped = true
notifyAll()
def notifyStop() = {
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 = {
lock.lock()
try {
if (timeout < 0) {
while (true) {
// 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()
}
}
}

0 comments on commit e06bd4f

Please sign in to comment.