一、前言
目的是分析Spark Shuffle Read 阶段里的 fetch block 流程,看是否有优化的空间或者优化的配置参数
1. 相关版本:Spark Master branch(2018.10, compiled-version spark-2.5.0, 相关的测试设置了spark.shuffle.sort.bypassMergeThreshold 1 和 YARN-client 模式) ,HiBench-6.0 and Hadoop-2.7.1
2. 建议先了解Spark 的 RDD、DAG、Shuffle(MapoutputTrack/MapoutputTrackMaster/MapoutputTrackWorker)、Memory 的基本概念。
3. 本故事基于HiBench 的 Terasort Test Case,我配置了spark.shuffle.sort.bypassMergeThreshold 1,代码走的是SerializedShuffleHandle(即UnsafeShuffle)
二、故事要从 ResultTask 说起。。。
1) ResultTask 里调用read (callout ) 也就是 BlockStoreShuffleReader.read(),即下面调用栈里的
read:102, BlockStoreShuffleReader (org.apache.spark.shuffle)
next:400, ShuffleBlockFetcherIterator (org.apache.spark.storage)
hasNext:31, CompletionIterator (org.apache.spark.util)
hasNext:37, InterruptibleIterator (org.apache.spark)
insertAll:199, ExternalSorter (org.apache.spark.util.collection)
read:102, BlockStoreShuffleReader (org.apache.spark.shuffle)
compute:105, ShuffledRDD (org.apache.spark.rdd)
computeOrReadCheckpoint:324, RDD (org.apache.spark.rdd)
iterator:288, RDD (org.apache.spark.rdd)
compute:52, MapPartitionsRDD (org.apache.spark.rdd)
computeOrReadCheckpoint:324, RDD (org.apache.spark.rdd)
iterator:288, RDD (org.apache.spark.rdd)
runTask:90, ResultTask (org.apache.spark.scheduler)
run:121, Task (org.apache.spark.scheduler)
apply:402, Executor$TaskRunner$$anonfun$10 (org.apache.spark.executor)
tryWithSafeFinally:1360, Utils$ (org.apache.spark.util)
run:408, Executor$TaskRunner (org.apache.spark.executor)
2)BlockStoreShuffleReader.read() 先是 调用 mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition) 获得 mapStatus也就是Iterator[(BlockManagerId, Seq[(BlockId, Long)])],该mapStatus作为 ShuffleBlockFetcherIterator 入参,其实就是Map Output的meta data,格式为 [(BlockManagerId, Seq[(BlockId, Long)])]
override def read(): Iterator[Product2[K, C]] = {
val wrappedStreams = new ShuffleBlockFetcherIterator(
context,
blockManager.shuffleClient,
blockManager,
mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition),
serializerManager.wrapStream,
// Note: we use getSizeAsMb when no suffix is provided for backwards compatibility
SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024,
SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue),
SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS),
SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM),
SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true))
。。。
}
cluster里用的是 MapoutputTrackerWorker 的 getMapSizesByExecutorId的定义如下:
/**
* Executor-side client for fetching map output info from the driver's MapOutputTrackerMaster.
* Note that this is not used in local-mode; instead, local-mode Executors access the
* MapOutputTrackerMaster directly (which is possible because the master and worker share a comon
* superclass).
*/
private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) {
val mapStatuses: Map[Int, Array[MapStatus]] =
new ConcurrentHashMap[Int, Array[MapStatus]]().asScala
/** Remembers which map output locations are currently being fetched on an executor. */
private val fetching = new HashSet[Int]
// Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result.
override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int)
: Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = {
logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition")
val statuses = getStatuses(shuffleId)
try {
MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses)
} catch {
case e: MetadataFetchFailedException =>
// We experienced a fetch failure so our mapStatuses cache is outdated; clear it:
mapStatuses.clear()
throw e
}
}
3)BlockStoreShuffleReader.read() new 了 ShuffleBlockFetcherIterator 对象,而new 对象中得到了格式为 [(BlockManagerId, Seq[(BlockId, Long)])] 的 Map Output的meta data 用来初始化ShuffleBlockFetcherIterator 的私有变量,然后对象初始化会调用私有函数innitialize().
private[this] def initialize(): Unit = {
// Add a task completion callback (called in both success case and failure case) to cleanup.
context.addTaskCompletionListener[Unit](_ => cleanup())
// Split local and remote blocks.
val remoteRequests = splitLocalRemoteBlocks()
// Add the remote requests into our queue in a random order
fetchRequests ++= Utils.randomize(remoteRequests)
assert ((0 == reqsInFlight) == (0 == bytesInFlight),
"expected reqsInFlight = 0 but found reqsInFlight = " + reqsInFlight +
", expected bytesInFlight = 0 but found bytesInFlight = " + bytesInFlight)
// Send out initial requests for blocks, up to our maxBytesInFlight
fetchUpToMaxBytes()
val numFetches = remoteRequests.size - fetchRequests.size
logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime))
// Get Local Blocks
fetchLocalBlocks()
logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime))
}
4) 接着分析ShuffleBlockFetcherIterator.innitialize() 函数,它会使用 mapStatus (也就是Iterator[(BlockManagerId, Seq[(BlockId, Long)])]),先调用 splitLocalRemoteBlocks 函数区分从mapStatus里区分是localBlocks 还是 remoteBlocks ,区分条件是executorId => if (address.executorId == blockManager.blockManagerId.executorId)。
private[this] def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = {
// Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them
// smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5
// nodes, rather than blocking on reading output from one node.
val targetRequestSize = math.max(maxBytesInFlight / 5, 1L)
logDebug("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize
+ ", maxBlocksInFlightPerAddress: " + maxBlocksInFlightPerAddress)
// Split local and remote blocks. Remote blocks are further split into FetchRequests of size
// at most maxBytesInFlight in order to limit the amount of data in flight.
val remoteRequests = new ArrayBuffer[FetchRequest]
for ((address, blockInfos) <- blocksByAddress) {
if (address.executorId == blockManager.blockManagerId.executorId) {
blockInfos.find(_._2 <= 0) match {
case Some((blockId, size)) if size < 0 =>
throw new BlockException(blockId, "Negative block size " + size)
case Some((blockId, size)) if size == 0 =>
throw new BlockException(blockId, "Zero-sized blocks should be excluded.")
case None => // do nothing.
}
localBlocks ++= blockInfos.map(_._1)
numBlocksToFetch += localBlocks.size
} else {
val iterator = blockInfos.iterator
var curRequestSize = 0L
var curBlocks = new ArrayBuffer[(BlockId, Long)]
while (iterator.hasNext) {
val (blockId, size) = iterator.next()
if (size < 0) {
throw new BlockException(blockId, "Negative block size " + size)
} else if (size == 0) {
throw new BlockException(blockId, "Zero-sized blocks should be excluded.")
} else {
curBlocks += ((blockId, size))
remoteBlocks += blockId
numBlocksToFetch += 1
curRequestSize += size
}
if (curRequestSize >= targetRequestSize ||
curBlocks.size >= maxBlocksInFlightPerAddress) {
// Add this FetchRequest
remoteRequests += new FetchRequest(address, curBlocks)
logDebug(s"Creating fetch request of $curRequestSize at $address "
+ s"with ${curBlocks.size} blocks")
curBlocks = new ArrayBuffer[(BlockId, Long)]
curRequestSize = 0
}
}
// Add in the final request
if (curBlocks.nonEmpty) {
remoteRequests += new FetchRequest(address, curBlocks)
}
}
}
logInfo(s"Getting $numBlocksToFetch non-empty blocks including ${localBlocks.size}" +
s" local blocks and ${remoteBlocks.size} remote blocks")
remoteRequests
}
通过 remoteBlocks 来 构建 remoteRequests += new FetchRequest(address, curBlocks). 一共有多少个 remoteRequests 及 每个remoteRequest包含多少个blocks及blocks的大小 由下面条件来决定:
if (curRequestSize >= targetRequestSize ||
curBlocks.size >= maxBlocksInFlightPerAddress) {
// Add this FetchRequest
remoteRequests += new FetchRequest(address, curBlocks)
logDebug(s"Creating fetch request of $curRequestSize at $address "
+ s"with ${curBlocks.size} blocks")
curBlocks = new ArrayBuffer[(BlockId, Long)]
curRequestSize = 0
}
对下面几个变量做说明:
全局变量:
a) remoteBlocks, ArrayBuffer[BlockId],blockId数组
b) localBlocks, ArrayBuffer[BlockId],blockId数组
c) numBlocksToFetch, Int, 记录一共要fetch的block数(包括local+remote)
d) remoteRequests, new ArrayBuffer[FetchRequest], 而 FetchRequest 由 (address, curBlocks) 组成。
remoteBlocks计算时的局部变量:
a) curBlocks , new ArrayBuffer[(BlockId, Long)] ,单个 address 里 累计的 remoteBlocks
b) curRequestSize,Long, 当前remoteRequest包含的blocks的总size
//splitLocalRemoteBlocks 是否有优化条件?过网路还是直接磁盘copy
5) 赋值好 remoteRequests 后,打乱其顺序得到fetchRequests。接着分析initialize()里的调用fetchUpToMaxBytes ()
// Split local and remote blocks.
val remoteRequests = splitLocalRemoteBlocks()
// Add the remote requests into our queue in a random order
fetchRequests ++= Utils.randomize(remoteRequests)
然后通过ShuffleBlockFetcherIterator.send 发送fetchRequest给对应的Executor 了,由函数里 fetchUpToMaxBytes 调用 ShuffleBlockFetcherIterator.send, 而函数里 fetchUpToMaxByte 会先做 check 也就是 while(isRemoteBlockFetchable(fetchRequests)) ,其实就是做些 max check。如果没超过就调用ShuffleBlockFetcherIterator.send, 如果超过了,等 ShuffleBlockFetcherIterator.hasNext判断,然后调用ShuffleBlockFetcherIterator.next(该函数会再次调用fetchUpToMaxByte)
private def fetchUpToMaxBytes(): Unit = {
// Send fetch requests up to maxBytesInFlight. If you cannot fetch from a remote host
// immediately, defer the request until the next time it can be processed.
// Process any outstanding deferred fetch requests if possible.
if (deferredFetchRequests.nonEmpty) {
for ((remoteAddress, defReqQueue) <- deferredFetchRequests) {
while (isRemoteBlockFetchable(defReqQueue) &&
!isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) {
val request = defReqQueue.dequeue()
logDebug(s"Processing deferred fetch request for $remoteAddress with "
+ s"${request.blocks.length} blocks")
send(remoteAddress, request)
if (defReqQueue.isEmpty) {
deferredFetchRequests -= remoteAddress
}
}
}
}
// Process any regular fetch requests if possible.
while (isRemoteBlockFetchable(fetchRequests)) {
val request = fetchRequests.dequeue()
val remoteAddress = request.address
if (isRemoteAddressMaxedOut(remoteAddress, request)) {
logDebug(s"Deferring fetch request for $remoteAddress with ${request.blocks.size} blocks")
val defReqQueue = deferredFetchRequests.getOrElse(remoteAddress, new Queue[FetchRequest]())
defReqQueue.enqueue(request)
deferredFetchRequests(remoteAddress) = defReqQueue
} else {
send(remoteAddress, request)
}
}
6)接着讲ShuffleBlockFetcherIterator.send 的实现,它又调用 ShuffleBlockFetcherIterator.sendRequest 里判断 req.size(也就是该fetchRequest的blocks的总大小) 是否 超过maxReqSizeShuffleToMem。如果超过,就传入ShuffleBlockFetcherIterator对象,而ShuffleBlockFetcherIterator 类继承了 DownloadFileManager;如果没超过,传入的DownloadFileManager=NULL.
ShuffleBlockFetcherIterator.sendRequest函数的相关代码如下:
// Fetch remote shuffle blocks to disk when the request is too large. Since the shuffle data is
// already encrypted and compressed over the wire(w.r.t. the related configs), we can just fetch
// the data and write it to file directly.
if (req.size > maxReqSizeShuffleToMem) {
shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
blockFetchingListener, this)
} else {
shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
blockFetchingListener, null)
}
正常req.size都不会超过maxReqSizeShuffleToMem,因为maxReqSizeShuffleToMem的值是传入参数值SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM),而其定义如下:
private[spark] val MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM =
ConfigBuilder("spark.maxRemoteBlockSizeFetchToMem")
.doc("Remote block will be fetched to disk when size of the block is above this threshold " +
"in bytes. This is to avoid a giant request takes too much memory. We can enable this " +
"config by setting a specific value(e.g. 200m). Note this configuration will affect " +
"both shuffle fetch and block manager remote block fetch. For users who enabled " +
"external shuffle service, this feature can only be worked when external shuffle" +
"service is newer than Spark 2.2.")
.bytesConf(ByteUnit.BYTE)
// fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB, so we might
// as well use fetch-to-disk in that case. The message includes some metadata in addition
// to the block data itself (in particular UploadBlock has a lot of metadata), so we leave
// extra room.
.createWithDefault(Int.MaxValue - 512)
7)接着分析shuffleClient.fetchBlocks,而 shuffleClient 实现类就是NettyBlockTransferService (它继承了父类abstract class BlockTransferService ,BlockTransferService 继承了父类 abstract class ShuffleClient )
BlockTransferService 类:
abstract class BlockTransferService extends ShuffleClient with Closeable with Logging {
/**
* Initialize the transfer service by giving it the BlockDataManager that can be used to fetch
* local blocks or put local blocks.
*/
def init(blockDataManager: BlockDataManager): Unit
/**
* Tear down the transfer service.
*/
def close(): Unit
/**
* Port number the service is listening on, available only after [[init]] is invoked.
*/
def port: Int
/**
* Host name the service is listening on, available only after [[init]] is invoked.
*/
def hostName: String
/**
* Fetch a sequence of blocks from a remote node asynchronously,
* available only after [[init]] is invoked.
*
* Note that this API takes a sequence so the implementation can batch requests, and does not
* return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as
* the data of a block is fetched, rather than waiting for all blocks to be fetched.
*/
override def fetchBlocks(
host: String,
port: Int,
execId: String,
blockIds: Array[String],
listener: BlockFetchingListener,
tempFileManager: DownloadFileManager): Unit
NettyBlockTransferService 类:
/**
* A BlockTransferService that uses Netty to fetch a set of blocks at time.
*/
private[spark] class NettyBlockTransferService(
conf: SparkConf,
securityManager: SecurityManager,
bindAddress: String,
override val hostName: String,
_port: Int,
numCores: Int)
extends BlockTransferService {
NettyBlockTransferService.fetchBlocks函数如下:
override def fetchBlocks(
host: String,
port: Int,
execId: String,
blockIds: Array[String],
listener: BlockFetchingListener,
tempFileManager: DownloadFileManager): Unit = {
logTrace(s"Fetch blocks from $host:$port (executor id $execId)")
try {
val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter {
override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) {
val client = clientFactory.createClient(host, port)
new OneForOneBlockFetcher(client, appId, execId, blockIds, listener,
transportConf, tempFileManager).start()
}
}
val maxRetries = transportConf.maxIORetries()
if (maxRetries > 0) {
// Note this Fetcher will correctly handle maxRetries == 0; we avoid it just in case there's
// a bug in this code. We should remove the if statement once we're sure of the stability.
new RetryingBlockFetcher(transportConf, blockFetchStarter, blockIds, listener).start()
} else {
blockFetchStarter.createAndStart(blockIds, listener)
}
} catch {
case e: Exception =>
logError("Exception while beginning fetchBlocks", e)
blockIds.foreach(listener.onBlockFetchFailure(_, e))
}
}
NettyBlockTransferService.fetchBlocks函数里先创建TransportClient
val client = clientFactory.createClient(host, port)
往后的代码NettyBlockTransferService.fetchBlocks函数横竖都会调用到 OneForOneBlockFetcher,而new RetryingBlockFetcher(transportConf, blockFetchStarter, blockIds, listener).start()调用与 下面调用的区别是:前者会retry,后者只会fetch一次。
val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter {
override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) {
val client = clientFactory.createClient(host, port)
new OneForOneBlockFetcher(client, appId, execId, blockIds, listener,
transportConf, tempFileManager).start()
}
}
8)继续分析OneForOneBlockFetcher.start() 函数,它会先发一个RPC调用。如果成功了,回调onSuccess;否则调用onFailure。
client.sendRpc(openMessage.toByteBuffer(), new RpcResponseCallback() {
public void onSuccess(ByteBuffer response) {...}
public void onFailure(Throwable e) {...}
}
public void onSuccess(ByteBuffer response)函数里,会解析response得到streamHandle(它包含了numChunks和streamId等信息)
streamHandle = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response);
logger.trace("Successfully opened blocks {}, preparing to fetch chunks.", streamHandle);
MapoutputTrackMaster表示已经Ready,可以传送chunk(即请求的blockIds)数据。然后NettyBlockTransferService.fetchBlocks函数里创建的TransportClient会通过它的stream()或者fetchChunk()去下载remote block.
// Immediately request all chunks -- we expect that the total size of the request is
// reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]].
for (int i = 0; i < streamHandle.numChunks; i++) {
if (downloadFileManager != null) {
client.stream(OneForOneStreamManager.genStreamChunkId(streamHandle.streamId, i),
new DownloadCallback(i));
} else {
client.fetchChunk(streamHandle.streamId, i, chunkCallback);
}
}
这里要注意到:stream()的入参DownloadCallback(i),fetchChunk()的入参是ChunkReceivedCallback类型的变量 chunkCallback。DownloadCallback(i) 最终使用createTempLocalBlock,也就是磁盘。而chunkCallback使用的是ManagedBuffer。(这里是有优化的空间的)
9)现在要回到1)里的调用栈了
next:400, ShuffleBlockFetcherIterator (org.apache.spark.storage)
hasNext:31, CompletionIterator (org.apache.spark.util)
hasNext:37, InterruptibleIterator (org.apache.spark)
insertAll:199, ExternalSorter (org.apache.spark.util.collection)
ExternalSorter里的insertAll代码如下,while (records.hasNext)里的records就是ShuffleBlockFetcherIterator对象(它extends Iterator[(BlockId, InputStream)])。
def insertAll(records: Iterator[Product2[K, V]]): Unit = {
// TODO: stop combining if we find that the reduction factor isn't high
val shouldCombine = aggregator.isDefined
if (shouldCombine) {
// Combine values in-memory first using our AppendOnlyMap
val mergeValue = aggregator.get.mergeValue
val createCombiner = aggregator.get.createCombiner
var kv: Product2[K, V] = null
val update = (hadValue: Boolean, oldValue: C) => {
if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
}
while (records.hasNext) {
addElementsRead()
kv = records.next()
map.changeValue((getPartition(kv._1), kv._1), update)
maybeSpillCollection(usingMap = true)
}
} else {
// Stick values into our buffer
while (records.hasNext) {
addElementsRead()
val kv = records.next()
buffer.insert(getPartition(kv._1), kv._1, kv._2.asInstanceOf[C])
maybeSpillCollection(usingMap = false)
}
}
而ShuffleBlockFetcherIterator.hasNext只是检查所有的blocks是否已经取完。
override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch
如果没有取完,就调用fetchUpToMaxBytes()继续发送fetchRequest.
/**
* Fetches the next (BlockId, InputStream). If a task fails, the ManagedBuffers
* underlying each InputStream will be freed by the cleanup() method registered with the
* TaskCompletionListener. However, callers should close() these InputStreams
* as soon as they are no longer needed, in order to release memory as early as possible.
*
* Throws a FetchFailedException if the next block could not be fetched.
*/
override def next(): (BlockId, InputStream) = {
if (!hasNext) {
throw new NoSuchElementException
}
numBlocksProcessed += 1
var result: FetchResult = null
var input: InputStream = null
// Take the next fetched result and try to decompress it to detect data corruption,
// then fetch it one more time if it's corrupt, throw FailureFetchResult if the second fetch
// is also corrupt, so the previous stage could be retried.
// For local shuffle block, throw FailureFetchResult for the first IOException.
while (result == null) {
val startFetchWait = System.currentTimeMillis()
result = results.take()
val stopFetchWait = System.currentTimeMillis()
shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait)
result match {
case r @ SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) =>
if (address != blockManager.blockManagerId) {
numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1
shuffleMetrics.incRemoteBytesRead(buf.size)
if (buf.isInstanceOf[FileSegmentManagedBuffer]) {
shuffleMetrics.incRemoteBytesReadToDisk(buf.size)
}
shuffleMetrics.incRemoteBlocksFetched(1)
}
if (!localBlocks.contains(blockId)) {
bytesInFlight -= size
}
if (isNetworkReqDone) {
reqsInFlight -= 1
logDebug("Number of requests in flight " + reqsInFlight)
}
if (buf.size == 0) {
// We will never legitimately receive a zero-size block. All blocks with zero records
// have zero size and all zero-size blocks have no records (and hence should never
// have been requested in the first place). This statement relies on behaviors of the
// shuffle writers, which are guaranteed by the following test cases:
//
// - BypassMergeSortShuffleWriterSuite: "write with some empty partitions"
// - UnsafeShuffleWriterSuite: "writeEmptyIterator"
// - DiskBlockObjectWriterSuite: "commit() and close() without ever opening or writing"
//
// There is not an explicit test for SortShuffleWriter but the underlying APIs that
// uses are shared by the UnsafeShuffleWriter (both writers use DiskBlockObjectWriter
// which returns a zero-size from commitAndGet() in case no records were written
// since the last call.
val msg = s"Received a zero-size buffer for block $blockId from $address " +
s"(expectedApproxSize = $size, isNetworkReqDone=$isNetworkReqDone)"
throwFetchFailedException(blockId, address, new IOException(msg))
}
val in = try {
buf.createInputStream()
} catch {
// The exception could only be throwed by local shuffle block
case e: IOException =>
assert(buf.isInstanceOf[FileSegmentManagedBuffer])
logError("Failed to create input stream from local block", e)
buf.release()
throwFetchFailedException(blockId, address, e)
}
var isStreamCopied: Boolean = false
try {
input = streamWrapper(blockId, in)
// Only copy the stream if it's wrapped by compression or encryption, also the size of
// block is small (the decompressed block is smaller than maxBytesInFlight)
if (detectCorrupt && !input.eq(in) && size < maxBytesInFlight / 3) {
isStreamCopied = true
val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate)
// Decompress the whole block at once to detect any corruption, which could increase
// the memory usage tne potential increase the chance of OOM.
// TODO: manage the memory used here, and spill it into disk in case of OOM.
Utils.copyStream(input, out, closeStreams = true)
input = out.toChunkedByteBuffer.toInputStream(dispose = true)
}
} catch {
case e: IOException =>
buf.release()
if (buf.isInstanceOf[FileSegmentManagedBuffer]
|| corruptedBlocks.contains(blockId)) {
throwFetchFailedException(blockId, address, e)
} else {
logWarning(s"got an corrupted block $blockId from $address, fetch again", e)
corruptedBlocks += blockId
fetchRequests += FetchRequest(address, Array((blockId, size)))
result = null
}
} finally {
// TODO: release the buf here to free memory earlier
if (isStreamCopied) {
in.close()
}
}
case FailureFetchResult(blockId, address, e) =>
throwFetchFailedException(blockId, address, e)
}
// Send fetch requests up to maxBytesInFlight
fetchUpToMaxBytes()
}
currentResult = result.asInstanceOf[SuccessFetchResult]
(currentResult.blockId, new BufferReleasingInputStream(input, this))
}
再继续就是ExternalSorter处理shuffle reader 的数据了。
三、测试新的配置:
core-site.xml:
//HDD
<property>
<name>hadoop.tmp.dir</name>
<value>/hdd/data/tmpDir</value>
</property>
=>
//SSD
<property>
<name>hadoop.tmp.dir</name>
<value>/ssd/data/tmpDir</value>
</property>
spark.conf
spark.maxRemoteBlockSizeFetchToMem 83886080
测试结果:
//HDD
ScalaSparkTerasort 2018-12-29 16:01:25 32000000000 279.339 114556148 57278074
//SSD
ScalaSparkTerasort 2019-01-02 09:45:02 32000000000 196.287 163026588 81513294
//SSD + spark.maxRemoteBlockSizeFetchToMem
ScalaSparkTerasort 2019-01-02 09:37:57 32000000000 186.881 171231960 85615980
结束,明年2019见。配个图