内存管理器可以说是spark内核中最重要的基础模块之一,shuffle时的排序,rdd缓存,展开内存,广播变量,Task运行结果的存储等等,凡是需要使用内存的地方都需要向内存管理器定额申请。我认为内存管理器的主要作用是为了尽可能减小内存溢出的同时提高内存利用率。旧版本的spark的内存管理是静态内存管理器StaticMemoryManager,而新版本(应该是从1.6之后吧,记不清了)则改成了统一内存管理器UnifiedMemoryManager,同一内存管理器相对于静态内存管理器最大的区别在于执行内存和存储内存二者之间没有明确的界限,可以相互借用,但是执行内存的优先级更高,也就是说如果执行内存不够用就会挤占存储内存,这时会将一部分缓存的rdd溢写到磁盘上直到腾出足够的空间。但是执行内存任何情况下都不会被挤占,想想这也可以理解,毕竟执行内存是用于shuffle时排序的,这只能在内存中进行,而rdd缓存的要求就没有这么严格。
有几个参数控制各个部分内存的使用比例,

  • spark.memory.fraction,默认值0.6,这个参数控制spark内存管理器管理的内存占内存存的比例(准确地说是:堆内存-300m,300m是为永久代预留),也就是说执行内存和存储内存加起来只有(堆内存-300m)的0.6,剩余的0.4是用于用户代码执行过程中的内存占用,比如你的代码中可能会加载一些较大的文件到内存中,或者做一些排序,用户代码使用的内存并不受内存管理器管理,所以需要预留一定的比例。
  • spark.memory.storageFraction,默认值0.5,顾名思义,这个值决定了存储内存的占比,注意是占内存管理器管理的那部分内存的比例,剩余的部分用作执行内存。例如,默认情况下,存储内存占堆内存的比例是0.6 * 0.5 = 0.3(当然准确地说是占堆内存-300m的比例)。

我们首先整体看一下MemoryManager这个类,

  1. maxOnHeapStorageMemory
  2. maxOffHeapStorageMemory
  3. setMemoryStore
  4. acquireStorageMemory
  5. acquireUnrollMemory
  6. acquireExecutionMemory
  7. releaseExecutionMemory
  8. releaseAllExecutionMemoryForTask
  9. releaseStorageMemory
  10. releaseAllStorageMemory
  11. releaseUnrollMemory
  12. executionMemoryUsed
  13. storageMemoryUsed
  14. getExecutionMemoryUsageForTask

可以发现,MemoryManager内部的方法比较少而且是有规律的,它将内存在功能上分为三种:StorageMemory,UnrollMemory,ExecutionMemory,
针对这三种内存分别有申请内存的方法和释放内存的方法,并且三种申请内存的方法都是抽象方法,由子类实现。
此外,我们看一下MemoryManager内部有哪些成员变量:

  1. protected val onHeapStorageMemoryPool = new StorageMemoryPool(this, MemoryMode.ON_HEAP)
  2. protected val offHeapStorageMemoryPool = new StorageMemoryPool(this, MemoryMode.OFF_HEAP)
  3. protected val onHeapExecutionMemoryPool = new ExecutionMemoryPool(this, MemoryMode.ON_HEAP)
  4. protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool(this, MemoryMode.OFF_HEAP)

这四个成员变量分别代表四种内存池。这里要注意的是,MemoryPool的构造其中有一个Object类型参数用于同步锁,MemoryPool内部的一些方法会获取该对象锁用于同步。
我们看一下他们的初始化:

  1. onHeapStorageMemoryPool.incrementPoolSize(onHeapStorageMemory)
  2. onHeapExecutionMemoryPool.incrementPoolSize(onHeapExecutionMemory)
  3. offHeapExecutionMemoryPool.incrementPoolSize(maxOffHeapMemory - offHeapStorageMemory)
  4. offHeapStorageMemoryPool.incrementPoolSize(offHeapStorageMemory)

其实就是调用ExecutionMemoryPool的相关方法,

  1. private[memory]
  2. def releaseExecutionMemory(
  3. numBytes: Long,
  4. taskAttemptId: Long,
  5. memoryMode: MemoryMode): Unit = synchronized {
  6. memoryMode match {
  7. case MemoryMode.ON_HEAP => onHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId)
  8. case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId)
  9. }
  10. }

