/
MapOutputTracker.scala
1014 lines (921 loc) · 39.2 KB
/
MapOutputTracker.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
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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
import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream}
import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit}
import java.util.concurrent.locks.ReentrantReadWriteLock
import scala.collection.JavaConverters._
import scala.collection.mutable.{HashMap, ListBuffer, Map}
import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration.Duration
import scala.reflect.ClassTag
import scala.util.control.NonFatal
import org.apache.commons.io.output.{ByteArrayOutputStream => ApacheByteArrayOutputStream}
import org.apache.spark.broadcast.{Broadcast, BroadcastManager}
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.io.CompressionCodec
import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv}
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.shuffle.MetadataFetchFailedException
import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId}
import org.apache.spark.util._
/**
* Helper class used by the [[MapOutputTrackerMaster]] to perform bookkeeping for a single
* ShuffleMapStage.
*
* This class maintains a mapping from map index to `MapStatus`. It also maintains a cache of
* serialized map statuses in order to speed up tasks' requests for map output statuses.
*
* All public methods of this class are thread-safe.
*/
private class ShuffleStatus(numPartitions: Int) extends Logging {
private val (readLock, writeLock) = {
val lock = new ReentrantReadWriteLock()
(lock.readLock(), lock.writeLock())
}
// All accesses to the following state must be guarded with `withReadLock` or `withWriteLock`.
private def withReadLock[B](fn: => B): B = {
readLock.lock()
try {
fn
} finally {
readLock.unlock()
}
}
private def withWriteLock[B](fn: => B): B = {
writeLock.lock()
try {
fn
} finally {
writeLock.unlock()
}
}
/**
* MapStatus for each partition. The index of the array is the map partition id.
* Each value in the array is the MapStatus for a partition, or null if the partition
* is not available. Even though in theory a task may run multiple times (due to speculation,
* stage retries, etc.), in practice the likelihood of a map output being available at multiple
* locations is so small that we choose to ignore that case and store only a single location
* for each output.
*/
// Exposed for testing
val mapStatuses = new Array[MapStatus](numPartitions)
/**
* The cached result of serializing the map statuses array. This cache is lazily populated when
* [[serializedMapStatus]] is called. The cache is invalidated when map outputs are removed.
*/
private[this] var cachedSerializedMapStatus: Array[Byte] = _
/**
* Broadcast variable holding serialized map output statuses array. When [[serializedMapStatus]]
* serializes the map statuses array it may detect that the result is too large to send in a
* single RPC, in which case it places the serialized array into a broadcast variable and then
* sends a serialized broadcast variable instead. This variable holds a reference to that
* broadcast variable in order to keep it from being garbage collected and to allow for it to be
* explicitly destroyed later on when the ShuffleMapStage is garbage-collected.
*/
private[this] var cachedSerializedBroadcast: Broadcast[Array[Byte]] = _
/**
* Counter tracking the number of partitions that have output. This is a performance optimization
* to avoid having to count the number of non-null entries in the `mapStatuses` array and should
* be equivalent to`mapStatuses.count(_ ne null)`.
*/
private[this] var _numAvailableOutputs: Int = 0
/**
* Register a map output. If there is already a registered location for the map output then it
* will be replaced by the new location.
*/
def addMapOutput(mapIndex: Int, status: MapStatus): Unit = withWriteLock {
if (mapStatuses(mapIndex) == null) {
_numAvailableOutputs += 1
invalidateSerializedMapOutputStatusCache()
}
mapStatuses(mapIndex) = status
}
/**
* Update the map output location (e.g. during migration).
*/
def updateMapOutput(mapId: Long, bmAddress: BlockManagerId): Unit = withWriteLock {
try {
val mapStatusOpt = mapStatuses.find(_.mapId == mapId)
mapStatusOpt match {
case Some(mapStatus) =>
logInfo(s"Updating map output for ${mapId} to ${bmAddress}")
mapStatus.updateLocation(bmAddress)
invalidateSerializedMapOutputStatusCache()
case None =>
logWarning(s"Asked to update map output ${mapId} for untracked map status.")
}
} catch {
case e: java.lang.NullPointerException =>
logWarning(s"Unable to update map output for ${mapId}, status removed in-flight")
}
}
/**
* Remove the map output which was served by the specified block manager.
* This is a no-op if there is no registered map output or if the registered output is from a
* different block manager.
*/
def removeMapOutput(mapIndex: Int, bmAddress: BlockManagerId): Unit = withWriteLock {
logDebug(s"Removing existing map output ${mapIndex} ${bmAddress}")
if (mapStatuses(mapIndex) != null && mapStatuses(mapIndex).location == bmAddress) {
_numAvailableOutputs -= 1
mapStatuses(mapIndex) = null
invalidateSerializedMapOutputStatusCache()
}
}
/**
* Removes all shuffle outputs associated with this host. Note that this will also remove
* outputs which are served by an external shuffle server (if one exists).
*/
def removeOutputsOnHost(host: String): Unit = withWriteLock {
logDebug(s"Removing outputs for host ${host}")
removeOutputsByFilter(x => x.host == host)
}
/**
* Removes all map outputs associated with the specified executor. Note that this will also
* remove outputs which are served by an external shuffle server (if one exists), as they are
* still registered with that execId.
*/
def removeOutputsOnExecutor(execId: String): Unit = withWriteLock {
logDebug(s"Removing outputs for execId ${execId}")
removeOutputsByFilter(x => x.executorId == execId)
}
/**
* Removes all shuffle outputs which satisfies the filter. Note that this will also
* remove outputs which are served by an external shuffle server (if one exists).
*/
def removeOutputsByFilter(f: BlockManagerId => Boolean): Unit = withWriteLock {
for (mapIndex <- mapStatuses.indices) {
if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) {
_numAvailableOutputs -= 1
mapStatuses(mapIndex) = null
invalidateSerializedMapOutputStatusCache()
}
}
}
/**
* Number of partitions that have shuffle outputs.
*/
def numAvailableOutputs: Int = withReadLock {
_numAvailableOutputs
}
/**
* Returns the sequence of partition ids that are missing (i.e. needs to be computed).
*/
def findMissingPartitions(): Seq[Int] = withReadLock {
val missing = (0 until numPartitions).filter(id => mapStatuses(id) == null)
assert(missing.size == numPartitions - _numAvailableOutputs,
s"${missing.size} missing, expected ${numPartitions - _numAvailableOutputs}")
missing
}
/**
* Serializes the mapStatuses array into an efficient compressed format. See the comments on
* `MapOutputTracker.serializeMapStatuses()` for more details on the serialization format.
*
* This method is designed to be called multiple times and implements caching in order to speed
* up subsequent requests. If the cache is empty and multiple threads concurrently attempt to
* serialize the map statuses then serialization will only be performed in a single thread and all
* other threads will block until the cache is populated.
*/
def serializedMapStatus(
broadcastManager: BroadcastManager,
isLocal: Boolean,
minBroadcastSize: Int,
conf: SparkConf): Array[Byte] = {
var result: Array[Byte] = null
withReadLock {
if (cachedSerializedMapStatus != null) {
result = cachedSerializedMapStatus
}
}
if (result == null) withWriteLock {
if (cachedSerializedMapStatus == null) {
val serResult = MapOutputTracker.serializeMapStatuses(
mapStatuses, broadcastManager, isLocal, minBroadcastSize, conf)
cachedSerializedMapStatus = serResult._1
cachedSerializedBroadcast = serResult._2
}
// The following line has to be outside if statement since it's possible that another thread
// initializes cachedSerializedMapStatus in-between `withReadLock` and `withWriteLock`.
result = cachedSerializedMapStatus
}
result
}
// Used in testing.
def hasCachedSerializedBroadcast: Boolean = withReadLock {
cachedSerializedBroadcast != null
}
/**
* Helper function which provides thread-safe access to the mapStatuses array.
* The function should NOT mutate the array.
*/
def withMapStatuses[T](f: Array[MapStatus] => T): T = withReadLock {
f(mapStatuses)
}
/**
* Clears the cached serialized map output statuses.
*/
def invalidateSerializedMapOutputStatusCache(): Unit = withWriteLock {
if (cachedSerializedBroadcast != null) {
// Prevent errors during broadcast cleanup from crashing the DAGScheduler (see SPARK-21444)
Utils.tryLogNonFatalError {
// Use `blocking = false` so that this operation doesn't hang while trying to send cleanup
// RPCs to dead executors.
cachedSerializedBroadcast.destroy()
}
cachedSerializedBroadcast = null
}
cachedSerializedMapStatus = null
}
}
private[spark] sealed trait MapOutputTrackerMessage
private[spark] case class GetMapOutputStatuses(shuffleId: Int)
extends MapOutputTrackerMessage
private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage
private[spark] case class GetMapOutputMessage(shuffleId: Int, context: RpcCallContext)
/** RpcEndpoint class for MapOutputTrackerMaster */
private[spark] class MapOutputTrackerMasterEndpoint(
override val rpcEnv: RpcEnv, tracker: MapOutputTrackerMaster, conf: SparkConf)
extends RpcEndpoint with Logging {
logDebug("init") // force eager creation of logger
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
case GetMapOutputStatuses(shuffleId: Int) =>
val hostPort = context.senderAddress.hostPort
logInfo(s"Asked to send map output locations for shuffle ${shuffleId} to ${hostPort}")
tracker.post(new GetMapOutputMessage(shuffleId, context))
case StopMapOutputTracker =>
logInfo("MapOutputTrackerMasterEndpoint stopped!")
context.reply(true)
stop()
}
}
/**
* Class that keeps track of the location of the map output of a stage. This is abstract because the
* driver and executor have different versions of the MapOutputTracker. In principle the driver-
* and executor-side classes don't need to share a common base class; the current shared base class
* is maintained primarily for backwards-compatibility in order to avoid having to update existing
* test code.
*/
private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging {
/** Set to the MapOutputTrackerMasterEndpoint living on the driver. */
var trackerEndpoint: RpcEndpointRef = _
/**
* The driver-side counter is incremented every time that a map output is lost. This value is sent
* to executors as part of tasks, where executors compare the new epoch number to the highest
* epoch number that they received in the past. If the new epoch number is higher then executors
* will clear their local caches of map output statuses and will re-fetch (possibly updated)
* statuses from the driver.
*/
protected var epoch: Long = 0
protected val epochLock = new AnyRef
/**
* Send a message to the trackerEndpoint and get its result within a default timeout, or
* throw a SparkException if this fails.
*/
protected def askTracker[T: ClassTag](message: Any): T = {
try {
trackerEndpoint.askSync[T](message)
} catch {
case e: Exception =>
logError("Error communicating with MapOutputTracker", e)
throw new SparkException("Error communicating with MapOutputTracker", e)
}
}
/** Send a one-way message to the trackerEndpoint, to which we expect it to reply with true. */
protected def sendTracker(message: Any): Unit = {
val response = askTracker[Boolean](message)
if (response != true) {
throw new SparkException(
"Error reply received from MapOutputTracker. Expecting true, got " + response.toString)
}
}
// For testing
def getMapSizesByExecutorId(shuffleId: Int, reduceId: Int)
: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = {
getMapSizesByExecutorId(shuffleId, 0, Int.MaxValue, reduceId, reduceId + 1)
}
/**
* Called from executors to get the server URIs and output sizes for each shuffle block that
* needs to be read from a given range of map output partitions (startPartition is included but
* endPartition is excluded from the range) within a range of mappers (startMapIndex is included
* but endMapIndex is excluded). If endMapIndex=Int.MaxValue, the actual endMapIndex will be
* changed to the length of total map outputs.
*
* @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId,
* and the second item is a sequence of (shuffle block id, shuffle block size, map index)
* tuples describing the shuffle blocks that are stored at that block manager.
* Note that zero-sized blocks are excluded in the result.
*/
def getMapSizesByExecutorId(
shuffleId: Int,
startMapIndex: Int,
endMapIndex: Int,
startPartition: Int,
endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])]
/**
* Deletes map output status information for the specified shuffle stage.
*/
def unregisterShuffle(shuffleId: Int): Unit
def stop(): Unit = {}
}
/**
* Driver-side class that keeps track of the location of the map output of a stage.
*
* The DAGScheduler uses this class to (de)register map output statuses and to look up statistics
* for performing locality-aware reduce task scheduling.
*
* ShuffleMapStage uses this class for tracking available / missing outputs in order to determine
* which tasks need to be run.
*/
private[spark] class MapOutputTrackerMaster(
conf: SparkConf,
private[spark] val broadcastManager: BroadcastManager,
private[spark] val isLocal: Boolean)
extends MapOutputTracker(conf) {
// The size at which we use Broadcast to send the map output statuses to the executors
private val minSizeForBroadcast = conf.get(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST).toInt
/** Whether to compute locality preferences for reduce tasks */
private val shuffleLocalityEnabled = conf.get(SHUFFLE_REDUCE_LOCALITY_ENABLE)
// Number of map and reduce tasks above which we do not assign preferred locations based on map
// output sizes. We limit the size of jobs for which assign preferred locations as computing the
// top locations by size becomes expensive.
private val SHUFFLE_PREF_MAP_THRESHOLD = 1000
// NOTE: This should be less than 2000 as we use HighlyCompressedMapStatus beyond that
private val SHUFFLE_PREF_REDUCE_THRESHOLD = 1000
// Fraction of total map output that must be at a location for it to considered as a preferred
// location for a reduce task. Making this larger will focus on fewer locations where most data
// can be read locally, but may lead to more delay in scheduling if those locations are busy.
private val REDUCER_PREF_LOCS_FRACTION = 0.2
// HashMap for storing shuffleStatuses in the driver.
// Statuses are dropped only by explicit de-registering.
// Exposed for testing
val shuffleStatuses = new ConcurrentHashMap[Int, ShuffleStatus]().asScala
private val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf)
// requests for map output statuses
private val mapOutputRequests = new LinkedBlockingQueue[GetMapOutputMessage]
// Thread pool used for handling map output status requests. This is a separate thread pool
// to ensure we don't block the normal dispatcher threads.
private val threadpool: ThreadPoolExecutor = {
val numThreads = conf.get(SHUFFLE_MAPOUTPUT_DISPATCHER_NUM_THREADS)
val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "map-output-dispatcher")
for (i <- 0 until numThreads) {
pool.execute(new MessageLoop)
}
pool
}
// Make sure that we aren't going to exceed the max RPC message size by making sure
// we use broadcast to send large map output statuses.
if (minSizeForBroadcast > maxRpcMessageSize) {
val msg = s"${SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST.key} ($minSizeForBroadcast bytes) " +
s"must be <= spark.rpc.message.maxSize ($maxRpcMessageSize bytes) to prevent sending an " +
"rpc message that is too large."
logError(msg)
throw new IllegalArgumentException(msg)
}
def post(message: GetMapOutputMessage): Unit = {
mapOutputRequests.offer(message)
}
/** Message loop used for dispatching messages. */
private class MessageLoop extends Runnable {
override def run(): Unit = {
try {
while (true) {
try {
val data = mapOutputRequests.take()
if (data == PoisonPill) {
// Put PoisonPill back so that other MessageLoops can see it.
mapOutputRequests.offer(PoisonPill)
return
}
val context = data.context
val shuffleId = data.shuffleId
val hostPort = context.senderAddress.hostPort
logDebug("Handling request to send map output locations for shuffle " + shuffleId +
" to " + hostPort)
val shuffleStatus = shuffleStatuses.get(shuffleId).head
context.reply(
shuffleStatus.serializedMapStatus(broadcastManager, isLocal, minSizeForBroadcast,
conf))
} catch {
case NonFatal(e) => logError(e.getMessage, e)
}
}
} catch {
case ie: InterruptedException => // exit
}
}
}
/** A poison endpoint that indicates MessageLoop should exit its message loop. */
private val PoisonPill = new GetMapOutputMessage(-99, null)
// Used only in unit tests.
private[spark] def getNumCachedSerializedBroadcast: Int = {
shuffleStatuses.valuesIterator.count(_.hasCachedSerializedBroadcast)
}
def registerShuffle(shuffleId: Int, numMaps: Int): Unit = {
if (shuffleStatuses.put(shuffleId, new ShuffleStatus(numMaps)).isDefined) {
throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
}
}
def updateMapOutput(shuffleId: Int, mapId: Long, bmAddress: BlockManagerId): Unit = {
shuffleStatuses.get(shuffleId) match {
case Some(shuffleStatus) =>
shuffleStatus.updateMapOutput(mapId, bmAddress)
case None =>
logError(s"Asked to update map output for unknown shuffle ${shuffleId}")
}
}
def registerMapOutput(shuffleId: Int, mapIndex: Int, status: MapStatus): Unit = {
shuffleStatuses(shuffleId).addMapOutput(mapIndex, status)
}
/** Unregister map output information of the given shuffle, mapper and block manager */
def unregisterMapOutput(shuffleId: Int, mapIndex: Int, bmAddress: BlockManagerId): Unit = {
shuffleStatuses.get(shuffleId) match {
case Some(shuffleStatus) =>
shuffleStatus.removeMapOutput(mapIndex, bmAddress)
incrementEpoch()
case None =>
throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID")
}
}
/** Unregister all map output information of the given shuffle. */
def unregisterAllMapOutput(shuffleId: Int): Unit = {
shuffleStatuses.get(shuffleId) match {
case Some(shuffleStatus) =>
shuffleStatus.removeOutputsByFilter(x => true)
incrementEpoch()
case None =>
throw new SparkException(
s"unregisterAllMapOutput called for nonexistent shuffle ID $shuffleId.")
}
}
/** Unregister shuffle data */
def unregisterShuffle(shuffleId: Int): Unit = {
shuffleStatuses.remove(shuffleId).foreach { shuffleStatus =>
shuffleStatus.invalidateSerializedMapOutputStatusCache()
}
}
/**
* Removes all shuffle outputs associated with this host. Note that this will also remove
* outputs which are served by an external shuffle server (if one exists).
*/
def removeOutputsOnHost(host: String): Unit = {
shuffleStatuses.valuesIterator.foreach { _.removeOutputsOnHost(host) }
incrementEpoch()
}
/**
* Removes all shuffle outputs associated with this executor. Note that this will also remove
* outputs which are served by an external shuffle server (if one exists), as they are still
* registered with this execId.
*/
def removeOutputsOnExecutor(execId: String): Unit = {
shuffleStatuses.valuesIterator.foreach { _.removeOutputsOnExecutor(execId) }
incrementEpoch()
}
/** Check if the given shuffle is being tracked */
def containsShuffle(shuffleId: Int): Boolean = shuffleStatuses.contains(shuffleId)
def getNumAvailableOutputs(shuffleId: Int): Int = {
shuffleStatuses.get(shuffleId).map(_.numAvailableOutputs).getOrElse(0)
}
/**
* Returns the sequence of partition ids that are missing (i.e. needs to be computed), or None
* if the MapOutputTrackerMaster doesn't know about this shuffle.
*/
def findMissingPartitions(shuffleId: Int): Option[Seq[Int]] = {
shuffleStatuses.get(shuffleId).map(_.findMissingPartitions())
}
/**
* Grouped function of Range, this is to avoid traverse of all elements of Range using
* IterableLike's grouped function.
*/
def rangeGrouped(range: Range, size: Int): Seq[Range] = {
val start = range.start
val step = range.step
val end = range.end
for (i <- start.until(end, size * step)) yield {
i.until(i + size * step, step)
}
}
/**
* To equally divide n elements into m buckets, basically each bucket should have n/m elements,
* for the remaining n%m elements, add one more element to the first n%m buckets each.
*/
def equallyDivide(numElements: Int, numBuckets: Int): Seq[Seq[Int]] = {
val elementsPerBucket = numElements / numBuckets
val remaining = numElements % numBuckets
val splitPoint = (elementsPerBucket + 1) * remaining
if (elementsPerBucket == 0) {
rangeGrouped(0.until(splitPoint), elementsPerBucket + 1)
} else {
rangeGrouped(0.until(splitPoint), elementsPerBucket + 1) ++
rangeGrouped(splitPoint.until(numElements), elementsPerBucket)
}
}
/**
* Return statistics about all of the outputs for a given shuffle.
*/
def getStatistics(dep: ShuffleDependency[_, _, _]): MapOutputStatistics = {
shuffleStatuses(dep.shuffleId).withMapStatuses { statuses =>
val totalSizes = new Array[Long](dep.partitioner.numPartitions)
val parallelAggThreshold = conf.get(
SHUFFLE_MAP_OUTPUT_PARALLEL_AGGREGATION_THRESHOLD)
val parallelism = math.min(
Runtime.getRuntime.availableProcessors(),
statuses.length.toLong * totalSizes.length / parallelAggThreshold + 1).toInt
if (parallelism <= 1) {
for (s <- statuses) {
for (i <- 0 until totalSizes.length) {
totalSizes(i) += s.getSizeForBlock(i)
}
}
} else {
val threadPool = ThreadUtils.newDaemonFixedThreadPool(parallelism, "map-output-aggregate")
try {
implicit val executionContext = ExecutionContext.fromExecutor(threadPool)
val mapStatusSubmitTasks = equallyDivide(totalSizes.length, parallelism).map {
reduceIds => Future {
for (s <- statuses; i <- reduceIds) {
totalSizes(i) += s.getSizeForBlock(i)
}
}
}
ThreadUtils.awaitResult(Future.sequence(mapStatusSubmitTasks), Duration.Inf)
} finally {
threadPool.shutdown()
}
}
new MapOutputStatistics(dep.shuffleId, totalSizes)
}
}
/**
* Return the preferred hosts on which to run the given map output partition in a given shuffle,
* i.e. the nodes that the most outputs for that partition are on.
*
* @param dep shuffle dependency object
* @param partitionId map output partition that we want to read
* @return a sequence of host names
*/
def getPreferredLocationsForShuffle(dep: ShuffleDependency[_, _, _], partitionId: Int)
: Seq[String] = {
if (shuffleLocalityEnabled && dep.rdd.partitions.length < SHUFFLE_PREF_MAP_THRESHOLD &&
dep.partitioner.numPartitions < SHUFFLE_PREF_REDUCE_THRESHOLD) {
val blockManagerIds = getLocationsWithLargestOutputs(dep.shuffleId, partitionId,
dep.partitioner.numPartitions, REDUCER_PREF_LOCS_FRACTION)
if (blockManagerIds.nonEmpty) {
blockManagerIds.get.map(_.host)
} else {
Nil
}
} else {
Nil
}
}
/**
* Return a list of locations that each have fraction of map output greater than the specified
* threshold.
*
* @param shuffleId id of the shuffle
* @param reducerId id of the reduce task
* @param numReducers total number of reducers in the shuffle
* @param fractionThreshold fraction of total map output size that a location must have
* for it to be considered large.
*/
def getLocationsWithLargestOutputs(
shuffleId: Int,
reducerId: Int,
numReducers: Int,
fractionThreshold: Double)
: Option[Array[BlockManagerId]] = {
val shuffleStatus = shuffleStatuses.get(shuffleId).orNull
if (shuffleStatus != null) {
shuffleStatus.withMapStatuses { statuses =>
if (statuses.nonEmpty) {
// HashMap to add up sizes of all blocks at the same location
val locs = new HashMap[BlockManagerId, Long]
var totalOutputSize = 0L
var mapIdx = 0
while (mapIdx < statuses.length) {
val status = statuses(mapIdx)
// status may be null here if we are called between registerShuffle, which creates an
// array with null entries for each output, and registerMapOutputs, which populates it
// with valid status entries. This is possible if one thread schedules a job which
// depends on an RDD which is currently being computed by another thread.
if (status != null) {
val blockSize = status.getSizeForBlock(reducerId)
if (blockSize > 0) {
locs(status.location) = locs.getOrElse(status.location, 0L) + blockSize
totalOutputSize += blockSize
}
}
mapIdx = mapIdx + 1
}
val topLocs = locs.filter { case (loc, size) =>
size.toDouble / totalOutputSize >= fractionThreshold
}
// Return if we have any locations which satisfy the required threshold
if (topLocs.nonEmpty) {
return Some(topLocs.keys.toArray)
}
}
}
}
None
}
/**
* Return the locations where the Mappers ran. The locations each includes both a host and an
* executor id on that host.
*
* @param dep shuffle dependency object
* @param startMapIndex the start map index
* @param endMapIndex the end map index (exclusive)
* @return a sequence of locations where task runs.
*/
def getMapLocation(
dep: ShuffleDependency[_, _, _],
startMapIndex: Int,
endMapIndex: Int): Seq[String] =
{
val shuffleStatus = shuffleStatuses.get(dep.shuffleId).orNull
if (shuffleStatus != null) {
shuffleStatus.withMapStatuses { statuses =>
if (startMapIndex < endMapIndex &&
(startMapIndex >= 0 && endMapIndex <= statuses.length)) {
val statusesPicked = statuses.slice(startMapIndex, endMapIndex).filter(_ != null)
statusesPicked.map(_.location.host).toSeq
} else {
Nil
}
}
} else {
Nil
}
}
def incrementEpoch(): Unit = {
epochLock.synchronized {
epoch += 1
logDebug("Increasing epoch to " + epoch)
}
}
/** Called to get current epoch number. */
def getEpoch: Long = {
epochLock.synchronized {
return epoch
}
}
// This method is only called in local-mode.
def getMapSizesByExecutorId(
shuffleId: Int,
startMapIndex: Int,
endMapIndex: Int,
startPartition: Int,
endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = {
logDebug(s"Fetching outputs for shuffle $shuffleId")
shuffleStatuses.get(shuffleId) match {
case Some(shuffleStatus) =>
shuffleStatus.withMapStatuses { statuses =>
val actualEndMapIndex = if (endMapIndex == Int.MaxValue) statuses.length else endMapIndex
logDebug(s"Convert map statuses for shuffle $shuffleId, " +
s"mappers $startMapIndex-$actualEndMapIndex, partitions $startPartition-$endPartition")
MapOutputTracker.convertMapStatuses(
shuffleId, startPartition, endPartition, statuses, startMapIndex, actualEndMapIndex)
}
case None =>
Iterator.empty
}
}
override def stop(): Unit = {
mapOutputRequests.offer(PoisonPill)
threadpool.shutdown()
try {
sendTracker(StopMapOutputTracker)
} catch {
case e: SparkException =>
logError("Could not tell tracker we are stopping.", e)
}
trackerEndpoint = null
shuffleStatuses.clear()
}
}
/**
* 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 common
* superclass).
*/
private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) {
val mapStatuses: Map[Int, Array[MapStatus]] =
new ConcurrentHashMap[Int, Array[MapStatus]]().asScala
/**
* A [[KeyLock]] whose key is a shuffle id to ensure there is only one thread fetching
* the same shuffle block.
*/
private val fetchingLock = new KeyLock[Int]
override def getMapSizesByExecutorId(
shuffleId: Int,
startMapIndex: Int,
endMapIndex: Int,
startPartition: Int,
endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = {
logDebug(s"Fetching outputs for shuffle $shuffleId")
val statuses = getStatuses(shuffleId, conf)
try {
val actualEndMapIndex = if (endMapIndex == Int.MaxValue) statuses.length else endMapIndex
logDebug(s"Convert map statuses for shuffle $shuffleId, " +
s"mappers $startMapIndex-$actualEndMapIndex, partitions $startPartition-$endPartition")
MapOutputTracker.convertMapStatuses(
shuffleId, startPartition, endPartition, statuses, startMapIndex, actualEndMapIndex)
} catch {
case e: MetadataFetchFailedException =>
// We experienced a fetch failure so our mapStatuses cache is outdated; clear it:
mapStatuses.clear()
throw e
}
}
/**
* Get or fetch the array of MapStatuses for a given shuffle ID. NOTE: clients MUST synchronize
* on this array when reading it, because on the driver, we may be changing it in place.
*
* (It would be nice to remove this restriction in the future.)
*/
private def getStatuses(shuffleId: Int, conf: SparkConf): Array[MapStatus] = {
val statuses = mapStatuses.get(shuffleId).orNull
if (statuses == null) {
logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them")
val startTimeNs = System.nanoTime()
fetchingLock.withLock(shuffleId) {
var fetchedStatuses = mapStatuses.get(shuffleId).orNull
if (fetchedStatuses == null) {
logInfo("Doing the fetch; tracker endpoint = " + trackerEndpoint)
val fetchedBytes = askTracker[Array[Byte]](GetMapOutputStatuses(shuffleId))
fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes, conf)
logInfo("Got the output locations")
mapStatuses.put(shuffleId, fetchedStatuses)
}
logDebug(s"Fetching map output statuses for shuffle $shuffleId took " +
s"${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms")
fetchedStatuses
}
} else {
statuses
}
}
/** Unregister shuffle data. */
def unregisterShuffle(shuffleId: Int): Unit = {
mapStatuses.remove(shuffleId)
}
/**
* Called from executors to update the epoch number, potentially clearing old outputs
* because of a fetch failure. Each executor task calls this with the latest epoch
* number on the driver at the time it was created.
*/
def updateEpoch(newEpoch: Long): Unit = {
epochLock.synchronized {
if (newEpoch > epoch) {
logInfo("Updating epoch to " + newEpoch + " and clearing cache")
epoch = newEpoch
mapStatuses.clear()
}
}
}
}
private[spark] object MapOutputTracker extends Logging {
val ENDPOINT_NAME = "MapOutputTracker"
private val DIRECT = 0
private val BROADCAST = 1
// Serialize an array of map output locations into an efficient byte format so that we can send
// it to reduce tasks. We do this by compressing the serialized bytes using Zstd. They will
// generally be pretty compressible because many map outputs will be on the same hostname.
def serializeMapStatuses(
statuses: Array[MapStatus],
broadcastManager: BroadcastManager,
isLocal: Boolean,
minBroadcastSize: Int,
conf: SparkConf): (Array[Byte], Broadcast[Array[Byte]]) = {
// Using `org.apache.commons.io.output.ByteArrayOutputStream` instead of the standard one
// This implementation doesn't reallocate the whole memory block but allocates
// additional buffers. This way no buffers need to be garbage collected and
// the contents don't have to be copied to the new buffer.
val out = new ApacheByteArrayOutputStream()
out.write(DIRECT)
val codec = CompressionCodec.createCodec(conf, conf.get(MAP_STATUS_COMPRESSION_CODEC))
val objOut = new ObjectOutputStream(codec.compressedOutputStream(out))
Utils.tryWithSafeFinally {
// Since statuses can be modified in parallel, sync on it
statuses.synchronized {
objOut.writeObject(statuses)
}
} {
objOut.close()
}
val arr = out.toByteArray
if (arr.length >= minBroadcastSize) {
// Use broadcast instead.
// Important arr(0) is the tag == DIRECT, ignore that while deserializing !
val bcast = broadcastManager.newBroadcast(arr, isLocal)
// toByteArray creates copy, so we can reuse out
out.reset()
out.write(BROADCAST)
val oos = new ObjectOutputStream(codec.compressedOutputStream(out))
Utils.tryWithSafeFinally {
oos.writeObject(bcast)
} {
oos.close()
}
val outArr = out.toByteArray
logInfo("Broadcast mapstatuses size = " + outArr.length + ", actual size = " + arr.length)
(outArr, bcast)
} else {
(arr, null)
}
}
// Opposite of serializeMapStatuses.
def deserializeMapStatuses(bytes: Array[Byte], conf: SparkConf): Array[MapStatus] = {
assert (bytes.length > 0)
def deserializeObject(arr: Array[Byte], off: Int, len: Int): AnyRef = {
val codec = CompressionCodec.createCodec(conf, conf.get(MAP_STATUS_COMPRESSION_CODEC))
// The ZStd codec is wrapped in a `BufferedInputStream` which avoids overhead excessive
// of JNI call while trying to decompress small amount of data for each element
// of `MapStatuses`
val objIn = new ObjectInputStream(codec.compressedInputStream(
new ByteArrayInputStream(arr, off, len)))
Utils.tryWithSafeFinally {
objIn.readObject()
} {
objIn.close()
}
}
bytes(0) match {
case DIRECT =>
deserializeObject(bytes, 1, bytes.length - 1).asInstanceOf[Array[MapStatus]]
case BROADCAST =>
// deserialize the Broadcast, pull .value array out of it, and then deserialize that
val bcast = deserializeObject(bytes, 1, bytes.length - 1).
asInstanceOf[Broadcast[Array[Byte]]]
logInfo("Broadcast mapstatuses size = " + bytes.length +
", actual size = " + bcast.value.length)
// Important - ignore the DIRECT tag ! Start from offset 1
deserializeObject(bcast.value, 1, bcast.value.length - 1).asInstanceOf[Array[MapStatus]]
case _ => throw new IllegalArgumentException("Unexpected byte tag = " + bytes(0))
}
}
/**
* Given an array of map statuses and a range of map output partitions, returns a sequence that,
* for each block manager ID, lists the shuffle block IDs and corresponding shuffle block sizes
* stored at that block manager.
* Note that empty blocks are filtered in the result.
*
* If any of the statuses is null (indicating a missing location due to a failed mapper),
* throws a FetchFailedException.
*
* @param shuffleId Identifier for the shuffle
* @param startPartition Start of map output partition ID range (included in range)
* @param endPartition End of map output partition ID range (excluded from range)
* @param statuses List of map statuses, indexed by map partition index.
* @param startMapIndex Start Map index.
* @param endMapIndex End Map index.
* @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId,
* and the second item is a sequence of (shuffle block id, shuffle block size, map index)
* tuples describing the shuffle blocks that are stored at that block manager.
*/
def convertMapStatuses(
shuffleId: Int,
startPartition: Int,
endPartition: Int,
statuses: Array[MapStatus],
startMapIndex : Int,
endMapIndex: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = {
assert (statuses != null)
val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]]
val iter = statuses.iterator.zipWithIndex
for ((status, mapIndex) <- iter.slice(startMapIndex, endMapIndex)) {
if (status == null) {
val errorMessage = s"Missing an output location for shuffle $shuffleId"
logError(errorMessage)
throw new MetadataFetchFailedException(shuffleId, startPartition, errorMessage)