Skip to content

Commit f46e927

Browse files
committed
Merge remote-tracking branch 'apache/master' into SPARK-3530
2 parents 1ef26e0 + 3d2b5bc commit f46e927

File tree

58 files changed

+2099
-357
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

58 files changed

+2099
-357
lines changed

LICENSE

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -754,7 +754,7 @@ SUCH DAMAGE.
754754

755755

756756
========================================================================
757-
For Timsort (core/src/main/java/org/apache/spark/util/collection/Sorter.java):
757+
For Timsort (core/src/main/java/org/apache/spark/util/collection/TimSort.java):
758758
========================================================================
759759
Copyright (C) 2008 The Android Open Source Project
760760

@@ -771,6 +771,25 @@ See the License for the specific language governing permissions and
771771
limitations under the License.
772772

773773

774+
========================================================================
775+
For LimitedInputStream
776+
(network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java):
777+
========================================================================
778+
Copyright (C) 2007 The Guava Authors
779+
780+
Licensed under the Apache License, Version 2.0 (the "License");
781+
you may not use this file except in compliance with the License.
782+
You may obtain a copy of the License at
783+
784+
http://www.apache.org/licenses/LICENSE-2.0
785+
786+
Unless required by applicable law or agreed to in writing, software
787+
distributed under the License is distributed on an "AS IS" BASIS,
788+
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
789+
See the License for the specific language governing permissions and
790+
limitations under the License.
791+
792+
774793
========================================================================
775794
BSD-style licenses
776795
========================================================================

core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala

Lines changed: 27 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -66,7 +66,6 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
6666
// Lower and upper bounds on the number of executors. These are required.
6767
private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
6868
private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
69-
verifyBounds()
7069

7170
// How long there must be backlogged tasks for before an addition is triggered
7271
private val schedulerBacklogTimeout = conf.getLong(
@@ -77,9 +76,14 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
7776
"spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
7877

7978
// How long an executor must be idle for before it is removed
80-
private val removeThresholdSeconds = conf.getLong(
79+
private val executorIdleTimeout = conf.getLong(
8180
"spark.dynamicAllocation.executorIdleTimeout", 600)
8281

82+
// During testing, the methods to actually kill and add executors are mocked out
83+
private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false)
84+
85+
validateSettings()
86+
8387
// Number of executors to add in the next round
8488
private var numExecutorsToAdd = 1
8589

@@ -103,17 +107,14 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
103107
// Polling loop interval (ms)
104108
private val intervalMillis: Long = 100
105109

106-
// Whether we are testing this class. This should only be used internally.
107-
private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false)
108-
109110
// Clock used to schedule when executors should be added and removed
110111
private var clock: Clock = new RealClock
111112

112113
/**
113-
* Verify that the lower and upper bounds on the number of executors are valid.
114+
* Verify that the settings specified through the config are valid.
114115
* If not, throw an appropriate exception.
115116
*/
116-
private def verifyBounds(): Unit = {
117+
private def validateSettings(): Unit = {
117118
if (minNumExecutors < 0 || maxNumExecutors < 0) {
118119
throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
119120
}
@@ -124,6 +125,22 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
124125
throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " +
125126
s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!")
126127
}
128+
if (schedulerBacklogTimeout <= 0) {
129+
throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!")
130+
}
131+
if (sustainedSchedulerBacklogTimeout <= 0) {
132+
throw new SparkException(
133+
"spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!")
134+
}
135+
if (executorIdleTimeout <= 0) {
136+
throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!")
137+
}
138+
// Require external shuffle service for dynamic allocation
139+
// Otherwise, we may lose shuffle files when killing executors
140+
if (!conf.getBoolean("spark.shuffle.service.enabled", false) && !testing) {
141+
throw new SparkException("Dynamic allocation of executors requires the external " +
142+
"shuffle service. You may enable this through spark.shuffle.service.enabled.")
143+
}
127144
}
128145