代码逻辑很简单,就不多说了。
其实从这个方法,我们大概可以看出,spark内存管理的含义,其实spark的内存管理说到底就是对内存使用量的记录和管理,而并不是像操作系统或jvm那样真正地进行内存的分配和回收。

  1. def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = lock.synchronized {
  2. // 从内部的簿记量中获取该任务使用的内存
  3. val curMem = memoryForTask.getOrElse(taskAttemptId, 0L)
  4. // 检查要释放的内存是否超过了该任务实际使用的内存,并打印告警日志
  5. var memoryToFree = if (curMem < numBytes) {
  6. logWarning(
  7. s"Internal error: release called on $numBytes bytes but task only has $curMem bytes " +
  8. s"of memory from the $poolName pool")
  9. curMem
  10. } else {
  11. numBytes
  12. }
  13. if (memoryForTask.contains(taskAttemptId)) {
  14. // 更新簿记量
  15. memoryForTask(taskAttemptId) -= memoryToFree
  16. // 如果该任务的内存使用量小于等于0,那么从簿记量中移除该任务
  17. if (memoryForTask(taskAttemptId) <= 0) {
  18. memoryForTask.remove(taskAttemptId)
  19. }
  20. }
  21. // 最后通知其他等待的线程
  22. // 因为可能会有其他的任务在等待获取执行内存
  23. lock.notifyAll() // Notify waiters in acquireMemory() that memory has been freed
  24. }

把堆上的执行内存和直接内存的执行内存中该任务使用的内存都释放掉,
onHeapExecutionMemoryPool和offHeapExecutionMemoryPool是同一个类,只是一个记录执行内存对直接内存的使用,一个记录执行内存对堆内存的使用。

  1. private[memory] def releaseAllExecutionMemoryForTask(taskAttemptId: Long): Long = synchronized {
  2. onHeapExecutionMemoryPool.releaseAllMemoryForTask(taskAttemptId) +
  3. offHeapExecutionMemoryPool.releaseAllMemoryForTask(taskAttemptId)
  4. }

对于存储内存的使用的记录并没有执行内存那么细,不会记录每个RDD使用了多少内存

  1. def releaseStorageMemory(numBytes: Long, memoryMode: MemoryMode): Unit = synchronized {
  2. memoryMode match {
  3. case MemoryMode.ON_HEAP => onHeapStorageMemoryPool.releaseMemory(numBytes)
  4. case MemoryMode.OFF_HEAP => offHeapStorageMemoryPool.releaseMemory(numBytes)
  5. }
  6. }

这里,我们看一下释放展开内存的方法,发现展开内存使用的就是存储内存。回顾一下BlockManager部分,展开内存的申请主要是在将数据通过MemoryStore存储成块时需要将数据临时放在内存中,这时就需要申请展开内存。

  1. final def releaseUnrollMemory(numBytes: Long, memoryMode: MemoryMode): Unit = synchronized {
  2. releaseStorageMemory(numBytes, memoryMode)
  3. }

从上面分析的几个释放内存的方法不难看出,所谓的释放内存其实只是对内存管理器内部的一些簿记量的改变,这就要求外部的调用者必须确保它们确实释放了这么多的内存,否则内存管理就会和实际的内存使用情况出现很大偏差。当然,好在内存管理器是spark内部的模块,并不向用户开放,所以在用户代码中不会调用内存管理模块。

开篇我们讲到,spark的内存管理器分为两种,而新的版本默认都是使用统一内存管理器UnifiedMemoryManager,后面静态内存管理器会逐渐启用,所以这里我们也重点分析统一内存管理。
前面,我们分析了父类MemoryManager中释放内存的几个方法,而申请内存的几个方法都是抽象方法,这些方法的实现都是在子类中,也就是UnifiedMemoryManager中实现的。

