Repository: spark
Updated Branches:
  refs/heads/branch-1.1 e484b8af2 -> af7695456


[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 <[email protected]>

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

a193ae6 [zsxwing] Make sure tryToAcquire won't return a negative value


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/af769545
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/af769545
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/af769545

Branch: refs/heads/branch-1.1
Commit: af7695456e04dc92dc47d55ab169c623e3f29d23
Parents: e484b8a
Author: zsxwing <[email protected]>
Authored: Wed Dec 3 12:19:40 2014 -0800
Committer: Andrew Or <[email protected]>
Committed: Wed Dec 3 12:20:33 2014 -0800

----------------------------------------------------------------------
 .../spark/shuffle/ShuffleMemoryManager.scala       |  5 +++--
 .../spark/shuffle/ShuffleMemoryManagerSuite.scala  | 17 ++++++++++++++++-
 2 files changed, 19 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/af769545/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala 
b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala
index c746e13..d82bb24 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala
@@ -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;

http://git-wip-us.apache.org/repos/asf/spark/blob/af769545/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala
----------------------------------------------------------------------
diff --git 
a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala 
b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala
index d31bc22..e0e646f 100644
--- 
a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala
@@ -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")
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to