[SPARK-4715][Core] Make sure tryToAcquire won't return a negative value

ShuffleMemoryManager.tryToAcquire may return a negative value. The unit test demonstrates this bug. It will output `0 did not equal -200 granted is negative`.

Author: zsxwing <zsxwing@gmail.com>

Closes #3575 from zsxwing/SPARK-4715 and squashes the following commits:

a193ae6 [zsxwing] Make sure tryToAcquire won't return a negative value
This commit is contained in:
zsxwing 2014-12-03 12:19:40 -08:00 committed by Andrew Or
parent 96786e3ee5
commit edd3cd477c
2 changed files with 19 additions and 3 deletions

View file

@ -66,8 +66,9 @@ private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging {
val curMem = threadMemory(threadId)
val freeMemory = maxMemory - threadMemory.values.sum
// How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads
val maxToGrant = math.min(numBytes, (maxMemory / numActiveThreads) - curMem)
// How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads;
// don't let it be negative
val maxToGrant = math.min(numBytes, math.max(0, (maxMemory / numActiveThreads) - curMem))
if (curMem < maxMemory / (2 * numActiveThreads)) {
// We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking;

View file

@ -159,7 +159,7 @@ class ShuffleMemoryManagerSuite extends FunSuite with Timeouts {
test("threads can block to get at least 1 / 2N memory") {
// t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps
// for a bit and releases 250 bytes, which should then be greanted to t2. Further requests
// for a bit and releases 250 bytes, which should then be granted to t2. Further requests
// by t2 will return false right away because it now has 1 / 2N of the memory.
val manager = new ShuffleMemoryManager(1000L)
@ -291,4 +291,19 @@ class ShuffleMemoryManagerSuite extends FunSuite with Timeouts {
assert(state.t2WaitTime > 200, s"t2 waited less than 200 ms (${state.t2WaitTime})")
}
}
test("threads should not be granted a negative size") {
val manager = new ShuffleMemoryManager(1000L)
manager.tryToAcquire(700L)
val latch = new CountDownLatch(1)
startThread("t1") {
manager.tryToAcquire(300L)
latch.countDown()
}
latch.await() // Wait until `t1` calls `tryToAcquire`
val granted = manager.tryToAcquire(300L)
assert(0 === granted, "granted is negative")
}
}