这个方法是用来申请执行内存的。其中定义了几个局部方法,maybeGrowExecutionPool方法用来挤占存储内存以扩展执行内存空间;
computeMaxExecutionPoolSize方法用来计算最大的执行内存大小。
最后调用了executionPool.acquireMemory方法实际申请执行内存。

  1. override private[memory] def acquireExecutionMemory(
  2. numBytes: Long,
  3. taskAttemptId: Long,
  4. memoryMode: MemoryMode): Long = synchronized {
  5. // 检查内存大小是否正确
  6. assertInvariants()
  7. assert(numBytes >= 0)
  8. // 根据堆内存还是直接内存决定使用不同的内存池和内存大小
  9. val (executionPool, storagePool, storageRegionSize, maxMemory) = memoryMode match {
  10. case MemoryMode.ON_HEAP => (
  11. onHeapExecutionMemoryPool,
  12. onHeapStorageMemoryPool,
  13. onHeapStorageRegionSize,
  14. maxHeapMemory)
  15. case MemoryMode.OFF_HEAP => (
  16. offHeapExecutionMemoryPool,
  17. offHeapStorageMemoryPool,
  18. offHeapStorageMemory,
  19. maxOffHeapMemory)
  20. }
  21. /**
  22. * Grow the execution pool by evicting cached blocks, thereby shrinking the storage pool.
  23. *
  24. * When acquiring memory for a task, the execution pool may need to make multiple
  25. * attempts. Each attempt must be able to evict storage in case another task jumps in
  26. * and caches a large block between the attempts. This is called once per attempt.
  27. */
  28. // 通过挤占存储内存来扩张执行内存,
  29. // 通过将缓存的块溢写到磁盘上,从而为执行内存腾出空间
  30. def maybeGrowExecutionPool(extraMemoryNeeded: Long): Unit = {
  31. if (extraMemoryNeeded > 0) {
  32. // There is not enough free memory in the execution pool, so try to reclaim memory from
  33. // storage. We can reclaim any free memory from the storage pool. If the storage pool
  34. // has grown to become larger than `storageRegionSize`, we can evict blocks and reclaim
  35. // the memory that storage has borrowed from execution.
  36. // 我们可以将剩余的存储内存都借过来用作执行内存
  37. // 另外,如果存储内存向执行内存借用了一部分内存,也就是说此时存储内存的实际大小大于配置的值
  38. // 那么我们就将所有的借用的存储内存都还回来
  39. val memoryReclaimableFromStorage = math.max(
  40. storagePool.memoryFree,
  41. storagePool.poolSize - storageRegionSize)
  42. if (memoryReclaimableFromStorage > 0) {
  43. // Only reclaim as much space as is necessary and available:
  44. // 只腾出必要大小的内存空间,这个方法会将内存中的block挤到磁盘中
  45. val spaceToReclaim = storagePool.freeSpaceToShrinkPool(
  46. math.min(extraMemoryNeeded, memoryReclaimableFromStorage))
  47. // 更新一些簿记量,存储内存少了这么多内存,相应的执行内存增加了这么多内存
  48. storagePool.decrementPoolSize(spaceToReclaim)
  49. executionPool.incrementPoolSize(spaceToReclaim)
  50. }
  51. }
  52. }
  53. /**
  54. * The size the execution pool would have after evicting storage memory.
  55. *
  56. * The execution memory pool divides this quantity among the active tasks evenly to cap
  57. * the execution memory allocation for each task. It is important to keep this greater
  58. * than the execution pool size, which doesn't take into account potential memory that
  59. * could be freed by evicting storage. Otherwise we may hit SPARK-12155.
  60. *
  61. * Additionally, this quantity should be kept below `maxMemory` to arbitrate fairness
  62. * in execution memory allocation across tasks, Otherwise, a task may occupy more than
  63. * its fair share of execution memory, mistakenly thinking that other tasks can acquire
  64. * the portion of storage memory that cannot be evicted.
  65. */
  66. def computeMaxExecutionPoolSize(): Long = {
  67. maxMemory - math.min(storagePool.memoryUsed, storageRegionSize)
  68. }
  69. executionPool.acquireMemory(
  70. numBytes, taskAttemptId, maybeGrowExecutionPool, () => computeMaxExecutionPoolSize)
  71. }

这个方法的代码我就不贴了,主要是一些复杂的内存申请规则的计算,以及内部簿记量的维护,此外如果现有可用的内存量太小,则会等待(通过对象锁等待)直到其他任务释放一些内存;
除此之外最重要的就是对上面提到的maybeGrowExecutionPool方法的调用,所以我们重点还是看一下maybeGrowExecutionPool方法。

由于这个方法在前面已经贴出来,并且标上了很详细的注释,所以代码逻辑略过,其中有一个关键的调用storagePool.freeSpaceToShrinkPool,这个方法实现了将内存中的块挤出去的逻辑。

