Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
46 commits
Select commit Hold shift + click to select a range
b7c9c23
Move Unsafe mem. mgrs. to spark-core subproject.
JoshRosen Oct 14, 2015
25ba4b5
Merge ExecutorMemoryManager into MemoryManager.
JoshRosen Oct 14, 2015
3d997ce
Naming and formatting fixes.
JoshRosen Oct 16, 2015
d9e6b84
Move Tungsten-related methods to end of MemoryManager file.
JoshRosen Oct 16, 2015
98ef86b
Add taskAttemptId to TaskMemoryManager constructor.
JoshRosen Oct 16, 2015
8f93e94
Move ShuffleMemoryManager into memory package.
JoshRosen Oct 16, 2015
3bbc54d
Merge remote-tracking branch 'origin/master' into SPARK-10984
JoshRosen Oct 16, 2015
88a7970
Fix bug in AbstractBytesToBytesMapSuite.
JoshRosen Oct 16, 2015
ec48ff9
Refactor the existing Tungsten TaskMemoryManager interactions so Tung…
JoshRosen Oct 16, 2015
6f98bc4
Move TaskMemoryManager from unsafe to memory.
JoshRosen Oct 16, 2015
6459397
Further minimization of ShuffleMemoryManager usage.
JoshRosen Oct 16, 2015
60c66b2
Merge ShuffleMemoryManager into MemoryManager.
JoshRosen Oct 17, 2015
7d6a37f
Clean up interaction between TaskMemoryManager and MemoryManager.
JoshRosen Oct 17, 2015
0dc21dc
Merge remote-tracking branch 'origin/master' into SPARK-10984
JoshRosen Oct 22, 2015
f21b767
Fix compilation.
JoshRosen Oct 22, 2015
46ad693
Fix Scalastyle
JoshRosen Oct 22, 2015
c33e330
Fix import ordering in Executor.scala
JoshRosen Oct 22, 2015
ef45d91
Fix import ordering in Task.scala
JoshRosen Oct 22, 2015
c7eac69
Fix import ordering in TaskContextImpl
JoshRosen Oct 22, 2015
d86f435
Fix spillable collection tests
JoshRosen Oct 22, 2015
bba5550
Integrate TaskMemoryManager acquire/releasePage with MemoryManager bo…
JoshRosen Oct 22, 2015
66ae259
Move pooling logic into allocators themselves.
JoshRosen Oct 22, 2015
b1d5151
Scaladoc updates.
JoshRosen Oct 22, 2015
d0c0dd9
Update Spillable to properly integrate with TaskMemoryManager.
JoshRosen Oct 22, 2015
48149fc
Move pageSizeBytes to Tungsten section
JoshRosen Oct 23, 2015
c8ba196
Cleanup after merging of ShuffleMemoryManager into MemoryManager.
JoshRosen Oct 23, 2015
63a6cbc
Rename getMemoryConsumptionForThisTask to getExecutionMemoryUsageForTask
JoshRosen Oct 23, 2015
6ec9c30
Properly thread numCores to memory manager.
JoshRosen Oct 23, 2015
1593fad
Explain why MemoryBlock.pageNumber is public
JoshRosen Oct 23, 2015
64bec0b
Fix TaskMemoryManagerSuite tests.
JoshRosen Oct 23, 2015
f9240e9
Fix compilation
JoshRosen Oct 23, 2015
a95bc08
Fix a memory leak in UnsafeShuffleWriter's sorter
JoshRosen Oct 23, 2015
b3ad761
Remove println
JoshRosen Oct 23, 2015
a7e8320
Fix Scalastyle.
JoshRosen Oct 23, 2015
e874a45
Fix remaining TODOs in UnsafeShuffleWriterSuite.
JoshRosen Oct 23, 2015
2ba6e51
Fix DeveloperAPI change
JoshRosen Oct 23, 2015
0c13723
Address comments in MemoryManager
JoshRosen Oct 23, 2015
04ec429
Release memory acquired after unsuccessful allocatePage() call
JoshRosen Oct 23, 2015
e56d039
Fix EAOM compilation.
JoshRosen Oct 23, 2015
aa14113
Port tests from ShuffleMemoryManagerSuite
JoshRosen Oct 23, 2015
7addf8b
Remove unused non-page-memory allocation methods.
JoshRosen Oct 23, 2015
5af0b17
Update Tungsten tests
JoshRosen Oct 23, 2015
a264703
Fix execution memory leaks in Spillable collections
JoshRosen Oct 24, 2015
f2ab708
Fix NPE in UnsafeRowSerializerSuite
JoshRosen Oct 24, 2015
0b5c72f
Update EAOM tests to reflect fact that iterator() is destructive.
JoshRosen Oct 24, 2015
f68fdb1
Fix streaming test compilation
JoshRosen Oct 26, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Port tests from ShuffleMemoryManagerSuite
  • Loading branch information
