/
BlockManager.scala
2303 lines (2086 loc) · 89.5 KB
/
BlockManager.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.storage
import java.io._
import java.lang.ref.{ReferenceQueue => JReferenceQueue, WeakReference}
import java.nio.ByteBuffer
import java.nio.channels.Channels
import java.util.Collections
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit}
import scala.collection.mutable
import scala.collection.mutable.HashMap
import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration._
import scala.jdk.CollectionConverters._
import scala.reflect.ClassTag
import scala.util.{Failure, Random, Success, Try}
import scala.util.control.NonFatal
import com.codahale.metrics.{MetricRegistry, MetricSet}
import com.esotericsoftware.kryo.KryoException
import com.google.common.cache.CacheBuilder
import org.apache.commons.io.IOUtils
import org.apache.spark._
import org.apache.spark.errors.SparkCoreErrors
import org.apache.spark.executor.DataReadMethod
import org.apache.spark.internal.{config, Logging, MDC}
import org.apache.spark.internal.LogKey.BLOCK_ID
import org.apache.spark.internal.config.{Network, RDD_CACHE_VISIBILITY_TRACKING_ENABLED, Tests}
import org.apache.spark.memory.{MemoryManager, MemoryMode}
import org.apache.spark.metrics.source.Source
import org.apache.spark.network._
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
import org.apache.spark.network.client.StreamCallbackWithID
import org.apache.spark.network.netty.SparkTransportConf
import org.apache.spark.network.shuffle._
import org.apache.spark.network.shuffle.checksum.{Cause, ShuffleChecksumHelper}
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
import org.apache.spark.network.util.TransportConf
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.scheduler.ExecutorCacheTaskLocation
import org.apache.spark.serializer.{SerializerInstance, SerializerManager}
import org.apache.spark.shuffle.{IndexShuffleBlockResolver, MigratableResolver, ShuffleManager, ShuffleWriteMetricsReporter}
import org.apache.spark.storage.BlockManagerMessages.{DecommissionBlockManager, ReplicateBlock}
import org.apache.spark.storage.memory._
import org.apache.spark.unsafe.Platform
import org.apache.spark.util._
import org.apache.spark.util.ArrayImplicits._
import org.apache.spark.util.io.ChunkedByteBuffer
/* Class for returning a fetched block and associated metrics. */
private[spark] class BlockResult(
val data: Iterator[Any],
val readMethod: DataReadMethod.Value,
val bytes: Long)
/**
* Abstracts away how blocks are stored and provides different ways to read the underlying block
* data. Callers should call [[BlockData#dispose()]] when they're done with the block.
*/
private[spark] trait BlockData {
def toInputStream(): InputStream
/**
* Returns a Netty-friendly wrapper for the block's data.
*
* Please see `ManagedBuffer.convertToNetty()` for more details.
*/
def toNetty(): Object
/**
* Returns a Netty-friendly wrapper for the block's data.
*
* Please see `ManagedBuffer.convertToNettyForSsl()` for more details.
*/
def toNettyForSsl(): Object
def toChunkedByteBuffer(allocator: Int => ByteBuffer): ChunkedByteBuffer
def toByteBuffer(): ByteBuffer
def size: Long
def dispose(): Unit
}
private[spark] class ByteBufferBlockData(
val buffer: ChunkedByteBuffer,
val shouldDispose: Boolean) extends BlockData {
override def toInputStream(): InputStream = buffer.toInputStream()
override def toNetty(): Object = buffer.toNetty
override def toNettyForSsl(): AnyRef = buffer.toNettyForSsl
override def toChunkedByteBuffer(allocator: Int => ByteBuffer): ChunkedByteBuffer = {
buffer.copy(allocator)
}
override def toByteBuffer(): ByteBuffer = buffer.toByteBuffer
override def size: Long = buffer.size
override def dispose(): Unit = {
if (shouldDispose) {
buffer.dispose()
}
}
}
private[spark] class HostLocalDirManager(
cacheSize: Int,
blockStoreClient: BlockStoreClient) extends Logging {
private val executorIdToLocalDirsCache =
CacheBuilder
.newBuilder()
.maximumSize(cacheSize)
.build[String, Array[String]]()
private[spark] def getCachedHostLocalDirs: Map[String, Array[String]] =
executorIdToLocalDirsCache.synchronized {
executorIdToLocalDirsCache.asMap().asScala.toMap
}
private[spark] def getCachedHostLocalDirsFor(executorId: String): Option[Array[String]] =
executorIdToLocalDirsCache.synchronized {
Option(executorIdToLocalDirsCache.getIfPresent(executorId))
}
private[spark] def getHostLocalDirs(
host: String,
port: Int,
executorIds: Array[String])(
callback: Try[Map[String, Array[String]]] => Unit): Unit = {
val hostLocalDirsCompletable = new CompletableFuture[java.util.Map[String, Array[String]]]
blockStoreClient.getHostLocalDirs(
host,
port,
executorIds,
hostLocalDirsCompletable)
hostLocalDirsCompletable.whenComplete { (hostLocalDirs, throwable) =>
if (hostLocalDirs != null) {
callback(Success(hostLocalDirs.asScala.toMap))
executorIdToLocalDirsCache.synchronized {
executorIdToLocalDirsCache.putAll(hostLocalDirs)
}
} else {
callback(Failure(throwable))
}
}
}
}
/**
* Manager running on every node (driver and executors) which provides interfaces for putting and
* retrieving blocks both locally and remotely into various stores (memory, disk, and off-heap).
*
* Note that [[initialize()]] must be called before the BlockManager is usable.
*/
private[spark] class BlockManager(
val executorId: String,
rpcEnv: RpcEnv,
val master: BlockManagerMaster,
val serializerManager: SerializerManager,
val conf: SparkConf,
private val _memoryManager: MemoryManager,
mapOutputTracker: MapOutputTracker,
private val _shuffleManager: ShuffleManager,
val blockTransferService: BlockTransferService,
securityManager: SecurityManager,
externalBlockStoreClient: Option[ExternalBlockStoreClient])
extends BlockDataManager with BlockEvictionHandler with Logging {
// We initialize the ShuffleManager later in SparkContext and Executor, to allow
// user jars to define custom ShuffleManagers, as such `_shuffleManager` will be null here
// (except for tests) and we ask for the instance from the SparkEnv.
private lazy val shuffleManager = Option(_shuffleManager).getOrElse(SparkEnv.get.shuffleManager)
// Similarly, we also initialize MemoryManager later after DriverPlugin is loaded, to
// allow the plugin to overwrite certain memory configurations. The `_memoryManager` will be
// null here and we ask for the instance from SparkEnv
private[spark] lazy val memoryManager =
Option(_memoryManager).getOrElse(SparkEnv.get.memoryManager)
// same as `conf.get(config.SHUFFLE_SERVICE_ENABLED)`
private[spark] val externalShuffleServiceEnabled: Boolean = externalBlockStoreClient.isDefined
private val isDriver = executorId == SparkContext.DRIVER_IDENTIFIER
private val remoteReadNioBufferConversion =
conf.get(Network.NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION)
private[spark] val subDirsPerLocalDir = conf.get(config.DISKSTORE_SUB_DIRECTORIES)
val diskBlockManager = {
// Only perform cleanup if an external service is not serving our shuffle files.
val deleteFilesOnStop =
!externalShuffleServiceEnabled || isDriver
new DiskBlockManager(conf, deleteFilesOnStop = deleteFilesOnStop, isDriver = isDriver)
}
/** Whether rdd cache visibility tracking is enabled. */
private val trackingCacheVisibility: Boolean = conf.get(RDD_CACHE_VISIBILITY_TRACKING_ENABLED)
// Visible for testing
private[storage] val blockInfoManager = new BlockInfoManager(trackingCacheVisibility)
private val futureExecutionContext = ExecutionContext.fromExecutorService(
ThreadUtils.newDaemonCachedThreadPool("block-manager-future", 128))
// Actual storage of where blocks are kept
private[spark] lazy val memoryStore = {
val store = new MemoryStore(conf, blockInfoManager, serializerManager, memoryManager, this)
memoryManager.setMemoryStore(store)
store
}
private[spark] val diskStore = new DiskStore(conf, diskBlockManager, securityManager)
// Note: depending on the memory manager, `maxMemory` may actually vary over time.
// However, since we use this only for reporting and logging, what we actually want here is
// the absolute maximum value that `maxMemory` can ever possibly reach. We may need
// to revisit whether reporting this value as the "max" is intuitive to the user.
private lazy val maxOnHeapMemory = memoryManager.maxOnHeapStorageMemory
private lazy val maxOffHeapMemory = memoryManager.maxOffHeapStorageMemory
private[spark] val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf)
var blockManagerId: BlockManagerId = _
// Address of the server that serves this executor's shuffle files. This is either an external
// service, or just our own Executor's BlockManager.
private[spark] var shuffleServerId: BlockManagerId = _
// Client to read other executors' blocks. This is either an external service, or just the
// standard BlockTransferService to directly connect to other Executors.
private[spark] val blockStoreClient = externalBlockStoreClient.getOrElse(blockTransferService)
// Max number of failures before this block manager refreshes the block locations from the driver
private val maxFailuresBeforeLocationRefresh =
conf.get(config.BLOCK_FAILURES_BEFORE_LOCATION_REFRESH)
private val storageEndpoint = rpcEnv.setupEndpoint(
"BlockManagerEndpoint" + BlockManager.ID_GENERATOR.next,
new BlockManagerStorageEndpoint(rpcEnv, this, mapOutputTracker))
// Pending re-registration action being executed asynchronously or null if none is pending.
// Accesses should synchronize on asyncReregisterLock.
private var asyncReregisterTask: Future[Unit] = null
private val asyncReregisterLock = new Object
// Field related to peer block managers that are necessary for block replication
@volatile private var cachedPeers: Seq[BlockManagerId] = _
private val peerFetchLock = new Object
private var lastPeerFetchTimeNs = 0L
private var blockReplicationPolicy: BlockReplicationPolicy = _
// visible for test
// This is volatile since if it's defined we should not accept remote blocks.
@volatile private[spark] var decommissioner: Option[BlockManagerDecommissioner] = None
// A DownloadFileManager used to track all the files of remote blocks which are above the
// specified memory threshold. Files will be deleted automatically based on weak reference.
// Exposed for test
private[storage] val remoteBlockTempFileManager =
new BlockManager.RemoteBlockDownloadFileManager(
this,
securityManager.getIOEncryptionKey())
private val maxRemoteBlockToMem = conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM)
var hostLocalDirManager: Option[HostLocalDirManager] = None
@inline final private def isDecommissioning() = {
decommissioner.isDefined
}
@inline final private def checkShouldStore(blockId: BlockId) = {
// Don't reject broadcast blocks since they may be stored during task exec and
// don't need to be migrated.
if (isDecommissioning() && !blockId.isBroadcast) {
throw SparkCoreErrors.cannotSaveBlockOnDecommissionedExecutorError(blockId)
}
}
// This is a lazy val so someone can migrating RDDs even if they don't have a MigratableResolver
// for shuffles. Used in BlockManagerDecommissioner & block puts.
private[storage] lazy val migratableResolver: MigratableResolver = {
shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver]
}
override def getLocalDiskDirs: Array[String] = diskBlockManager.localDirsString
/**
* Diagnose the possible cause of the shuffle data corruption by verifying the shuffle checksums
*
* @param blockId The blockId of the corrupted shuffle block
* @param checksumByReader The checksum value of the corrupted block
* @param algorithm The checksum algorithm that is used when calculating the checksum value
*/
override def diagnoseShuffleBlockCorruption(
blockId: BlockId,
checksumByReader: Long,
algorithm: String): Cause = {
assert(blockId.isInstanceOf[ShuffleBlockId],
s"Corruption diagnosis only supports shuffle block yet, but got $blockId")
val shuffleBlock = blockId.asInstanceOf[ShuffleBlockId]
val resolver = shuffleManager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver]
val checksumFile =
resolver.getChecksumFile(shuffleBlock.shuffleId, shuffleBlock.mapId, algorithm)
val reduceId = shuffleBlock.reduceId
ShuffleChecksumHelper.diagnoseCorruption(
algorithm, checksumFile, reduceId, resolver.getBlockData(shuffleBlock), checksumByReader)
}
/**
* Abstraction for storing blocks from bytes, whether they start in memory or on disk.
*
* @param blockSize the decrypted size of the block
*/
private[spark] abstract class BlockStoreUpdater[T](
blockSize: Long,
blockId: BlockId,
level: StorageLevel,
classTag: ClassTag[T],
tellMaster: Boolean,
keepReadLock: Boolean) {
/**
* Reads the block content into the memory. If the update of the block store is based on a
* temporary file this could lead to loading the whole file into a ChunkedByteBuffer.
*/
protected def readToByteBuffer(): ChunkedByteBuffer
protected def blockData(): BlockData
protected def saveToDiskStore(): Unit
private def saveDeserializedValuesToMemoryStore(inputStream: InputStream): Boolean = {
try {
val values = serializerManager.dataDeserializeStream(blockId, inputStream)(classTag)
memoryStore.putIteratorAsValues(blockId, values, level.memoryMode, classTag) match {
case Right(_) => true
case Left(iter) =>
// If putting deserialized values in memory failed, we will put the bytes directly
// to disk, so we don't need this iterator and can close it to free resources
// earlier.
iter.close()
false
}
} catch {
case ex: KryoException if ex.getCause.isInstanceOf[IOException] =>
logInfo(extendMessageWithBlockDetails(ex.getMessage, blockId))
throw ex
} finally {
IOUtils.closeQuietly(inputStream)
}
}
private def saveSerializedValuesToMemoryStore(bytes: ChunkedByteBuffer): Boolean = {
val memoryMode = level.memoryMode
memoryStore.putBytes(blockId, blockSize, memoryMode, () => {
if (memoryMode == MemoryMode.OFF_HEAP && bytes.chunks.exists(!_.isDirect)) {
bytes.copy(Platform.allocateDirectBuffer)
} else {
bytes
}
})
}
/**
* Put the given data according to the given level in one of the block stores, replicating
* the values if necessary.
*
* If the block already exists, this method will not overwrite it.
*
* If keepReadLock is true, this method will hold the read lock when it returns (even if the
* block already exists). If false, this method will hold no locks when it returns.
*
* @return true if the block was already present or if the put succeeded, false otherwise.
*/
def save(): Boolean = {
doPut(blockId, level, classTag, tellMaster, keepReadLock) { info =>
val startTimeNs = System.nanoTime()
// Since we're storing bytes, initiate the replication before storing them locally.
// This is faster as data is already serialized and ready to send.
val replicationFuture = if (level.replication > 1) {
Future {
// This is a blocking action and should run in futureExecutionContext which is a cached
// thread pool.
replicate(blockId, blockData(), level, classTag)
}(futureExecutionContext)
} else {
null
}
if (level.useMemory) {
// Put it in memory first, even if it also has useDisk set to true;
// We will drop it to disk later if the memory store can't hold it.
val putSucceeded = if (level.deserialized) {
saveDeserializedValuesToMemoryStore(blockData().toInputStream())
} else {
saveSerializedValuesToMemoryStore(readToByteBuffer())
}
if (!putSucceeded && level.useDisk) {
logWarning(s"Persisting block $blockId to disk instead.")
saveToDiskStore()
}
} else if (level.useDisk) {
saveToDiskStore()
}
val putBlockStatus = getCurrentBlockStatus(blockId, info)
val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid
if (blockWasSuccessfullyStored) {
// Now that the block is in either the memory or disk store,
// tell the master about it.
info.size = blockSize
if (tellMaster && info.tellMaster) {
reportBlockStatus(blockId, putBlockStatus)
}
addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus)
}
logDebug(s"Put block ${blockId} locally took ${Utils.getUsedTimeNs(startTimeNs)}")
if (level.replication > 1) {
// Wait for asynchronous replication to finish
try {
ThreadUtils.awaitReady(replicationFuture, Duration.Inf)
} catch {
case NonFatal(t) => throw SparkCoreErrors.waitingForReplicationToFinishError(t)
}
}
if (blockWasSuccessfullyStored) {
None
} else {
Some(blockSize)
}
}.isEmpty
}
}
/**
* Helper for storing a block from bytes already in memory.
* '''Important!''' Callers must not mutate or release the data buffer underlying `bytes`. Doing
* so may corrupt or change the data stored by the `BlockManager`.
*/
private case class ByteBufferBlockStoreUpdater[T](
blockId: BlockId,
level: StorageLevel,
classTag: ClassTag[T],
bytes: ChunkedByteBuffer,
tellMaster: Boolean = true,
keepReadLock: Boolean = false)
extends BlockStoreUpdater[T](bytes.size, blockId, level, classTag, tellMaster, keepReadLock) {
override def readToByteBuffer(): ChunkedByteBuffer = bytes
/**
* The ByteBufferBlockData wrapper is not disposed of to avoid releasing buffers that are
* owned by the caller.
*/
override def blockData(): BlockData = new ByteBufferBlockData(bytes, false)
override def saveToDiskStore(): Unit = diskStore.putBytes(blockId, bytes)
}
/**
* Helper for storing a block based from bytes already in a local temp file.
*/
private[spark] case class TempFileBasedBlockStoreUpdater[T](
blockId: BlockId,
level: StorageLevel,
classTag: ClassTag[T],
tmpFile: File,
blockSize: Long,
tellMaster: Boolean = true,
keepReadLock: Boolean = false)
extends BlockStoreUpdater[T](blockSize, blockId, level, classTag, tellMaster, keepReadLock) {
override def readToByteBuffer(): ChunkedByteBuffer = {
val allocator = level.memoryMode match {
case MemoryMode.ON_HEAP => ByteBuffer.allocate _
case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _
}
blockData().toChunkedByteBuffer(allocator)
}
override def blockData(): BlockData = diskStore.getBytes(tmpFile, blockSize)
override def saveToDiskStore(): Unit = diskStore.moveFileToBlock(tmpFile, blockSize, blockId)
override def save(): Boolean = {
val res = super.save()
tmpFile.delete()
res
}
}
/**
* Initializes the BlockManager with the given appId. This is not performed in the constructor as
* the appId may not be known at BlockManager instantiation time (in particular for the driver,
* where it is only learned after registration with the TaskScheduler).
*
* This method initializes the BlockTransferService and BlockStoreClient, registers with the
* BlockManagerMaster, starts the BlockManagerWorker endpoint, and registers with a local shuffle
* service if configured.
*/
def initialize(appId: String): Unit = {
blockTransferService.init(this)
externalBlockStoreClient.foreach { blockStoreClient =>
blockStoreClient.init(appId)
}
blockReplicationPolicy = {
val priorityClass = conf.get(config.STORAGE_REPLICATION_POLICY)
val clazz = Utils.classForName(priorityClass)
val ret = clazz.getConstructor().newInstance().asInstanceOf[BlockReplicationPolicy]
logInfo(s"Using $priorityClass for block replication policy")
ret
}
// Register Executors' configuration with the local shuffle service, if one should exist.
// Registration with the ESS should happen before registering the block manager with the
// BlockManagerMaster. In push-based shuffle, the registered BM is selected by the driver
// as a merger. However, for the ESS on this host to be able to merge blocks successfully,
// it needs the merge directories metadata which is provided by the local executor during
// the registration with the ESS. Therefore, this registration should be prior to
// the BlockManager registration. See SPARK-39647.
if (externalShuffleServiceEnabled) {
logInfo(s"external shuffle service port = $externalShuffleServicePort")
shuffleServerId = BlockManagerId(executorId, blockTransferService.hostName,
externalShuffleServicePort)
if (!isDriver && !(Utils.isTesting && conf.get(Tests.TEST_SKIP_ESS_REGISTER))) {
registerWithExternalShuffleServer()
}
}
val id =
BlockManagerId(executorId, blockTransferService.hostName, blockTransferService.port, None)
// The idFromMaster has just additional topology information. Otherwise, it has the same
// executor id/host/port of idWithoutTopologyInfo which is not expected to be changed.
val idFromMaster = master.registerBlockManager(
id,
diskBlockManager.localDirsString,
maxOnHeapMemory,
maxOffHeapMemory,
storageEndpoint)
blockManagerId = if (idFromMaster != null) idFromMaster else id
if (!externalShuffleServiceEnabled) {
shuffleServerId = blockManagerId
}
hostLocalDirManager = {
if ((conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
!conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) ||
Utils.isPushBasedShuffleEnabled(conf, isDriver)) {
Some(new HostLocalDirManager(
conf.get(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE),
blockStoreClient))
} else {
None
}
}
logInfo(s"Initialized BlockManager: $blockManagerId")
}
def shuffleMetricsSource: Source = {
import BlockManager._
if (externalShuffleServiceEnabled) {
new ShuffleMetricsSource("ExternalShuffle", blockStoreClient.shuffleMetrics())
} else {
new ShuffleMetricsSource("NettyBlockTransfer", blockStoreClient.shuffleMetrics())
}
}
private def registerWithExternalShuffleServer(): Unit = {
logInfo("Registering executor with local external shuffle service.")
// we obtain the class name from the configuration, instead of the ShuffleManager
// instance because the ShuffleManager has not been created at this point.
val shuffleMgrClass = ShuffleManager.getShuffleManagerClassName(conf)
val shuffleManagerMeta =
if (Utils.isPushBasedShuffleEnabled(conf, isDriver = isDriver, checkSerializer = false)) {
s"${shuffleMgrClass}:" +
s"${diskBlockManager.getMergeDirectoryAndAttemptIDJsonString()}}}"
} else {
shuffleMgrClass
}
val shuffleConfig = new ExecutorShuffleInfo(
diskBlockManager.localDirsString,
diskBlockManager.subDirsPerLocalDir,
shuffleManagerMeta)
val MAX_ATTEMPTS = conf.get(config.SHUFFLE_REGISTRATION_MAX_ATTEMPTS)
val SLEEP_TIME_SECS = 5
for (i <- 1 to MAX_ATTEMPTS) {
try {
// Synchronous and will throw an exception if we cannot connect.
blockStoreClient.asInstanceOf[ExternalBlockStoreClient].registerWithShuffleServer(
shuffleServerId.host, shuffleServerId.port, shuffleServerId.executorId, shuffleConfig)
return
} catch {
case e: Exception if i < MAX_ATTEMPTS =>
logError(s"Failed to connect to external shuffle server, will retry ${MAX_ATTEMPTS - i}"
+ s" more times after waiting $SLEEP_TIME_SECS seconds...", e)
Thread.sleep(SLEEP_TIME_SECS * 1000L)
case NonFatal(e) => throw SparkCoreErrors.unableToRegisterWithExternalShuffleServerError(e)
}
}
}
/**
* Report all blocks to the BlockManager again. This may be necessary if we are dropped
* by the BlockManager and come back or if we become capable of recovering blocks on disk after
* an executor crash.
*
* This function deliberately fails silently if the master returns false (indicating that
* the storage endpoint needs to re-register). The error condition will be detected again by the
* next heart beat attempt or new block registration and another try to re-register all blocks
* will be made then.
*/
private def reportAllBlocks(): Unit = {
logInfo(s"Reporting ${blockInfoManager.size} blocks to the master.")
for ((blockId, info) <- blockInfoManager.entries) {
val status = getCurrentBlockStatus(blockId, info)
if (info.tellMaster && !tryToReportBlockStatus(blockId, status)) {
logError(log"Failed to report ${MDC(BLOCK_ID, blockId)} to master; giving up.")
return
}
}
}
/**
* Re-register with the master and report all blocks to it. This will be called by the heart beat
* thread if our heartbeat to the block manager indicates that we were not registered.
*
* Note that this method must be called without any BlockInfo locks held.
*/
def reregister(): Unit = {
// TODO: We might need to rate limit re-registering.
logInfo(s"BlockManager $blockManagerId re-registering with master")
val id = master.registerBlockManager(blockManagerId, diskBlockManager.localDirsString,
maxOnHeapMemory, maxOffHeapMemory, storageEndpoint, isReRegister = true)
if (id.executorId != BlockManagerId.INVALID_EXECUTOR_ID) {
reportAllBlocks()
} else {
logError("Exiting executor due to block manager re-registration failure")
System.exit(-1)
}
}
/**
* Re-register with the master sometime soon.
*/
private def asyncReregister(): Unit = {
asyncReregisterLock.synchronized {
if (asyncReregisterTask == null) {
asyncReregisterTask = Future[Unit] {
// This is a blocking action and should run in futureExecutionContext which is a cached
// thread pool
reregister()
asyncReregisterLock.synchronized {
asyncReregisterTask = null
}
}(futureExecutionContext)
}
}
}
/**
* For testing. Wait for any pending asynchronous re-registration; otherwise, do nothing.
*/
def waitForAsyncReregister(): Unit = {
val task = asyncReregisterTask
if (task != null) {
try {
ThreadUtils.awaitReady(task, Duration.Inf)
} catch {
case NonFatal(t) =>
throw SparkCoreErrors.waitingForAsyncReregistrationError(t)
}
}
}
override def getHostLocalShuffleData(
blockId: BlockId,
dirs: Array[String]): ManagedBuffer = {
shuffleManager.shuffleBlockResolver.getBlockData(blockId, Some(dirs))
}
/**
* Interface to get local block data. Throws an exception if the block cannot be found or
* cannot be read successfully.
*/
override def getLocalBlockData(blockId: BlockId): ManagedBuffer = {
if (blockId.isShuffle) {
logDebug(s"Getting local shuffle block ${blockId}")
try {
shuffleManager.shuffleBlockResolver.getBlockData(blockId)
} catch {
case e: IOException =>
if (conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
FallbackStorage.read(conf, blockId)
} else {
throw e
}
}
} else {
getLocalBytes(blockId) match {
case Some(blockData) =>
new BlockManagerManagedBuffer(blockInfoManager, blockId, blockData, true)
case None =>
// If this block manager receives a request for a block that it doesn't have then it's
// likely that the master has outdated block statuses for this block. Therefore, we send
// an RPC so that this block is marked as being unavailable from this block manager.
reportBlockStatus(blockId, BlockStatus.empty)
throw SparkCoreErrors.blockNotFoundError(blockId)
}
}
}
/**
* Put the block locally, using the given storage level.
*
* '''Important!''' Callers must not mutate or release the data buffer underlying `bytes`. Doing
* so may corrupt or change the data stored by the `BlockManager`.
*/
override def putBlockData(
blockId: BlockId,
data: ManagedBuffer,
level: StorageLevel,
classTag: ClassTag[_]): Boolean = {
putBytes(blockId, new ChunkedByteBuffer(data.nioByteBuffer()), level)(classTag)
}
override def putBlockDataAsStream(
blockId: BlockId,
level: StorageLevel,
classTag: ClassTag[_]): StreamCallbackWithID = {
checkShouldStore(blockId)
if (blockId.isShuffle) {
logDebug(s"Putting shuffle block ${blockId}")
try {
return migratableResolver.putShuffleBlockAsStream(blockId, serializerManager)
} catch {
case _: ClassCastException =>
throw SparkCoreErrors.unexpectedShuffleBlockWithUnsupportedResolverError(shuffleManager,
blockId)
}
}
logDebug(s"Putting regular block ${blockId}")
// All other blocks
val (_, tmpFile) = diskBlockManager.createTempLocalBlock()
val channel = new CountingWritableChannel(
Channels.newChannel(serializerManager.wrapForEncryption(new FileOutputStream(tmpFile))))
logTrace(s"Streaming block $blockId to tmp file $tmpFile")
new StreamCallbackWithID {
override def getID: String = blockId.name
override def onData(streamId: String, buf: ByteBuffer): Unit = {
while (buf.hasRemaining) {
channel.write(buf)
}
}
override def onComplete(streamId: String): Unit = {
logTrace(s"Done receiving block $blockId, now putting into local blockManager")
// Note this is all happening inside the netty thread as soon as it reads the end of the
// stream.
channel.close()
val blockSize = channel.getCount
val blockStored = TempFileBasedBlockStoreUpdater(
blockId, level, classTag, tmpFile, blockSize).save()
if (!blockStored) {
throw SparkCoreErrors.failToStoreBlockOnBlockManagerError(blockManagerId, blockId)
}
}
override def onFailure(streamId: String, cause: Throwable): Unit = {
// the framework handles the connection itself, we just need to do local cleanup
channel.close()
tmpFile.delete()
}
}
}
/**
* Get the local merged shuffle block data for the given block ID as multiple chunks.
* A merged shuffle file is divided into multiple chunks according to the index file.
* Instead of reading the entire file as a single block, we split it into smaller chunks
* which will be memory efficient when performing certain operations.
*/
def getLocalMergedBlockData(
blockId: ShuffleMergedBlockId,
dirs: Array[String]): Seq[ManagedBuffer] = {
shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId, Some(dirs))
}
/**
* Get the local merged shuffle block meta data for the given block ID.
*/
def getLocalMergedBlockMeta(
blockId: ShuffleMergedBlockId,
dirs: Array[String]): MergedBlockMeta = {
shuffleManager.shuffleBlockResolver.getMergedBlockMeta(blockId, Some(dirs))
}
/**
* Get the BlockStatus for the block identified by the given ID, if it exists.
* NOTE: This is mainly for testing.
*/
def getStatus(blockId: BlockId): Option[BlockStatus] = {
blockInfoManager.get(blockId).map { info =>
val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L
val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L
BlockStatus(info.level, memSize = memSize, diskSize = diskSize)
}
}
/**
* Get the ids of existing blocks that match the given filter. Note that this will
* query the blocks stored in the disk block manager (that the block manager
* may not know of).
*/
def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = {
// The `toArray` is necessary here in order to force the list to be materialized so that we
// don't try to serialize a lazy iterator when responding to client requests.
(blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks())
.filter(filter)
.toArray
.toImmutableArraySeq
}
/**
* Tell the master about the current storage status of a block. This will send a block update
* message reflecting the current status, *not* the desired storage level in its block info.
* For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk.
*
* droppedMemorySize exists to account for when the block is dropped from memory to disk (so
* it is still valid). This ensures that update in master will compensate for the increase in
* memory on the storage endpoint.
*/
private[spark] def reportBlockStatus(
blockId: BlockId,
status: BlockStatus,
droppedMemorySize: Long = 0L): Unit = {
val needReregister = !tryToReportBlockStatus(blockId, status, droppedMemorySize)
if (needReregister) {
logInfo(s"Got told to re-register updating block $blockId")
// Re-registering will report our new block for free.
asyncReregister()
}
logDebug(s"Told master about block $blockId")
}
/**
* Actually send a UpdateBlockInfo message. Returns the master's response,
* which will be true if the block was successfully recorded and false if
* the storage endpoint needs to re-register.
*/
private def tryToReportBlockStatus(
blockId: BlockId,
status: BlockStatus,
droppedMemorySize: Long = 0L): Boolean = {
val storageLevel = status.storageLevel
val inMemSize = Math.max(status.memSize, droppedMemorySize)
val onDiskSize = status.diskSize
// Yet `blockId` could only be `ShuffleIndexBlockId` or `ShuffleDataBlockId` when it's a
// shuffle block because of decommission.
val bmId = if (blockId.isShuffle) shuffleServerId else blockManagerId
master.updateBlockInfo(bmId, blockId, storageLevel, inMemSize, onDiskSize)
}
/**
* Return the updated storage status of the block with the given ID. More specifically, if
* the block is dropped from memory and possibly added to disk, return the new storage level
* and the updated in-memory and on-disk sizes.
*/
private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = {
info.synchronized {
info.level match {
case null =>
BlockStatus.empty
case level =>
val inMem = level.useMemory && memoryStore.contains(blockId)
val onDisk = level.useDisk && diskStore.contains(blockId)
val deserialized = if (inMem) level.deserialized else false
val replication = if (inMem || onDisk) level.replication else 1
val storageLevel = StorageLevel(
useDisk = onDisk,
useMemory = inMem,
useOffHeap = level.useOffHeap,
deserialized = deserialized,
replication = replication)
val memSize = if (inMem) memoryStore.getSize(blockId) else 0L
val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L
BlockStatus(storageLevel, memSize, diskSize)
}
}
}
/**
* Get locations of an array of blocks.
*/
private def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = {
val startTimeNs = System.nanoTime()
val locations = master.getLocations(blockIds).toArray
logDebug(s"Got multiple block location in ${Utils.getUsedTimeNs(startTimeNs)}")
locations
}
/**
* Cleanup code run in response to a failed local read.
* Must be called while holding a read lock on the block.
*/
private def handleLocalReadFailure(blockId: BlockId): Nothing = {
releaseLock(blockId)
// Remove the missing block so that its unavailability is reported to the driver
removeBlock(blockId)
throw SparkCoreErrors.readLockedBlockNotFoundError(blockId)
}
private def isIORelatedException(t: Throwable): Boolean =
t.isInstanceOf[IOException] ||
(t.isInstanceOf[KryoException] && t.getCause.isInstanceOf[IOException])
/**
* Get block from local block manager as an iterator of Java objects.
*/
def getLocalValues(blockId: BlockId): Option[BlockResult] = {
logDebug(s"Getting local block $blockId")
blockInfoManager.lockForReading(blockId) match {
case None =>
logDebug(s"Block $blockId was not found")
None
case Some(info) =>
val level = info.level
logDebug(s"Level for block $blockId is $level")
val taskContext = Option(TaskContext.get())
if (level.useMemory && memoryStore.contains(blockId)) {
val iter: Iterator[Any] = if (level.deserialized) {
memoryStore.getValues(blockId).get
} else {
serializerManager.dataDeserializeStream(
blockId, memoryStore.getBytes(blockId).get.toInputStream())(info.classTag)
}
// We need to capture the current taskId in case the iterator completion is triggered
// from a different thread which does not have TaskContext set; see SPARK-18406 for
// discussion.
val ci = CompletionIterator[Any, Iterator[Any]](iter, {
releaseLock(blockId, taskContext)
})
Some(new BlockResult(ci, DataReadMethod.Memory, info.size))
} else if (level.useDisk && diskStore.contains(blockId)) {
var diskData: BlockData = null
try {
diskData = diskStore.getBytes(blockId)
val iterToReturn = if (level.deserialized) {
val diskValues = serializerManager.dataDeserializeStream(
blockId,
diskData.toInputStream())(info.classTag)
maybeCacheDiskValuesInMemory(info, blockId, level, diskValues)
} else {
val stream = maybeCacheDiskBytesInMemory(info, blockId, level, diskData)
.map { _.toInputStream() }
.getOrElse { diskData.toInputStream() }
serializerManager.dataDeserializeStream(blockId, stream)(info.classTag)
}
val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, {
releaseLockAndDispose(blockId, diskData, taskContext)
})
Some(new BlockResult(ci, DataReadMethod.Disk, info.size))