我们发现其中调用了memoryStore.evictBlocksToFreeSpace方法,

  1. def freeSpaceToShrinkPool(spaceToFree: Long): Long = lock.synchronized {
  2. val spaceFreedByReleasingUnusedMemory = math.min(spaceToFree, memoryFree)
  3. val remainingSpaceToFree = spaceToFree - spaceFreedByReleasingUnusedMemory
  4. if (remainingSpaceToFree > 0) {
  5. // If reclaiming free memory did not adequately shrink the pool, begin evicting blocks:
  6. val spaceFreedByEviction =
  7. memoryStore.evictBlocksToFreeSpace(None, remainingSpaceToFree, memoryMode)
  8. // When a block is released, BlockManager.dropFromMemory() calls releaseMemory(), so we do
  9. // not need to decrement _memoryUsed here. However, we do need to decrement the pool size.
  10. spaceFreedByReleasingUnusedMemory + spaceFreedByEviction
  11. } else {
  12. spaceFreedByReleasingUnusedMemory
  13. }
  14. }

这个方法看似很长,其实大概可以总结为一点。
因为MemoryStore存储了内存中所有块的实际数据,所以可以根据这些信息知道每个块实际大小,这样就能计算出需要挤出哪些块,当然这个过程中还有一些细节的处理,比如块的写锁的获取和释放等等。
这里面,实际将块从内存中释放(本质上就是将块的数据对应的MemoryEntry的引用设为null,这样gc就可以回收这个块)的功能代码在blockEvictionHandler.dropFromMemory方法中实现,也就是
BlockManager.dropFromMemory。

  1. private[spark] def evictBlocksToFreeSpace(
  2. blockId: Option[BlockId],
  3. space: Long,
  4. memoryMode: MemoryMode): Long = {
  5. assert(space > 0)
  6. memoryManager.synchronized {
  7. var freedMemory = 0L
  8. val rddToAdd = blockId.flatMap(getRddId)
  9. val selectedBlocks = new ArrayBuffer[BlockId]
  10. def blockIsEvictable(blockId: BlockId, entry: MemoryEntry[_]): Boolean = {
  11. entry.memoryMode == memoryMode && (rddToAdd.isEmpty || rddToAdd != getRddId(blockId))
  12. }
  13. // This is synchronized to ensure that the set of entries is not changed
  14. // (because of getValue or getBytes) while traversing the iterator, as that
  15. // can lead to exceptions.
  16. entries.synchronized {
  17. val iterator = entries.entrySet().iterator()
  18. while (freedMemory < space && iterator.hasNext) {
  19. val pair = iterator.next()
  20. val blockId = pair.getKey
  21. val entry = pair.getValue
  22. if (blockIsEvictable(blockId, entry)) {
  23. // We don't want to evict blocks which are currently being read, so we need to obtain
  24. // an exclusive write lock on blocks which are candidates for eviction. We perform a
  25. // non-blocking "tryLock" here in order to ignore blocks which are locked for reading:
  26. // 这里之所以要获取写锁是为了防止在块正在被读取或写入的时候将其挤出去
  27. if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) {
  28. selectedBlocks += blockId
  29. freedMemory += pair.getValue.size
  30. }
  31. }
  32. }
  33. }
  34. def dropBlock[T](blockId: BlockId, entry: MemoryEntry[T]): Unit = {
  35. val data = entry match {
  36. case DeserializedMemoryEntry(values, _, _) => Left(values)
  37. case SerializedMemoryEntry(buffer, _, _) => Right(buffer)
  38. }
  39. // 这里的调用将块挤出内存,如果允许写到磁盘则溢写到磁盘上
  40. // 注意blockEvictionHandler的实现类就是BlockManager
  41. val newEffectiveStorageLevel =
  42. blockEvictionHandler.dropFromMemory(blockId, () => data)(entry.classTag)
  43. if (newEffectiveStorageLevel.isValid) {
  44. // The block is still present in at least one store, so release the lock
  45. // but don't delete the block info
  46. // 因为前面获取了这些块的写锁,还没有释放,
  47. // 所以在这里释放这些块的写锁
  48. blockInfoManager.unlock(blockId)
  49. } else {
  50. // The block isn't present in any store, so delete the block info so that the
  51. // block can be stored again
  52. // 因为块由于从内存中移除又没有写到磁盘上,所以直接从内部的簿记量中移除该块的信息
  53. blockInfoManager.removeBlock(blockId)
  54. }
  55. }
  56. // 如果腾出的内存足够多,比申请的量要大,这时才会真正释放相应的块
  57. if (freedMemory >= space) {
  58. var lastSuccessfulBlock = -1
  59. try {
  60. logInfo(s"${selectedBlocks.size} blocks selected for dropping " +
  61. s"(${Utils.bytesToString(freedMemory)} bytes)")
  62. (0 until selectedBlocks.size).foreach { idx =>
  63. val blockId = selectedBlocks(idx)
  64. val entry = entries.synchronized {
  65. entries.get(blockId)
  66. }
  67. // This should never be null as only one task should be dropping
  68. // blocks and removing entries. However the check is still here for
  69. // future safety.
  70. if (entry != null) {
  71. dropBlock(blockId, entry)
  72. // 这时为测试留的一个钩子方法
  73. afterDropAction(blockId)
  74. }
  75. lastSuccessfulBlock = idx
  76. }
  77. logInfo(s"After dropping ${selectedBlocks.size} blocks, " +
  78. s"free memory is ${Utils.bytesToString(maxMemory - blocksMemoryUsed)}")
  79. freedMemory
  80. } finally {
  81. // like BlockManager.doPut, we use a finally rather than a catch to avoid having to deal
  82. // with InterruptedException
  83. // 如果不是所有的块都转移成功,那么必然有的块的写锁可能没有释放
  84. // 所以在这里将这些没有移除成功的块的写锁释放掉
  85. if (lastSuccessfulBlock != selectedBlocks.size - 1) {
  86. // the blocks we didn't process successfully are still locked, so we have to unlock them
  87. (lastSuccessfulBlock + 1 until selectedBlocks.size).foreach { idx =>
  88. val blockId = selectedBlocks(idx)
  89. blockInfoManager.unlock(blockId)
  90. }
  91. }
  92. }
  93. } else {// 如果不能腾出足够多的内存,那么取消这次行动,释放所有已经持有的块的写锁
  94. blockId.foreach { id =>
  95. logInfo(s"Will not store $id")
  96. }
  97. selectedBlocks.foreach { id =>
  98. blockInfoManager.unlock(id)
  99. }
  100. 0L
  101. }
  102. }
  103. }