129146
/**
@@ -254,7 +271,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
254271
val removeRequestAcknowledged = testing || sc.killExecutor(executorId)
255272
if (removeRequestAcknowledged) {
256273
logInfo(s"Removing executor $executorId because it has been idle for " +
257-
s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})")
274+
s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})")
258275
executorsPendingToRemove.add(executorId)
259276
true
260277
} else {
@@ -329,8 +346,8 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
329346
private def onExecutorIdle(executorId: String): Unit = synchronized {
330347
if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) {
331348
logDebug(s"Starting idle timer for $executorId because there are no more tasks " +
332-
s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)")
333-
removeTimes(executorId) = clock.getTimeMillis + removeThresholdSeconds * 1000
349+
s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)")
350+
removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000
334351
}
335352
}
336353

core/src/main/scala/org/apache/spark/SparkEnv.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -287,7 +287,7 @@ object SparkEnv extends Logging {
287287

288288
// NB: blockManager is not valid until initialize() is called later.
289289
val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster,
290-
serializer, conf, mapOutputTracker, shuffleManager, blockTransferService)
290+
serializer, conf, mapOutputTracker, shuffleManager, blockTransferService, securityManager)
291291

292292
val broadcastManager = new BroadcastManager(isDriver, conf, securityManager)
293293

core/src/main/scala/org/apache/spark/executor/Executor.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -161,7 +161,7 @@ private[spark] class Executor(
161161
}
162162

163163
override def run() {
164-
val startTime = System.currentTimeMillis()
164+
val deserializeStartTime = System.currentTimeMillis()
165165
Thread.currentThread.setContextClassLoader(replClassLoader)
166166
val ser = SparkEnv.get.closureSerializer.newInstance()
167167
logInfo(s"Running $taskName (TID $taskId)")
@@ -206,7 +206,7 @@ private[spark] class Executor(
206206
val afterSerialization = System.currentTimeMillis()
207207

208208
for (m <- task.metrics) {
209-
m.executorDeserializeTime = taskStart - startTime
209+
m.executorDeserializeTime = taskStart - deserializeStartTime
210210
m.executorRunTime = taskFinish - taskStart
211211
m.jvmGCTime = gcTime - startGCTime
212212
m.resultSerializationTime = afterSerialization - beforeSerialization

core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,7 @@ private[spark] class FixedLengthBinaryRecordReader
115115
if (currentPosition < splitEnd) {
116116
// setup a buffer to store the record
117117
val buffer = recordValue.getBytes
118-
fileInputStream.read(buffer, 0, recordLength)
118+
fileInputStream.readFully(buffer)
119119
// update our current position
120120
currentPosition = currentPosition + recordLength
121121
// return true

core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -93,7 +93,7 @@ private[spark] class CoarseMesosSchedulerBackend(
9393
setDaemon(true)
9494
override def run() {
9595
val scheduler = CoarseMesosSchedulerBackend.this
96-
val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build()
96+
val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build()
9797
driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
9898
try { {
9999
val ret = driver.run()
@@ -242,8 +242,7 @@ private[spark] class CoarseMesosSchedulerBackend(
242242
for (r <- res if r.getName == name) {
243243
return r.getScalar.getValue
244244
}
245-
// If we reached here, no resource with the required name was present
246-
throw new IllegalArgumentException("No resource called " + name + " in " + res)
245+
0
247246
}
248247

249248
/** Build a Mesos resource protobuf object */

