/
ShuffleBlockFetcherIterator.scala
969 lines (871 loc) · 39.7 KB
/
ShuffleBlockFetcherIterator.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.storage
import java.io.{InputStream, IOException}
import java.nio.channels.ClosedByInterruptException
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
import javax.annotation.concurrent.GuardedBy
import scala.collection.mutable
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, LinkedHashMap, Queue}
import scala.util.{Failure, Success}
import org.apache.commons.io.IOUtils
import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
import org.apache.spark.network.shuffle._
import org.apache.spark.network.util.TransportConf
import org.apache.spark.shuffle.{FetchFailedException, ShuffleReadMetricsReporter}
import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils}
/**
* An iterator that fetches multiple blocks. For local blocks, it fetches from the local block
* manager. For remote blocks, it fetches them using the provided BlockTransferService.
*
* This creates an iterator of (BlockID, InputStream) tuples so the caller can handle blocks
* in a pipelined fashion as they are received.
*
* The implementation throttles the remote fetches so they don't exceed maxBytesInFlight to avoid
* using too much memory.
*
* @param context [[TaskContext]], used for metrics update
* @param shuffleClient [[BlockStoreClient]] for fetching remote blocks
* @param blockManager [[BlockManager]] for reading local blocks
* @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]].
* For each block we also require two info: 1. the size (in bytes as a long
* field) in order to throttle the memory usage; 2. the mapIndex for this
* block, which indicate the index in the map stage.
* Note that zero-sized blocks are already excluded, which happened in
* [[org.apache.spark.MapOutputTracker.convertMapStatuses]].
* @param streamWrapper A function to wrap the returned input stream.
* @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point.
* @param maxReqsInFlight max number of remote requests to fetch blocks at any given point.
* @param maxBlocksInFlightPerAddress max number of shuffle blocks being fetched at any given point
* for a given remote host:port.
* @param maxReqSizeShuffleToMem max size (in bytes) of a request that can be shuffled to memory.
* @param detectCorrupt whether to detect any corruption in fetched blocks.
* @param shuffleMetrics used to report shuffle metrics.
* @param doBatchFetch fetch continuous shuffle blocks from same executor in batch if the server
* side supports.
*/
private[spark]
final class ShuffleBlockFetcherIterator(
context: TaskContext,
shuffleClient: BlockStoreClient,
blockManager: BlockManager,
blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])],
streamWrapper: (BlockId, InputStream) => InputStream,
maxBytesInFlight: Long,
maxReqsInFlight: Int,
maxBlocksInFlightPerAddress: Int,
maxReqSizeShuffleToMem: Long,
detectCorrupt: Boolean,
detectCorruptUseExtraMemory: Boolean,
shuffleMetrics: ShuffleReadMetricsReporter,
doBatchFetch: Boolean)
extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging {
import ShuffleBlockFetcherIterator._
// 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.
private val targetRemoteRequestSize = math.max(maxBytesInFlight / 5, 1L)
/**
* Total number of blocks to fetch. This should be equal to the total number of blocks
* in [[blocksByAddress]] because we already filter out zero-sized blocks in [[blocksByAddress]].
*/
private[this] var numBlocksToFetch = 0
/**
* The number of blocks processed by the caller. The iterator is exhausted when
* [[numBlocksProcessed]] == [[numBlocksToFetch]].
*/
private[this] var numBlocksProcessed = 0
private[this] val startTimeNs = System.nanoTime()
/** Local blocks to fetch, excluding zero-sized blocks. */
private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]()
/** Host local blockIds to fetch by executors, excluding zero-sized blocks. */
private[this] val hostLocalBlocksByExecutor =
LinkedHashMap[BlockManagerId, Seq[(BlockId, Long, Int)]]()
/** Host local blocks to fetch, excluding zero-sized blocks. */
private[this] val hostLocalBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]()
/**
* A queue to hold our results. This turns the asynchronous model provided by
* [[org.apache.spark.network.BlockTransferService]] into a synchronous model (iterator).
*/
private[this] val results = new LinkedBlockingQueue[FetchResult]
/**
* Current [[FetchResult]] being processed. We track this so we can release the current buffer
* in case of a runtime exception when processing the current buffer.
*/
@volatile private[this] var currentResult: SuccessFetchResult = null
/**
* Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that
* the number of bytes in flight is limited to maxBytesInFlight.
*/
private[this] val fetchRequests = new Queue[FetchRequest]
/**
* Queue of fetch requests which could not be issued the first time they were dequeued. These
* requests are tried again when the fetch constraints are satisfied.
*/
private[this] val deferredFetchRequests = new HashMap[BlockManagerId, Queue[FetchRequest]]()
/** Current bytes in flight from our requests */
private[this] var bytesInFlight = 0L
/** Current number of requests in flight */
private[this] var reqsInFlight = 0
/** Current number of blocks in flight per host:port */
private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]()
/**
* The blocks that can't be decompressed successfully, it is used to guarantee that we retry
* at most once for those corrupted blocks.
*/
private[this] val corruptedBlocks = mutable.HashSet[BlockId]()
/**
* Whether the iterator is still active. If isZombie is true, the callback interface will no
* longer place fetched blocks into [[results]].
*/
@GuardedBy("this")
private[this] var isZombie = false
/**
* A set to store the files used for shuffling remote huge blocks. Files in this set will be
* deleted when cleanup. This is a layer of defensiveness against disk file leaks.
*/
@GuardedBy("this")
private[this] val shuffleFilesSet = mutable.HashSet[DownloadFile]()
private[this] val onCompleteCallback = new ShuffleFetchCompletionListener(this)
initialize()
// Decrements the buffer reference count.
// The currentResult is set to null to prevent releasing the buffer again on cleanup()
private[storage] def releaseCurrentResultBuffer(): Unit = {
// Release the current buffer if necessary
if (currentResult != null) {
currentResult.buf.release()
}
currentResult = null
}
override def createTempFile(transportConf: TransportConf): DownloadFile = {
// we never need to do any encryption or decryption here, regardless of configs, because that
// is handled at another layer in the code. When encryption is enabled, shuffle data is written
// to disk encrypted in the first place, and sent over the network still encrypted.
new SimpleDownloadFile(
blockManager.diskBlockManager.createTempLocalBlock()._2, transportConf)
}
override def registerTempFileToClean(file: DownloadFile): Boolean = synchronized {
if (isZombie) {
false
} else {
shuffleFilesSet += file
true
}
}
/**
* Mark the iterator as zombie, and release all buffers that haven't been deserialized yet.
*/
private[storage] def cleanup(): Unit = {
synchronized {
isZombie = true
}
releaseCurrentResultBuffer()
// Release buffers in the results queue
val iter = results.iterator()
while (iter.hasNext) {
val result = iter.next()
result match {
case SuccessFetchResult(_, _, address, _, buf, _) =>
if (address != blockManager.blockManagerId) {
shuffleMetrics.incRemoteBytesRead(buf.size)
if (buf.isInstanceOf[FileSegmentManagedBuffer]) {
shuffleMetrics.incRemoteBytesReadToDisk(buf.size)
}
shuffleMetrics.incRemoteBlocksFetched(1)
}
buf.release()
case _ =>
}
}
shuffleFilesSet.foreach { file =>
if (!file.delete()) {
logWarning("Failed to cleanup shuffle fetch temp file " + file.path())
}
}
}
private[this] def sendRequest(req: FetchRequest): Unit = {
logDebug("Sending request for %d blocks (%s) from %s".format(
req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort))
bytesInFlight += req.size
reqsInFlight += 1
// so we can look up the block info of each blockID
val infoMap = req.blocks.map {
case FetchBlockInfo(blockId, size, mapIndex) => (blockId.toString, (size, mapIndex))
}.toMap
val remainingBlocks = new HashSet[String]() ++= infoMap.keys
val blockIds = req.blocks.map(_.blockId.toString)
val address = req.address
val blockFetchingListener = new BlockFetchingListener {
override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
// Only add the buffer to results queue if the iterator is not zombie,
// i.e. cleanup() has not been called yet.
ShuffleBlockFetcherIterator.this.synchronized {
if (!isZombie) {
// Increment the ref count because we need to pass this to a different thread.
// This needs to be released after use.
buf.retain()
remainingBlocks -= blockId
results.put(new SuccessFetchResult(BlockId(blockId), infoMap(blockId)._2,
address, infoMap(blockId)._1, buf, remainingBlocks.isEmpty))
logDebug("remainingBlocks: " + remainingBlocks)
}
}
logTrace(s"Got remote block $blockId after ${Utils.getUsedTimeNs(startTimeNs)}")
}
override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
results.put(new FailureFetchResult(BlockId(blockId), infoMap(blockId)._2, address, e))
}
}
// 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)
}
}
private[this] def partitionBlocksByFetchMode(): ArrayBuffer[FetchRequest] = {
logDebug(s"maxBytesInFlight: $maxBytesInFlight, targetRemoteRequestSize: "
+ s"$targetRemoteRequestSize, maxBlocksInFlightPerAddress: $maxBlocksInFlightPerAddress")
// Partition to local, host-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 collectedRemoteRequests = new ArrayBuffer[FetchRequest]
var localBlockBytes = 0L
var hostLocalBlockBytes = 0L
var remoteBlockBytes = 0L
var numRemoteBlocks = 0
val hostLocalDirReadingEnabled =
blockManager.hostLocalDirManager != null && blockManager.hostLocalDirManager.isDefined
for ((address, blockInfos) <- blocksByAddress) {
if (address.executorId == blockManager.blockManagerId.executorId) {
checkBlockSizes(blockInfos)
val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded(
blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)).to[ArrayBuffer])
localBlocks ++= mergedBlockInfos.map(info => (info.blockId, info.mapIndex))
localBlockBytes += mergedBlockInfos.map(_.size).sum
} else if (hostLocalDirReadingEnabled && address.host == blockManager.blockManagerId.host) {
checkBlockSizes(blockInfos)
val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded(
blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)).to[ArrayBuffer])
val blocksForAddress =
mergedBlockInfos.map(info => (info.blockId, info.size, info.mapIndex))
hostLocalBlocksByExecutor += address -> blocksForAddress
hostLocalBlocks ++= blocksForAddress.map(info => (info._1, info._3))
hostLocalBlockBytes += mergedBlockInfos.map(_.size).sum
} else {
numRemoteBlocks += blockInfos.size
remoteBlockBytes += blockInfos.map(_._2).sum
collectFetchRequests(address, blockInfos, collectedRemoteRequests)
}
}
val totalBytes = localBlockBytes + remoteBlockBytes + hostLocalBlockBytes
logInfo(s"Getting $numBlocksToFetch (${Utils.bytesToString(totalBytes)}) non-empty blocks " +
s"including ${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local and " +
s"${hostLocalBlocks.size} (${Utils.bytesToString(hostLocalBlockBytes)}) " +
s"host-local and $numRemoteBlocks (${Utils.bytesToString(remoteBlockBytes)}) remote blocks")
collectedRemoteRequests
}
private def collectFetchRequests(
address: BlockManagerId,
blockInfos: Seq[(BlockId, Long, Int)],
collectedRemoteRequests: ArrayBuffer[FetchRequest]): Unit = {
val iterator = blockInfos.iterator
var curRequestSize = 0L
var curBlocks = new ArrayBuffer[FetchBlockInfo]
def createFetchRequest(blocks: Seq[FetchBlockInfo]): Unit = {
collectedRemoteRequests += FetchRequest(address, blocks)
logDebug(s"Creating fetch request of $curRequestSize at $address "
+ s"with ${blocks.size} blocks")
}
def createFetchRequests(): Unit = {
val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks)
curBlocks = new ArrayBuffer[FetchBlockInfo]
if (mergedBlocks.length <= maxBlocksInFlightPerAddress) {
createFetchRequest(mergedBlocks)
} else {
mergedBlocks.grouped(maxBlocksInFlightPerAddress).foreach { blocks =>
if (blocks.length == maxBlocksInFlightPerAddress) {
createFetchRequest(blocks)
} else {
// The last group does not exceed `maxBlocksInFlightPerAddress`. Put it back
// to `curBlocks`.
curBlocks = blocks
numBlocksToFetch -= blocks.size
}
}
}
curRequestSize = curBlocks.map(_.size).sum
}
while (iterator.hasNext) {
val (blockId, size, mapIndex) = iterator.next()
assertPositiveBlockSize(blockId, size)
curBlocks += FetchBlockInfo(blockId, size, mapIndex)
curRequestSize += size
// For batch fetch, the actual block in flight should count for merged block.
val mayExceedsMaxBlocks = !doBatchFetch && curBlocks.size >= maxBlocksInFlightPerAddress
if (curRequestSize >= targetRemoteRequestSize || mayExceedsMaxBlocks) {
createFetchRequests()
}
}
// Add in the final request
if (curBlocks.nonEmpty) {
createFetchRequests()
}
}
private def assertPositiveBlockSize(blockId: BlockId, blockSize: Long): Unit = {
if (blockSize < 0) {
throw BlockException(blockId, "Negative block size " + size)
} else if (blockSize == 0) {
throw BlockException(blockId, "Zero-sized blocks should be excluded.")
}
}
private def checkBlockSizes(blockInfos: Seq[(BlockId, Long, Int)]): Unit = {
blockInfos.foreach { case (blockId, size, _) => assertPositiveBlockSize(blockId, size) }
}
private[this] def mergeContinuousShuffleBlockIdsIfNeeded(
blocks: ArrayBuffer[FetchBlockInfo]): ArrayBuffer[FetchBlockInfo] = {
val result = if (doBatchFetch) {
var curBlocks = new ArrayBuffer[FetchBlockInfo]
val mergedBlockInfo = new ArrayBuffer[FetchBlockInfo]
def mergeFetchBlockInfo(toBeMerged: ArrayBuffer[FetchBlockInfo]): FetchBlockInfo = {
val startBlockId = toBeMerged.head.blockId.asInstanceOf[ShuffleBlockId]
// The last merged block may comes from the input, and we can merge more blocks
// into it, if the map id is the same.
def shouldMergeIntoPreviousBatchBlockId =
mergedBlockInfo.last.blockId.asInstanceOf[ShuffleBlockBatchId].mapId == startBlockId.mapId
val startReduceId = if (mergedBlockInfo.nonEmpty && shouldMergeIntoPreviousBatchBlockId) {
// Remove the previous batch block id as we will add a new one to replace it.
mergedBlockInfo.remove(mergedBlockInfo.length - 1).blockId
.asInstanceOf[ShuffleBlockBatchId].startReduceId
} else {
startBlockId.reduceId
}
FetchBlockInfo(
ShuffleBlockBatchId(
startBlockId.shuffleId,
startBlockId.mapId,
startReduceId,
toBeMerged.last.blockId.asInstanceOf[ShuffleBlockId].reduceId + 1),
toBeMerged.map(_.size).sum,
toBeMerged.head.mapIndex)
}
val iter = blocks.iterator
while (iter.hasNext) {
val info = iter.next()
// It's possible that the input block id is already a batch ID. For example, we merge some
// blocks, and then make fetch requests with the merged blocks according to "max blocks per
// request". The last fetch request may be too small, and we give up and put the remaining
// merged blocks back to the input list.
if (info.blockId.isInstanceOf[ShuffleBlockBatchId]) {
mergedBlockInfo += info
} else {
if (curBlocks.isEmpty) {
curBlocks += info
} else {
val curBlockId = info.blockId.asInstanceOf[ShuffleBlockId]
val currentMapId = curBlocks.head.blockId.asInstanceOf[ShuffleBlockId].mapId
if (curBlockId.mapId != currentMapId) {
mergedBlockInfo += mergeFetchBlockInfo(curBlocks)
curBlocks.clear()
}
curBlocks += info
}
}
}
if (curBlocks.nonEmpty) {
mergedBlockInfo += mergeFetchBlockInfo(curBlocks)
}
mergedBlockInfo
} else {
blocks
}
// update metrics
numBlocksToFetch += result.size
result
}
/**
* Fetch the local blocks while we are fetching remote blocks. This is ok because
* `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we
* track in-memory are the ManagedBuffer references themselves.
*/
private[this] def fetchLocalBlocks(): Unit = {
logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}")
val iter = localBlocks.iterator
while (iter.hasNext) {
val (blockId, mapIndex) = iter.next()
try {
val buf = blockManager.getLocalBlockData(blockId)
shuffleMetrics.incLocalBlocksFetched(1)
shuffleMetrics.incLocalBytesRead(buf.size)
buf.retain()
results.put(new SuccessFetchResult(blockId, mapIndex, blockManager.blockManagerId,
buf.size(), buf, false))
} catch {
// If we see an exception, stop immediately.
case e: Exception =>
e match {
// ClosedByInterruptException is an excepted exception when kill task,
// don't log the exception stack trace to avoid confusing users.
// See: SPARK-28340
case ce: ClosedByInterruptException =>
logError("Error occurred while fetching local blocks, " + ce.getMessage)
case ex: Exception => logError("Error occurred while fetching local blocks", ex)
}
results.put(new FailureFetchResult(blockId, mapIndex, blockManager.blockManagerId, e))
return
}
}
}
private[this] def fetchHostLocalBlock(
blockId: BlockId,
mapIndex: Int,
localDirs: Array[String],
blockManagerId: BlockManagerId): Boolean = {
try {
val buf = blockManager.getHostLocalShuffleData(blockId, localDirs)
buf.retain()
results.put(SuccessFetchResult(blockId, mapIndex, blockManagerId, buf.size(), buf,
isNetworkReqDone = false))
true
} catch {
case e: Exception =>
// If we see an exception, stop immediately.
logError(s"Error occurred while fetching local blocks", e)
results.put(FailureFetchResult(blockId, mapIndex, blockManagerId, e))
false
}
}
/**
* Fetch the host-local blocks while we are fetching remote blocks. This is ok because
* `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we
* track in-memory are the ManagedBuffer references themselves.
*/
private[this] def fetchHostLocalBlocks(hostLocalDirManager: HostLocalDirManager): Unit = {
val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
hostLocalBlocksByExecutor
.map { case (hostLocalBmId, bmInfos) =>
(hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
}.partition(_._3.isDefined)
val bmId = blockManager.blockManagerId
val immutableHostLocalBlocksWithoutDirs =
hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
hostLocalBmId -> bmInfos
}.toMap
if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
case Success(dirs) =>
immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
blockInfos.takeWhile { case (blockId, _, mapIndex) =>
fetchHostLocalBlock(
blockId,
mapIndex,
dirs.get(hostLocalBmId.executorId),
hostLocalBmId)
}
}
logDebug(s"Got host-local blocks (without cached executors' dir) in " +
s"${Utils.getUsedTimeNs(startTimeNs)}")
case Failure(throwable) =>
logError(s"Error occurred while fetching host local blocks", throwable)
val (hostLocalBmId, blockInfoSeq) = immutableHostLocalBlocksWithoutDirs.head
val (blockId, _, mapIndex) = blockInfoSeq.head
results.put(FailureFetchResult(blockId, mapIndex, hostLocalBmId, throwable))
}
}
if (hostLocalBlocksWithCachedDirs.nonEmpty) {
logDebug(s"Synchronous fetching host-local blocks with cached executors' dir: " +
s"${hostLocalBlocksWithCachedDirs.mkString(", ")}")
hostLocalBlocksWithCachedDirs.foreach { case (_, blockInfos, localDirs) =>
blockInfos.foreach { case (blockId, _, mapIndex) =>
if (!fetchHostLocalBlock(blockId, mapIndex, localDirs.get, bmId)) {
return
}
}
}
logDebug(s"Got host-local blocks (with cached executors' dir) in " +
s"${Utils.getUsedTimeNs(startTimeNs)}")
}
}
private[this] def initialize(): Unit = {
// Add a task completion callback (called in both success case and failure case) to cleanup.
context.addTaskCompletionListener(onCompleteCallback)
// Partition blocks by the different fetch modes: local, host-local and remote blocks.
val remoteRequests = partitionBlocksByFetchMode()
// 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(s"Started $numFetches remote fetches in ${Utils.getUsedTimeNs(startTimeNs)}")
// Get Local Blocks
fetchLocalBlocks()
logDebug(s"Got local blocks in ${Utils.getUsedTimeNs(startTimeNs)}")
if (hostLocalBlocks.nonEmpty) {
blockManager.hostLocalDirManager.foreach(fetchHostLocalBlocks)
}
}
override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch
/**
* 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
var streamCompressedOrEncrypted: Boolean = false
// 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.nanoTime()
result = results.take()
val fetchWaitTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startFetchWait)
shuffleMetrics.incFetchWaitTime(fetchWaitTime)
result match {
case r @ SuccessFetchResult(blockId, mapIndex, address, size, buf, isNetworkReqDone) =>
if (address != blockManager.blockManagerId) {
if (hostLocalBlocks.contains(blockId -> mapIndex)) {
shuffleMetrics.incLocalBlocksFetched(1)
shuffleMetrics.incLocalBytesRead(buf.size)
} else {
numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1
shuffleMetrics.incRemoteBytesRead(buf.size)
if (buf.isInstanceOf[FileSegmentManagedBuffer]) {
shuffleMetrics.incRemoteBytesReadToDisk(buf.size)
}
shuffleMetrics.incRemoteBlocksFetched(1)
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, mapIndex, 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])
e match {
case ce: ClosedByInterruptException =>
logError("Failed to create input stream from local block, " +
ce.getMessage)
case e: IOException => logError("Failed to create input stream from local block", e)
}
buf.release()
throwFetchFailedException(blockId, mapIndex, address, e)
}
try {
input = streamWrapper(blockId, in)
// If the stream is compressed or wrapped, then we optionally decompress/unwrap the
// first maxBytesInFlight/3 bytes into memory, to check for corruption in that portion
// of the data. But even if 'detectCorruptUseExtraMemory' configuration is off, or if
// the corruption is later, we'll still detect the corruption later in the stream.
streamCompressedOrEncrypted = !input.eq(in)
if (streamCompressedOrEncrypted && detectCorruptUseExtraMemory) {
// TODO: manage the memory used here, and spill it into disk in case of OOM.
input = Utils.copyStreamUpTo(input, maxBytesInFlight / 3)
}
} catch {
case e: IOException =>
buf.release()
if (buf.isInstanceOf[FileSegmentManagedBuffer]
|| corruptedBlocks.contains(blockId)) {
throwFetchFailedException(blockId, mapIndex, address, e)
} else {
logWarning(s"got an corrupted block $blockId from $address, fetch again", e)
corruptedBlocks += blockId
fetchRequests += FetchRequest(
address, Array(FetchBlockInfo(blockId, size, mapIndex)))
result = null
}
} finally {
// TODO: release the buf here to free memory earlier
if (input == null) {
// Close the underlying stream if there was an issue in wrapping the stream using
// streamWrapper
in.close()
}
}
case FailureFetchResult(blockId, mapIndex, address, e) =>
throwFetchFailedException(blockId, mapIndex, address, e)
}
// Send fetch requests up to maxBytesInFlight
fetchUpToMaxBytes()
}
currentResult = result.asInstanceOf[SuccessFetchResult]
(currentResult.blockId,
new BufferReleasingInputStream(
input,
this,
currentResult.blockId,
currentResult.mapIndex,
currentResult.address,
detectCorrupt && streamCompressedOrEncrypted))
}
def toCompletionIterator: Iterator[(BlockId, InputStream)] = {
CompletionIterator[(BlockId, InputStream), this.type](this,
onCompleteCallback.onComplete(context))
}
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)
}
}
def send(remoteAddress: BlockManagerId, request: FetchRequest): Unit = {
sendRequest(request)
numBlocksInFlightPerAddress(remoteAddress) =
numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size
}
def isRemoteBlockFetchable(fetchReqQueue: Queue[FetchRequest]): Boolean = {
fetchReqQueue.nonEmpty &&
(bytesInFlight == 0 ||
(reqsInFlight + 1 <= maxReqsInFlight &&
bytesInFlight + fetchReqQueue.front.size <= maxBytesInFlight))
}
// Checks if sending a new fetch request will exceed the max no. of blocks being fetched from a
// given remote address.
def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: FetchRequest): Boolean = {
numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size >
maxBlocksInFlightPerAddress
}
}
private[storage] def throwFetchFailedException(
blockId: BlockId,
mapIndex: Int,
address: BlockManagerId,
e: Throwable) = {
blockId match {
case ShuffleBlockId(shufId, mapId, reduceId) =>
throw new FetchFailedException(address, shufId, mapId, mapIndex, reduceId, e)
case ShuffleBlockBatchId(shuffleId, mapId, startReduceId, _) =>
throw new FetchFailedException(address, shuffleId, mapId, mapIndex, startReduceId, e)
case _ =>
throw new SparkException(
"Failed to get block " + blockId + ", which is not a shuffle block", e)
}
}
}
/**
* Helper class that ensures a ManagedBuffer is released upon InputStream.close() and
* also detects stream corruption if streamCompressedOrEncrypted is true
*/
private class BufferReleasingInputStream(
// This is visible for testing
private[storage] val delegate: InputStream,
private val iterator: ShuffleBlockFetcherIterator,
private val blockId: BlockId,
private val mapIndex: Int,
private val address: BlockManagerId,
private val detectCorruption: Boolean)
extends InputStream {
private[this] var closed = false
override def read(): Int = {
try {
delegate.read()
} catch {
case e: IOException if detectCorruption =>
IOUtils.closeQuietly(this)
iterator.throwFetchFailedException(blockId, mapIndex, address, e)
}
}
override def close(): Unit = {
if (!closed) {
delegate.close()
iterator.releaseCurrentResultBuffer()
closed = true
}
}
override def available(): Int = delegate.available()
override def mark(readlimit: Int): Unit = delegate.mark(readlimit)
override def skip(n: Long): Long = {
try {
delegate.skip(n)
} catch {
case e: IOException if detectCorruption =>
IOUtils.closeQuietly(this)
iterator.throwFetchFailedException(blockId, mapIndex, address, e)
}
}
override def markSupported(): Boolean = delegate.markSupported()
override def read(b: Array[Byte]): Int = {
try {
delegate.read(b)
} catch {
case e: IOException if detectCorruption =>
IOUtils.closeQuietly(this)
iterator.throwFetchFailedException(blockId, mapIndex, address, e)
}
}
override def read(b: Array[Byte], off: Int, len: Int): Int = {
try {
delegate.read(b, off, len)
} catch {
case e: IOException if detectCorruption =>
IOUtils.closeQuietly(this)
iterator.throwFetchFailedException(blockId, mapIndex, address, e)
}
}
override def reset(): Unit = delegate.reset()
}
/**
* A listener to be called at the completion of the ShuffleBlockFetcherIterator
* @param data the ShuffleBlockFetcherIterator to process
*/
private class ShuffleFetchCompletionListener(var data: ShuffleBlockFetcherIterator)
extends TaskCompletionListener {
override def onTaskCompletion(context: TaskContext): Unit = {
if (data != null) {
data.cleanup()
// Null out the referent here to make sure we don't keep a reference to this
// ShuffleBlockFetcherIterator, after we're done reading from it, to let it be
// collected during GC. Otherwise we can hold metadata on block locations(blocksByAddress)
data = null
}
}
// Just an alias for onTaskCompletion to avoid confusing
def onComplete(context: TaskContext): Unit = this.onTaskCompletion(context)
}
private[storage]
object ShuffleBlockFetcherIterator {
/**
* The block information to fetch used in FetchRequest.
* @param blockId block id
* @param size estimated size of the block. Note that this is NOT the exact bytes.
* Size of remote block is used to calculate bytesInFlight.
* @param mapIndex the mapIndex for this block, which indicate the index in the map stage.
*/
private[storage] case class FetchBlockInfo(
blockId: BlockId,
size: Long,
mapIndex: Int)
/**
* A request to fetch blocks from a remote BlockManager.
* @param address remote BlockManager to fetch from.
* @param blocks Sequence of the information for blocks to fetch from the same address.
*/
case class FetchRequest(address: BlockManagerId, blocks: Seq[FetchBlockInfo]) {
val size = blocks.map(_.size).sum
}
/**
* Result of a fetch from a remote block.
*/
private[storage] sealed trait FetchResult {
val blockId: BlockId
val address: BlockManagerId
}
/**
* Result of a fetch from a remote block successfully.
* @param blockId block id
* @param mapIndex the mapIndex for this block, which indicate the index in the map stage.
* @param address BlockManager that the block was fetched from.
* @param size estimated size of the block. Note that this is NOT the exact bytes.
* Size of remote block is used to calculate bytesInFlight.
* @param buf `ManagedBuffer` for the content.
* @param isNetworkReqDone Is this the last network request for this host in this fetch request.
*/
private[storage] case class SuccessFetchResult(
blockId: BlockId,
mapIndex: Int,
address: BlockManagerId,
size: Long,
buf: ManagedBuffer,
isNetworkReqDone: Boolean) extends FetchResult {
require(buf != null)
require(size >= 0)
}
/**
* Result of a fetch from a remote block unsuccessfully.
* @param blockId block id
* @param mapIndex the mapIndex for this block, which indicate the index in the map stage
* @param address BlockManager that the block was attempted to be fetched from
* @param e the failure exception
*/
private[storage] case class FailureFetchResult(
blockId: BlockId,
mapIndex: Int,
address: BlockManagerId,
e: Throwable)
extends FetchResult
}