总结一下这个方法的主要逻辑:

  • 如果存储级别允许存到磁盘,那么先溢写到磁盘上
  • 将block从MemoryStore内部的map结构中移除掉
  • 向driver上的BlockManagerMaster汇报块更新
  • 向任务度量系统汇报块更新的统计信息

所以,七绕八绕,饶了这么一大圈,其实所谓的内存挤占,其实就是把引用设为null ^_^当然肯定不是这么简单啦,其实在整个分析的过程中我们也能发现,所谓的内存管理大部分工作就是对任务使用内存一些簿记量的管理维护,这里面有一些比较复杂的逻辑,例如给每个任务分配多少内存的计算逻辑就比较复杂。

  1. private[storage] override def dropFromMemory[T: ClassTag](
  2. blockId: BlockId,
  3. data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel = {
  4. logInfo(s"Dropping block $blockId from memory")
  5. val info = blockInfoManager.assertBlockIsLockedForWriting(blockId)
  6. var blockIsUpdated = false
  7. val level = info.level
  8. // Drop to disk, if storage level requires
  9. // 如果存储级别允许存到磁盘,那么先溢写到磁盘上
  10. if (level.useDisk && !diskStore.contains(blockId)) {
  11. logInfo(s"Writing block $blockId to disk")
  12. data() match {
  13. case Left(elements) =>
  14. diskStore.put(blockId) { channel =>
  15. val out = Channels.newOutputStream(channel)
  16. serializerManager.dataSerializeStream(
  17. blockId,
  18. out,
  19. elements.toIterator)(info.classTag.asInstanceOf[ClassTag[T]])
  20. }
  21. case Right(bytes) =>
  22. diskStore.putBytes(blockId, bytes)
  23. }
  24. blockIsUpdated = true
  25. }
  26. // Actually drop from memory store
  27. val droppedMemorySize =
  28. if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L
  29. val blockIsRemoved = memoryStore.remove(blockId)
  30. if (blockIsRemoved) {
  31. blockIsUpdated = true
  32. } else {
  33. logWarning(s"Block $blockId could not be dropped from memory as it does not exist")
  34. }
  35. val status = getCurrentBlockStatus(blockId, info)
  36. if (info.tellMaster) {
  37. reportBlockStatus(blockId, status, droppedMemorySize)
  38. }
  39. // 向任务度量系统汇报块更新的统计信息
  40. if (blockIsUpdated) {
  41. addUpdatedBlockStatusToTaskMetrics(blockId, status)
  42. }
  43. status.storageLevel
  44. }

我们再来看一下对于存储内存的申请。
其中,存储内存向执行内存借用 的逻辑相对简单,仅仅是将两个内存池的大小改一下,执行内存池减少一定的大小,存储内存池则增加相应的大小。

  1. override def acquireStorageMemory(
  2. blockId: BlockId,
  3. numBytes: Long,
  4. memoryMode: MemoryMode): Boolean = synchronized {
  5. assertInvariants()
  6. assert(numBytes >= 0)
  7. val (executionPool, storagePool, maxMemory) = memoryMode match {
  8. case MemoryMode.ON_HEAP => (
  9. onHeapExecutionMemoryPool,
  10. onHeapStorageMemoryPool,
  11. maxOnHeapStorageMemory)
  12. case MemoryMode.OFF_HEAP => (
  13. offHeapExecutionMemoryPool,
  14. offHeapStorageMemoryPool,
  15. maxOffHeapStorageMemory)
  16. }
  17. // 因为执行内存挤占不了,所以这里如果申请的内存超过现在可用的内存,那么就申请不了了
  18. if (numBytes > maxMemory) {
  19. // Fail fast if the block simply won't fit
  20. logInfo(s"Will not store $blockId as the required space ($numBytes bytes) exceeds our " +
  21. s"memory limit ($maxMemory bytes)")
  22. return false
  23. }
  24. // 如果大于存储内存的可用内存,那么就需要向执行内存借用一部分内存
  25. if (numBytes > storagePool.memoryFree) {
  26. // There is not enough free memory in the storage pool, so try to borrow free memory from
  27. // the execution pool.
  28. val memoryBorrowedFromExecution = Math.min(executionPool.memoryFree,
  29. numBytes - storagePool.memoryFree)
  30. // 存储内存向执行内存借用的逻辑很简单,
  31. // 仅仅是将两个内存池的大小改一下,
  32. // 执行内存池减少一定的大小,存储内存池则增加相应的大小
  33. executionPool.decrementPoolSize(memoryBorrowedFromExecution)
  34. storagePool.incrementPoolSize(memoryBorrowedFromExecution)
  35. }
  36. // 通过storagePool申请一定量的内存
  37. storagePool.acquireMemory(blockId, numBytes)
  38. }
  1. def acquireMemory(
  2. blockId: BlockId,
  3. numBytesToAcquire: Long,
  4. numBytesToFree: Long): Boolean = lock.synchronized {
  5. assert(numBytesToAcquire >= 0)
  6. assert(numBytesToFree >= 0)
  7. assert(memoryUsed <= poolSize)
  8. // 首先调用MemoryStore的相关方法挤出一些块以释放内存
  9. if (numBytesToFree > 0) {
  10. memoryStore.evictBlocksToFreeSpace(Some(blockId), numBytesToFree, memoryMode)
  11. }
  12. // NOTE: If the memory store evicts blocks, then those evictions will synchronously call
  13. // back into this StorageMemoryPool in order to free memory. Therefore, these variables
  14. // should have been updated.
  15. // 因为前面挤出一些块后释放内存时,BlockManager会通过MemoryManager相关方法更新内部的簿记量,
  16. // 所以这里的memoryFree就会变化,会变大
  17. val enoughMemory = numBytesToAcquire <= memoryFree
  18. if (enoughMemory) {
  19. _memoryUsed += numBytesToAcquire
  20. }
  21. enoughMemory
  22. }

可以看到,这里也调用了memoryStore.evictBlocksToFreeSpace方法来讲一部分块挤出内存,以此来为新的block腾出空间。

另外还有对展开内存的申请,实际就是申请存储内存。

  1. override def acquireUnrollMemory(
  2. blockId: BlockId,
  3. numBytes: Long,
  4. memoryMode: MemoryMode): Boolean = synchronized {
  5. acquireStorageMemory(blockId, numBytes, memoryMode)
  6. }

内存管理,本质上是对shuffle排序过程中使用的内存和rdd缓存使用的内存的簿记,通过对内存使用量的详细精确的记录和管理,最大限度避免OOM的发生,同时尽量提高内存利用率。

posted on 2019-06-13 01:35 _朱葛 阅读() 评论() 编辑 收藏

版权声明:本文为zhuge134原创文章,遵循 CC 4.0 BY-SA 版权协议,转载请附上原文出处链接和本声明。
本文链接:https://www.cnblogs.com/zhuge134/p/11013826.html