JoshRosen committed Oct 23, 2015
commit aa141135ab5f55366de47e7d3cbad26eade5d1e5
134 changes: 134 additions & 0 deletions core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,14 @@ package org.apache.spark.memory

import java.util.concurrent.atomic.AtomicLong

import scala.concurrent.duration.Duration
import scala.concurrent.{Await, ExecutionContext, Future}

import org.mockito.Matchers.{any, anyLong}
import org.mockito.Mockito.{mock, when}
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
import org.scalatest.time.SpanSugar._

import org.apache.spark.SparkFunSuite
import org.apache.spark.storage.MemoryStore
Expand Down Expand Up @@ -126,6 +130,136 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite {
assert(ensureFreeSpaceCalled.get() === DEFAULT_ENSURE_FREE_SPACE_CALLED,
"ensure free space should not have been called!")
}

/**
* Create a MemoryManager with the specified execution memory limit and no storage memory.
*/
protected def createMemoryManager(maxExecutionMemory: Long): MemoryManager

// -- Tests of sharing of execution memory between tasks ----------------------------------------
// Prior to Spark 1.6, these tests were part of ShuffleMemoryManagerSuite.

implicit val ec = ExecutionContext.global

test("single task requesting execution memory") {
val manager = createMemoryManager(1000L)
val taskMemoryManager = new TaskMemoryManager(manager, 0)

assert(taskMemoryManager.acquireExecutionMemory(100L) === 100L)
assert(taskMemoryManager.acquireExecutionMemory(400L) === 400L)
assert(taskMemoryManager.acquireExecutionMemory(400L) === 400L)
assert(taskMemoryManager.acquireExecutionMemory(200L) === 100L)
assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L)
assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L)

taskMemoryManager.releaseExecutionMemory(500L)
assert(taskMemoryManager.acquireExecutionMemory(300L) === 300L)
assert(taskMemoryManager.acquireExecutionMemory(300L) === 200L)

taskMemoryManager.cleanUpAllAllocatedMemory()
assert(taskMemoryManager.acquireExecutionMemory(1000L) === 1000L)
assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L)
}

test("two tasks requesting full execution memory") {
val memoryManager = createMemoryManager(1000L)
val t1MemManager = new TaskMemoryManager(memoryManager, 1)
val t2MemManager = new TaskMemoryManager(memoryManager, 2)
val futureTimeout: Duration = 20.seconds

// Have both tasks request 500 bytes, then wait until both requests have been granted:
val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L) }
val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L) }
assert(Await.result(t1Result1, futureTimeout) === 500L)
assert(Await.result(t2Result1, futureTimeout) === 500L)

// Have both tasks each request 500 bytes more; both should immediately return 0 as they are
// both now at 1 / N
val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L) }
val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) }
assert(Await.result(t1Result2, 200.millis) === 0L)
assert(Await.result(t2Result2, 200.millis) === 0L)
}

