Skip to content
This repository was archived by the owner on Apr 17, 2024. It is now read-only.

Commit d2e26ee

Browse files
committed
Use use-as-needed ByteBuf to reduce the memory footprint of each core
1 parent b63a27c commit d2e26ee

File tree

2 files changed

+30
-13
lines changed

2 files changed

+30
-13
lines changed

core/src/main/scala/org/apache/spark/storage/pmof/NettyByteBufferPool.scala

Lines changed: 21 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -15,13 +15,9 @@ object NettyByteBufferPool extends Logging {
1515
private val allocatedBufferPool: Stack[ByteBuf] = Stack[ByteBuf]()
1616
private var reachRead = false
1717
private val allocator = UnpooledByteBufAllocator.DEFAULT
18+
private var bufferMap: Map[ByteBuf, Long] = Map()
1819

1920
def allocateNewBuffer(bufSize: Int): ByteBuf = synchronized {
20-
if (fixedBufferSize == 0) {
21-
fixedBufferSize = bufSize
22-
} else if (bufSize > fixedBufferSize) {
23-
throw new RuntimeException(s"allocateNewBuffer, expected size is ${fixedBufferSize}, actual size is ${bufSize}")
24-
}
2521
allocatedBufRenCnt.getAndIncrement()
2622
allocatedBytes.getAndAdd(bufSize)
2723
if (allocatedBytes.get > peakAllocatedBytes.get) {
@@ -33,17 +29,35 @@ object NettyByteBufferPool extends Logging {
3329
} else {
3430
allocator.directBuffer(bufSize, bufSize)
3531
}*/
36-
allocator.directBuffer(bufSize, bufSize)
32+
33+
val byteBuf = allocator.directBuffer(bufSize, bufSize)
34+
bufferMap += (byteBuf -> bufSize)
35+
byteBuf
36+
3737
} catch {
3838
case e : Throwable =>
3939
logError(s"allocateNewBuffer size is ${bufSize}")
4040
throw e
4141
}
4242
}
4343

44+
def allocateFlexibleNewBuffer(bufSize: Int): ByteBuf = synchronized {
45+
val initialCapacity = 65536
46+
val maxCapacity = bufSize * 2
47+
val byteBuf = allocator.directBuffer(initialCapacity, maxCapacity)
48+
bufferMap += (byteBuf -> bufSize)
49+
byteBuf
50+
}
51+
4452
def releaseBuffer(buf: ByteBuf): Unit = synchronized {
4553
allocatedBufRenCnt.getAndDecrement()
46-
allocatedBytes.getAndAdd(0 - fixedBufferSize)
54+
try {
55+
val bufSize = bufferMap(buf)
56+
allocatedBytes.getAndAdd(bufSize)
57+
58+
} catch {
59+
case e: NoSuchElementException => {}
60+
}
4761
buf.clear()
4862
//allocatedBufferPool.push(buf)
4963
buf.release(buf.refCnt())

core/src/main/scala/org/apache/spark/storage/pmof/PmemOutputStream.scala

Lines changed: 9 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,6 @@
11
package org.apache.spark.storage.pmof
22

33
import java.io.OutputStream
4-
import java.nio.ByteBuffer
54

65
import io.netty.buffer.{ByteBuf, PooledByteBufAllocator}
76
import org.apache.spark.internal.Logging
@@ -19,21 +18,25 @@ class PmemOutputStream(
1918
val length: Int = bufferSize
2019
var bufferFlushedSize: Int = 0
2120
var bufferRemainingSize: Int = 0
22-
val buf: ByteBuf = NettyByteBufferPool.allocateNewBuffer(length)
23-
val byteBuffer: ByteBuffer = buf.nioBuffer(0, length)
21+
val buf: ByteBuf = NettyByteBufferPool.allocateFlexibleNewBuffer(length);
22+
/**
23+
* Fix size byteBuffer, it will make each core occupy unwanted extra memory space
24+
* val byteBuffer: ByteBuffer = buf.nioBuffer(0, length)
25+
*/
2426

2527
override def write(bytes: Array[Byte], off: Int, len: Int): Unit = {
26-
byteBuffer.put(bytes, off, len)
28+
buf.writeBytes(bytes, off, len)
2729
bufferRemainingSize += len
2830
}
2931

3032
override def write(byte: Int): Unit = {
31-
byteBuffer.putInt(byte)
33+
buf.writeInt(byte)
3234
bufferRemainingSize += 4
3335
}
3436

3537
override def flush(): Unit = {
3638
if (bufferRemainingSize > 0) {
39+
val byteBuffer = buf.nioBuffer()
3740
persistentMemoryWriter.setPartition(numPartitions, blockId, byteBuffer, bufferRemainingSize, set_clean)
3841
bufferFlushedSize += bufferRemainingSize
3942
bufferRemainingSize = 0
@@ -54,7 +57,7 @@ class PmemOutputStream(
5457
def reset(): Unit = {
5558
bufferRemainingSize = 0
5659
bufferFlushedSize = 0
57-
byteBuffer.clear()
60+
buf.clear()
5861
}
5962

6063
override def close(): Unit = synchronized {

0 commit comments

Comments
 (0)