/
SparkEnv.scala
537 lines (478 loc) · 19.1 KB
/
SparkEnv.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
/*
* 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.File
import java.util.Locale
import scala.collection.JavaConverters._
import scala.collection.concurrent
import scala.collection.mutable
import scala.util.Properties
import com.google.common.cache.CacheBuilder
import org.apache.hadoop.conf.Configuration
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.api.python.{PythonWorker, PythonWorkerFactory}
import org.apache.spark.broadcast.BroadcastManager
import org.apache.spark.executor.ExecutorBackend
import org.apache.spark.internal.{config, Logging}
import org.apache.spark.internal.config._
import org.apache.spark.memory.{MemoryManager, UnifiedMemoryManager}
import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances}
import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf}
import org.apache.spark.network.shuffle.ExternalBlockStoreClient
import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv}
import org.apache.spark.scheduler.{LiveListenerBus, OutputCommitCoordinator}
import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorEndpoint
import org.apache.spark.security.CryptoStreamUtils
import org.apache.spark.serializer.{JavaSerializer, Serializer, SerializerManager}
import org.apache.spark.shuffle.ShuffleManager
import org.apache.spark.storage._
import org.apache.spark.util.{RpcUtils, Utils}
/**
* :: DeveloperApi ::
* Holds all the runtime environment objects for a running Spark instance (either master or worker),
* including the serializer, RpcEnv, block manager, map output tracker, etc. Currently
* Spark code finds the SparkEnv through a global variable, so all the threads can access the same
* SparkEnv. It can be accessed by SparkEnv.get (e.g. after creating a SparkContext).
*/
@DeveloperApi
class SparkEnv (
val executorId: String,
private[spark] val rpcEnv: RpcEnv,
val serializer: Serializer,
val closureSerializer: Serializer,
val serializerManager: SerializerManager,
val mapOutputTracker: MapOutputTracker,
val shuffleManager: ShuffleManager,
val broadcastManager: BroadcastManager,
val blockManager: BlockManager,
val securityManager: SecurityManager,
val metricsSystem: MetricsSystem,
val memoryManager: MemoryManager,
val outputCommitCoordinator: OutputCommitCoordinator,
val conf: SparkConf) extends Logging {
@volatile private[spark] var isStopped = false
/**
* A key for PythonWorkerFactory cache.
* @param pythonExec The python executable to run the Python worker.
* @param workerModule The worker module to be called in the worker, e.g., "pyspark.worker".
* @param daemonModule The daemon module name to reuse the worker, e.g., "pyspark.daemon".
* @param envVars The environment variables for the worker.
*/
private case class PythonWorkersKey(
pythonExec: String, workerModule: String, daemonModule: String, envVars: Map[String, String])
private val pythonWorkers = mutable.HashMap[PythonWorkersKey, PythonWorkerFactory]()
// A general, soft-reference map for metadata needed during HadoopRDD split computation
// (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
private[spark] val hadoopJobMetadata =
CacheBuilder.newBuilder().maximumSize(1000).softValues().build[String, AnyRef]().asMap()
private[spark] var driverTmpDir: Option[String] = None
private[spark] var executorBackend: Option[ExecutorBackend] = None
private[spark] def stop(): Unit = {
if (!isStopped) {
isStopped = true
pythonWorkers.values.foreach(_.stop())
mapOutputTracker.stop()
shuffleManager.stop()
broadcastManager.stop()
blockManager.stop()
blockManager.master.stop()
metricsSystem.stop()
outputCommitCoordinator.stop()
rpcEnv.shutdown()
rpcEnv.awaitTermination()
// If we only stop sc, but the driver process still run as a services then we need to delete
// the tmp dir, if not, it will create too many tmp dirs.
// We only need to delete the tmp dir create by driver
driverTmpDir match {
case Some(path) =>
try {
Utils.deleteRecursively(new File(path))
} catch {
case e: Exception =>
logWarning(s"Exception while deleting Spark temp dir: $path", e)
}
case None => // We just need to delete tmp dir created by driver, so do nothing on executor
}
}
}
private[spark] def createPythonWorker(
pythonExec: String,
workerModule: String,
daemonModule: String,
envVars: Map[String, String]): (PythonWorker, Option[Int]) = {
synchronized {
val key = PythonWorkersKey(pythonExec, workerModule, daemonModule, envVars)
pythonWorkers.getOrElseUpdate(key,
new PythonWorkerFactory(pythonExec, workerModule, daemonModule, envVars)).create()
}
}
private[spark] def createPythonWorker(
pythonExec: String,
workerModule: String,
envVars: Map[String, String]): (PythonWorker, Option[Int]) = {
createPythonWorker(
pythonExec, workerModule, PythonWorkerFactory.defaultDaemonModule, envVars)
}
private[spark] def destroyPythonWorker(
pythonExec: String,
workerModule: String,
daemonModule: String,
envVars: Map[String, String],
worker: PythonWorker): Unit = {
synchronized {
val key = PythonWorkersKey(pythonExec, workerModule, daemonModule, envVars)
pythonWorkers.get(key).foreach(_.stopWorker(worker))
}
}
private[spark] def destroyPythonWorker(
pythonExec: String,
workerModule: String,
envVars: Map[String, String],
worker: PythonWorker): Unit = {
destroyPythonWorker(
pythonExec, workerModule, PythonWorkerFactory.defaultDaemonModule, envVars, worker)
}
private[spark] def releasePythonWorker(
pythonExec: String,
workerModule: String,
daemonModule: String,
envVars: Map[String, String],
worker: PythonWorker): Unit = {
synchronized {
val key = PythonWorkersKey(pythonExec, workerModule, daemonModule, envVars)
pythonWorkers.get(key).foreach(_.releaseWorker(worker))
}
}
private[spark] def releasePythonWorker(
pythonExec: String,
workerModule: String,
envVars: Map[String, String],
worker: PythonWorker): Unit = {
releasePythonWorker(
pythonExec, workerModule, PythonWorkerFactory.defaultDaemonModule, envVars, worker)
}
}
object SparkEnv extends Logging {
@volatile private var env: SparkEnv = _
private[spark] val driverSystemName = "sparkDriver"
private[spark] val executorSystemName = "sparkExecutor"
def set(e: SparkEnv): Unit = {
env = e
}
/**
* Returns the SparkEnv.
*/
def get: SparkEnv = {
env
}
/**
* Create a SparkEnv for the driver.
*/
private[spark] def createDriverEnv(
conf: SparkConf,
isLocal: Boolean,
listenerBus: LiveListenerBus,
numCores: Int,
sparkContext: SparkContext,
mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = {
assert(conf.contains(DRIVER_HOST_ADDRESS),
s"${DRIVER_HOST_ADDRESS.key} is not set on the driver!")
assert(conf.contains(DRIVER_PORT), s"${DRIVER_PORT.key} is not set on the driver!")
val bindAddress = conf.get(DRIVER_BIND_ADDRESS)
val advertiseAddress = conf.get(DRIVER_HOST_ADDRESS)
val port = conf.get(DRIVER_PORT)
val ioEncryptionKey = if (conf.get(IO_ENCRYPTION_ENABLED)) {
Some(CryptoStreamUtils.createKey(conf))
} else {
None
}
create(
conf,
SparkContext.DRIVER_IDENTIFIER,
bindAddress,
advertiseAddress,
Option(port),
isLocal,
numCores,
ioEncryptionKey,
listenerBus = listenerBus,
Option(sparkContext),
mockOutputCommitCoordinator = mockOutputCommitCoordinator
)
}
/**
* Create a SparkEnv for an executor.
* In coarse-grained mode, the executor provides an RpcEnv that is already instantiated.
*/
private[spark] def createExecutorEnv(
conf: SparkConf,
executorId: String,
bindAddress: String,
hostname: String,
numCores: Int,
ioEncryptionKey: Option[Array[Byte]],
isLocal: Boolean): SparkEnv = {
val env = create(
conf,
executorId,
bindAddress,
hostname,
None,
isLocal,
numCores,
ioEncryptionKey
)
SparkEnv.set(env)
env
}
private[spark] def createExecutorEnv(
conf: SparkConf,
executorId: String,
hostname: String,
numCores: Int,
ioEncryptionKey: Option[Array[Byte]],
isLocal: Boolean): SparkEnv = {
createExecutorEnv(conf, executorId, hostname,
hostname, numCores, ioEncryptionKey, isLocal)
}
/**
* Helper method to create a SparkEnv for a driver or an executor.
*/
// scalastyle:off argcount
private def create(
conf: SparkConf,
executorId: String,
bindAddress: String,
advertiseAddress: String,
port: Option[Int],
isLocal: Boolean,
numUsableCores: Int,
ioEncryptionKey: Option[Array[Byte]],
listenerBus: LiveListenerBus = null,
sc: Option[SparkContext] = None,
mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = {
// scalastyle:on argcount
val isDriver = executorId == SparkContext.DRIVER_IDENTIFIER
// Listener bus is only used on the driver
if (isDriver) {
assert(listenerBus != null, "Attempted to create driver SparkEnv with null listener bus!")
}
val authSecretFileConf = if (isDriver) AUTH_SECRET_FILE_DRIVER else AUTH_SECRET_FILE_EXECUTOR
val securityManager = new SecurityManager(conf, ioEncryptionKey, authSecretFileConf)
if (isDriver) {
securityManager.initializeAuth()
}
ioEncryptionKey.foreach { _ =>
if (!securityManager.isEncryptionEnabled()) {
logWarning("I/O encryption enabled without RPC encryption: keys will be visible on the " +
"wire.")
}
}
val systemName = if (isDriver) driverSystemName else executorSystemName
val rpcEnv = RpcEnv.create(systemName, bindAddress, advertiseAddress, port.getOrElse(-1), conf,
securityManager, numUsableCores, !isDriver)
// Figure out which port RpcEnv actually bound to in case the original port is 0 or occupied.
if (isDriver) {
conf.set(DRIVER_PORT, rpcEnv.address.port)
}
val serializer = Utils.instantiateSerializerFromConf[Serializer](SERIALIZER, conf, isDriver)
logDebug(s"Using serializer: ${serializer.getClass}")
val serializerManager = new SerializerManager(serializer, conf, ioEncryptionKey)
val closureSerializer = new JavaSerializer(conf)
def registerOrLookupEndpoint(
name: String, endpointCreator: => RpcEndpoint):
RpcEndpointRef = {
if (isDriver) {
logInfo("Registering " + name)
rpcEnv.setupEndpoint(name, endpointCreator)
} else {
RpcUtils.makeDriverRef(name, conf, rpcEnv)
}
}
val broadcastManager = new BroadcastManager(isDriver, conf)
val mapOutputTracker = if (isDriver) {
new MapOutputTrackerMaster(conf, broadcastManager, isLocal)
} else {
new MapOutputTrackerWorker(conf)
}
// Have to assign trackerEndpoint after initialization as MapOutputTrackerEndpoint
// requires the MapOutputTracker itself
mapOutputTracker.trackerEndpoint = registerOrLookupEndpoint(MapOutputTracker.ENDPOINT_NAME,
new MapOutputTrackerMasterEndpoint(
rpcEnv, mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf))
// Let the user specify short names for shuffle managers
val shortShuffleMgrNames = Map(
"sort" -> classOf[org.apache.spark.shuffle.sort.SortShuffleManager].getName,
"tungsten-sort" -> classOf[org.apache.spark.shuffle.sort.SortShuffleManager].getName)
val shuffleMgrName = conf.get(config.SHUFFLE_MANAGER)
val shuffleMgrClass =
shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase(Locale.ROOT), shuffleMgrName)
val shuffleManager = Utils.instantiateSerializerOrShuffleManager[ShuffleManager](
shuffleMgrClass, conf, isDriver)
val memoryManager: MemoryManager = UnifiedMemoryManager(conf, numUsableCores)
val blockManagerPort = if (isDriver) {
conf.get(DRIVER_BLOCK_MANAGER_PORT)
} else {
conf.get(BLOCK_MANAGER_PORT)
}
val externalShuffleClient = if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) {
val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores)
Some(new ExternalBlockStoreClient(transConf, securityManager,
securityManager.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT)))
} else {
None
}
// Mapping from block manager id to the block manager's information.
val blockManagerInfo = new concurrent.TrieMap[BlockManagerId, BlockManagerInfo]()
val blockManagerMaster = new BlockManagerMaster(
registerOrLookupEndpoint(
BlockManagerMaster.DRIVER_ENDPOINT_NAME,
new BlockManagerMasterEndpoint(
rpcEnv,
isLocal,
conf,
listenerBus,
if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) {
externalShuffleClient
} else {
None
}, blockManagerInfo,
mapOutputTracker.asInstanceOf[MapOutputTrackerMaster],
shuffleManager,
isDriver)),
registerOrLookupEndpoint(
BlockManagerMaster.DRIVER_HEARTBEAT_ENDPOINT_NAME,
new BlockManagerMasterHeartbeatEndpoint(rpcEnv, isLocal, blockManagerInfo)),
conf,
isDriver)
val blockTransferService =
new NettyBlockTransferService(conf, securityManager, serializerManager, bindAddress,
advertiseAddress, blockManagerPort, numUsableCores, blockManagerMaster.driverEndpoint)
// NB: blockManager is not valid until initialize() is called later.
val blockManager = new BlockManager(
executorId,
rpcEnv,
blockManagerMaster,
serializerManager,
conf,
memoryManager,
mapOutputTracker,
shuffleManager,
blockTransferService,
securityManager,
externalShuffleClient)
val metricsSystem = if (isDriver) {
// Don't start metrics system right now for Driver.
// We need to wait for the task scheduler to give us an app ID.
// Then we can start the metrics system.
MetricsSystem.createMetricsSystem(MetricsSystemInstances.DRIVER, conf)
} else {
// We need to set the executor ID before the MetricsSystem is created because sources and
// sinks specified in the metrics configuration file will want to incorporate this executor's
// ID into the metrics they report.
conf.set(EXECUTOR_ID, executorId)
val ms = MetricsSystem.createMetricsSystem(MetricsSystemInstances.EXECUTOR, conf)
ms.start(conf.get(METRICS_STATIC_SOURCES_ENABLED))
ms
}
val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse {
if (isDriver) {
new OutputCommitCoordinator(conf, isDriver, sc)
} else {
new OutputCommitCoordinator(conf, isDriver)
}
}
val outputCommitCoordinatorRef = registerOrLookupEndpoint("OutputCommitCoordinator",
new OutputCommitCoordinatorEndpoint(rpcEnv, outputCommitCoordinator))
outputCommitCoordinator.coordinatorRef = Some(outputCommitCoordinatorRef)
val envInstance = new SparkEnv(
executorId,
rpcEnv,
serializer,
closureSerializer,
serializerManager,
mapOutputTracker,
shuffleManager,
broadcastManager,
blockManager,
securityManager,
metricsSystem,
memoryManager,
outputCommitCoordinator,
conf)
// Add a reference to tmp dir created by driver, we will delete this tmp dir when stop() is
// called, and we only need to do it for driver. Because driver may run as a service, and if we
// don't delete this tmp dir when sc is stopped, then will create too many tmp dirs.
if (isDriver) {
val sparkFilesDir = Utils.createTempDir(Utils.getLocalDir(conf), "userFiles").getAbsolutePath
envInstance.driverTmpDir = Some(sparkFilesDir)
}
envInstance
}
/**
* Return a map representation of jvm information, Spark properties, system properties, and
* class paths. Map keys define the category, and map values represent the corresponding
* attributes as a sequence of KV pairs. This is used mainly for SparkListenerEnvironmentUpdate.
*/
private[spark] def environmentDetails(
conf: SparkConf,
hadoopConf: Configuration,
schedulingMode: String,
addedJars: Seq[String],
addedFiles: Seq[String],
addedArchives: Seq[String],
metricsProperties: Map[String, String]): Map[String, Seq[(String, String)]] = {
import Properties._
val jvmInformation = Seq(
("Java Version", s"$javaVersion ($javaVendor)"),
("Java Home", javaHome),
("Scala Version", versionString)
).sorted
// Spark properties
// This includes the scheduling mode whether or not it is configured (used by SparkUI)
val schedulerMode =
if (!conf.contains(SCHEDULER_MODE)) {
Seq((SCHEDULER_MODE.key, schedulingMode))
} else {
Seq.empty[(String, String)]
}
val sparkProperties = (conf.getAll ++ schedulerMode).sorted
// System properties that are not java classpaths
val systemProperties = Utils.getSystemProperties.toSeq
val otherProperties = systemProperties.filter { case (k, _) =>
k != "java.class.path" && !k.startsWith("spark.")
}.sorted
// Class paths including all added jars and files
val classPathEntries = javaClassPath
.split(File.pathSeparator)
.filterNot(_.isEmpty)
.map((_, "System Classpath"))
val addedJarsAndFiles = (addedJars ++ addedFiles ++ addedArchives).map((_, "Added By User"))
val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted
// Add Hadoop properties, it will not ignore configs including in Spark. Some spark
// conf starting with "spark.hadoop" may overwrite it.
val hadoopProperties = hadoopConf.asScala
.map(entry => (entry.getKey, entry.getValue)).toSeq.sorted
Map[String, Seq[(String, String)]](
"JVM Information" -> jvmInformation,
"Spark Properties" -> sparkProperties,
"Hadoop Properties" -> hadoopProperties,
"System Properties" -> otherProperties,
"Classpath Entries" -> classPaths,
"Metrics Properties" -> metricsProperties.toSeq.sorted)
}
}