Skip to content

Commit

Permalink
Fix ShuffleMemoryManagerSuite
Browse files Browse the repository at this point in the history
  • Loading branch information
JoshRosen committed Jul 28, 2015
1 parent f57f3f2 commit 7b0f04b
Showing 1 changed file with 25 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,26 +17,39 @@

package org.apache.spark.shuffle

import java.util.concurrent.CountDownLatch
import java.util.concurrent.atomic.AtomicInteger

import org.mockito.Mockito._
import org.scalatest.concurrent.Timeouts
import org.scalatest.time.SpanSugar._
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.CountDownLatch

import org.apache.spark.SparkFunSuite
import org.apache.spark.{SparkFunSuite, TaskContext}

class ShuffleMemoryManagerSuite extends SparkFunSuite with Timeouts {

val nextTaskAttemptId = new AtomicInteger()

/** Launch a thread with the given body block and return it. */
private def startThread(name: String)(body: => Unit): Thread = {
val thread = new Thread("ShuffleMemorySuite " + name) {
override def run() {
body
try {
val taskAttemptId = nextTaskAttemptId.getAndIncrement
val mockTaskContext = mock(classOf[TaskContext], RETURNS_SMART_NULLS)
when(mockTaskContext.taskAttemptId()).thenReturn(taskAttemptId)
TaskContext.setTaskContext(mockTaskContext)
body
} finally {
TaskContext.unset()
}
}
}
thread.start()
thread
}

test("single thread requesting memory") {
test("single task requesting memory") {
val manager = new ShuffleMemoryManager(1000L)

assert(manager.tryToAcquire(100L) === 100L)
Expand Down Expand Up @@ -107,8 +120,8 @@ class ShuffleMemoryManagerSuite extends SparkFunSuite with Timeouts {
}


test("threads cannot grow past 1 / N") {
// Two threads request 250 bytes first, wait for each other to get it, and then request
test("tasks cannot grow past 1 / N") {
// Two tasks request 250 bytes first, wait for each other to get it, and then request
// 500 more; we should only grant 250 bytes to each of them on this second request

val manager = new ShuffleMemoryManager(1000L)
Expand Down Expand Up @@ -158,7 +171,7 @@ class ShuffleMemoryManagerSuite extends SparkFunSuite with Timeouts {
assert(state.t2Result2 === 250L)
}

test("threads can block to get at least 1 / 2N memory") {
test("tasks 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 granted to t2. Further requests
// by t2 will return false right away because it now has 1 / 2N of the memory.
Expand Down Expand Up @@ -224,7 +237,7 @@ class ShuffleMemoryManagerSuite extends SparkFunSuite with Timeouts {
}
}

test("releaseMemoryForThisThread") {
test("releaseMemoryForThisTask") {
// t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps
// for a bit and releases all its memory. t2 should now be able to grab all the memory.

Expand All @@ -251,7 +264,7 @@ class ShuffleMemoryManagerSuite extends SparkFunSuite with Timeouts {
}
}
// Sleep a bit before releasing our memory; this is hacky but it would be difficult to make
// sure the other thread blocks for some time otherwise
// sure the other task blocks for some time otherwise
Thread.sleep(300)
manager.releaseMemoryForThisTask()
}
Expand Down Expand Up @@ -282,7 +295,7 @@ class ShuffleMemoryManagerSuite extends SparkFunSuite with Timeouts {
t2.join()
}

// Both threads should've been able to acquire their memory; the second one will have waited
// Both tasks should've been able to acquire their memory; the second one will have waited
// until the first one acquired 1000 bytes and then released all of it
state.synchronized {
assert(state.t1Result === 1000L, "t1 could not allocate memory")
Expand All @@ -293,7 +306,7 @@ class ShuffleMemoryManagerSuite extends SparkFunSuite with Timeouts {
}
}

test("threads should not be granted a negative size") {
test("tasks should not be granted a negative size") {
val manager = new ShuffleMemoryManager(1000L)
manager.tryToAcquire(700L)

Expand Down

0 comments on commit 7b0f04b

Please sign in to comment.