core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,7 @@ private[spark] class MesosSchedulerBackend(
7272
setDaemon(true)
7373
override def run() {
7474
val scheduler = MesosSchedulerBackend.this
75-
val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build()
75+
val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build()
7676
driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
7777
try {
7878
val ret = driver.run()
@@ -278,8 +278,7 @@ private[spark] class MesosSchedulerBackend(
278278
for (r <- res if r.getName == name) {
279279
return r.getScalar.getValue
280280
}
281-
// If we reached here, no resource with the required name was present
282-
throw new IllegalArgumentException("No resource called " + name + " in " + res)
281+
0
283282
}
284283

285284
/** Turn a Spark TaskDescription into a Mesos task */

core/src/main/scala/org/apache/spark/storage/BlockManager.scala

Lines changed: 13 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,6 @@ import org.apache.spark.network.util.{ConfigProvider, TransportConf}
4040
import org.apache.spark.serializer.Serializer
4141
import org.apache.spark.shuffle.ShuffleManager
4242
import org.apache.spark.shuffle.hash.HashShuffleManager
43-
import org.apache.spark.shuffle.sort.SortShuffleManager
4443
import org.apache.spark.util._
4544

4645
private[spark] sealed trait BlockValues
@@ -72,7 +71,8 @@ private[spark] class BlockManager(
7271
val conf: SparkConf,
7372
mapOutputTracker: MapOutputTracker,
7473
shuffleManager: ShuffleManager,
75-
blockTransferService: BlockTransferService)
74+
blockTransferService: BlockTransferService,
75+
securityManager: SecurityManager)
7676
extends BlockDataManager with Logging {
7777

7878
val diskBlockManager = new DiskBlockManager(this, conf)
@@ -96,7 +96,12 @@ private[spark] class BlockManager(
9696

9797
private[spark]
9898
val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false)
99-
private val externalShuffleServicePort = conf.getInt("spark.shuffle.service.port", 7337)
99+
100+
// Port used by the external shuffle service. In Yarn mode, this may be already be
101+
// set through the Hadoop configuration as the server is launched in the Yarn NM.
102+
private val externalShuffleServicePort =
103+
Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt
104+
100105
// Check that we're not using external shuffle service with consolidated shuffle files.
101106
if (externalShuffleServiceEnabled
102107
&& conf.getBoolean("spark.shuffle.consolidateFiles", false)
@@ -115,7 +120,8 @@ private[spark] class BlockManager(
115120
// Client to read other executors' shuffle files. This is either an external service, or just the
116121
// standard BlockTranserService to directly connect to other Executors.
117122
private[spark] val shuffleClient = if (externalShuffleServiceEnabled) {
118-
new ExternalShuffleClient(SparkTransportConf.fromSparkConf(conf))
123+
new ExternalShuffleClient(SparkTransportConf.fromSparkConf(conf), securityManager,
124+
securityManager.isAuthenticationEnabled())
119125
} else {
120126
blockTransferService
121127
}
@@ -166,9 +172,10 @@ private[spark] class BlockManager(
166172
conf: SparkConf,
167173
mapOutputTracker: MapOutputTracker,
168174
shuffleManager: ShuffleManager,
169-
blockTransferService: BlockTransferService) = {
175+
blockTransferService: BlockTransferService,
176+
securityManager: SecurityManager) = {
170177
this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf),
171-
conf, mapOutputTracker, shuffleManager, blockTransferService)
178+
conf, mapOutputTracker, shuffleManager, blockTransferService, securityManager)
172179
}
173180

174181
/**
@@ -219,7 +226,6 @@ private[spark] class BlockManager(
219226
return
220227
} catch {
221228
case e: Exception if i < MAX_ATTEMPTS =>
222-
val attemptsRemaining =
223229
logError(s"Failed to connect to external shuffle server, will retry ${MAX_ATTEMPTS - i}}"
224230
+ s" more times after waiting $SLEEP_TIME_SECS seconds...", e)
225231
Thread.sleep(SLEEP_TIME_SECS * 1000)

core/src/main/scala/org/apache/spark/ui/ToolTips.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,9 @@ private[spark] object ToolTips {
2424
scheduler delay is large, consider decreasing the size of tasks or decreasing the size
2525
of task results."""
2626

27+
val TASK_DESERIALIZATION_TIME =
28+
"""Time spent deserializating the task closure on the executor."""
29+
2730
val INPUT = "Bytes read from Hadoop or from Spark storage."
2831

2932
val SHUFFLE_WRITE = "Bytes written to disk in order to be read by a shuffle in a future stage."

core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala

Lines changed: 30 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -112,6 +112,13 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
112112
<span class="additional-metric-title">Scheduler Delay</span>
113113
</span>
114114
</li>
115+
<li>
116+
<span data-toggle="tooltip"
117+
title={ToolTips.TASK_DESERIALIZATION_TIME} data-placement="right">
118+
<input type="checkbox" name={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}/>
119+
<span class="additional-metric-title">Task Deserialization Time</span>
120+
</span>
121+
</li>
115122
<li>
116123
<span data-toggle="tooltip"
117124
title={ToolTips.GC_TIME} data-placement="right">
@@ -147,6 +154,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
147154
("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""),
148155
("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""),
149156
("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY),
157+
("Task Deserialization Time", TaskDetailsClassNames.TASK_DESERIALIZATION_TIME),
150158
("GC Time", TaskDetailsClassNames.GC_TIME),
151159
("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME),
152160
("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++
@@ -179,6 +187,17 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
179187
}
180188
}
181189

190+
val deserializationTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
191+
metrics.get.executorDeserializeTime.toDouble
192+
}
193+
val deserializationQuantiles =
194+
<td>
195+
<span data-toggle="tooltip" title={ToolTips.TASK_DESERIALIZATION_TIME}
196+
data-placement="right">
197+
Task Deserialization Time
198+
</span>
199+
</td> +: getFormattedTimeQuantiles(deserializationTimes)
200+
182201
val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
183202
metrics.get.executorRunTime.toDouble
184203
}
@@ -266,6 +285,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
266285
val listings: Seq[Seq[Node]] = Seq(
267286
<tr>{serviceQuantiles}</tr>,
268287
<tr class={TaskDetailsClassNames.SCHEDULER_DELAY}>{schedulerDelayQuantiles}</tr>,
288+
<tr class={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}>
289+
{deserializationQuantiles}
290+
</tr>
269291
<tr class={TaskDetailsClassNames.GC_TIME}>{gcQuantiles}</tr>,
270292
<tr class={TaskDetailsClassNames.RESULT_SERIALIZATION_TIME}>
271293
{serializationQuantiles}
@@ -314,6 +336,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
314336
else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("")
315337
val schedulerDelay = metrics.map(getSchedulerDelay(info, _)).getOrElse(0L)
316338
val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L)
339+
val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L)
317340
val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L)
318341
val gettingResultTime = info.gettingResultTime
319342

@@ -367,6 +390,10 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
367390
class={TaskDetailsClassNames.SCHEDULER_DELAY}>
368391
{UIUtils.formatDuration(schedulerDelay.toLong)}
369392
</td>
393+
<td sorttable_customkey={taskDeserializationTime.toString}
394+
class={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}>
395+
{UIUtils.formatDuration(taskDeserializationTime.toLong)}
396+
</td>
370397
<td sorttable_customkey={gcTime.toString} class={TaskDetailsClassNames.GC_TIME}>
371398
{if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""}
372399
</td>
@@ -424,6 +451,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
424451
(info.finishTime - info.launchTime)
425452
}
426453
}
427-
totalExecutionTime - metrics.executorRunTime
454+
val executorOverhead = (metrics.executorDeserializeTime +
455+
metrics.resultSerializationTime)
456+
totalExecutionTime - metrics.executorRunTime - executorOverhead
428457
}
429458
}

0 commit comments

Comments
 (0)