test("two tasks cannot grow past 1 / N of execution memory") {
val memoryManager = createMemoryManager(1000L)
val t1MemManager = new TaskMemoryManager(memoryManager, 1)
val t2MemManager = new TaskMemoryManager(memoryManager, 2)
val futureTimeout: Duration = 20.seconds

// Have both tasks request 250 bytes, then wait until both requests have been granted:
val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L) }
val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L) }
assert(Await.result(t1Result1, futureTimeout) === 250L)
assert(Await.result(t2Result1, futureTimeout) === 250L)

// Have both tasks each request 500 bytes more.
// We should only grant 250 bytes to each of them on this second request
val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L) }
val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) }
assert(Await.result(t1Result2, futureTimeout) === 250L)
assert(Await.result(t2Result2, futureTimeout) === 250L)
}

test("tasks can block to get at least 1 / 2N of execution memory") {
val memoryManager = createMemoryManager(1000L)
val t1MemManager = new TaskMemoryManager(memoryManager, 1)
val t2MemManager = new TaskMemoryManager(memoryManager, 2)
val futureTimeout: Duration = 20.seconds

// t1 grabs 1000 bytes and then waits until t2 is ready to make a request.
val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L) }
assert(Await.result(t1Result1, futureTimeout) === 1000L)
val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L) }
// Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult
// to make sure the other thread blocks for some time otherwise.
Thread.sleep(300)
t1MemManager.releaseExecutionMemory(250L)
// The memory freed from t1 should now be granted to t2.
assert(Await.result(t2Result1, futureTimeout) === 250L)
// Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory.
val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L) }
assert(Await.result(t2Result2, 200.millis) === 0L)
}

test("TaskMemoryManager.cleanUpAllAllocatedMemory") {
val memoryManager = createMemoryManager(1000L)
val t1MemManager = new TaskMemoryManager(memoryManager, 1)
val t2MemManager = new TaskMemoryManager(memoryManager, 2)
val futureTimeout: Duration = 20.seconds

// t1 grabs 1000 bytes and then waits until t2 is ready to make a request.
val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L) }
assert(Await.result(t1Result1, futureTimeout) === 1000L)
val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L) }
// Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult
// to make sure the other thread blocks for some time otherwise.
Thread.sleep(300)
// t1 releases all of its memory, so t2 should be able to grab all of the memory
t1MemManager.cleanUpAllAllocatedMemory()
assert(Await.result(t2Result1, futureTimeout) === 500L)
val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) }
assert(Await.result(t2Result2, futureTimeout) === 500L)
val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L) }
assert(Await.result(t2Result3, 200.millis) === 0L)
}

test("tasks should not be granted a negative amount of execution memory") {
// This is a regression test for SPARK-4715.
val memoryManager = createMemoryManager(1000L)
val t1MemManager = new TaskMemoryManager(memoryManager, 1)
val t2MemManager = new TaskMemoryManager(memoryManager, 2)
val futureTimeout: Duration = 20.seconds

val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L) }
assert(Await.result(t1Result1, futureTimeout) === 700L)

val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L) }
assert(Await.result(t2Result1, futureTimeout) === 300L)

val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L) }
assert(Await.result(t1Result2, 200.millis) === 0L)
}
}

private object MemoryManagerSuite {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,14 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite {
(mm, ms)
}

override protected def createMemoryManager(maxMemory: Long): MemoryManager = {
new StaticMemoryManager(
conf,
maxExecutionMemory = maxMemory,
maxStorageMemory = 0,
numCores = 1)
}

test("basic execution memory") {
val maxExecutionMem = 1000L
val (mm, _) = makeThings(maxExecutionMem, Long.MaxValue)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes
(mm, ms)
}

override protected def createMemoryManager(maxMemory: Long): MemoryManager = {
new UnifiedMemoryManager(conf, maxMemory, numCores = 1)
}

private def getStorageRegionSize(mm: UnifiedMemoryManager): Long = {
mm invokePrivate PrivateMethod[Long]('storageRegionSize)()
}
Expand Down