-
Notifications
You must be signed in to change notification settings - Fork 28k
/
ShuffleWriteProcessor.scala
99 lines (94 loc) · 3.97 KB
/
ShuffleWriteProcessor.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
/*
* 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.shuffle
import org.apache.spark.{ShuffleDependency, SparkEnv, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.MapStatus
/**
* The interface for customizing shuffle write process. The driver create a ShuffleWriteProcessor
* and put it into [[ShuffleDependency]], and executors use it in each ShuffleMapTask.
*/
private[spark] class ShuffleWriteProcessor extends Serializable with Logging {
/**
* Create a [[ShuffleWriteMetricsReporter]] from the task context. As the reporter is a
* per-row operator, here need a careful consideration on performance.
*/
protected def createMetricsReporter(context: TaskContext): ShuffleWriteMetricsReporter = {
context.taskMetrics().shuffleWriteMetrics
}
/**
* The write process for particular partition, it controls the life circle of [[ShuffleWriter]]
* get from [[ShuffleManager]] finally return the [[MapStatus]] for this task.
*/
def write(
inputs: Iterator[_],
dep: ShuffleDependency[_, _, _],
mapId: Long,
mapIndex: Int,
context: TaskContext): MapStatus = {
var writer: ShuffleWriter[Any, Any] = null
try {
val manager = SparkEnv.get.shuffleManager
writer = manager.getWriter[Any, Any](
dep.shuffleHandle,
mapId,
context,
createMetricsReporter(context))
writer.write(inputs.asInstanceOf[Iterator[_ <: Product2[Any, Any]]])
val mapStatus = writer.stop(success = true)
if (mapStatus.isDefined) {
// Check if sufficient shuffle mergers are available now for the ShuffleMapTask to push
if (dep.shuffleMergeAllowed && dep.getMergerLocs.isEmpty) {
val mapOutputTracker = SparkEnv.get.mapOutputTracker
val mergerLocs =
mapOutputTracker.getShufflePushMergerLocations(dep.shuffleId)
if (mergerLocs.nonEmpty) {
dep.setMergerLocs(mergerLocs)
}
}
// Initiate shuffle push process if push based shuffle is enabled
// The map task only takes care of converting the shuffle data file into multiple
// block push requests. It delegates pushing the blocks to a different thread-pool -
// ShuffleBlockPusher.BLOCK_PUSHER_POOL.
if (!dep.shuffleMergeFinalized) {
manager.shuffleBlockResolver match {
case resolver: IndexShuffleBlockResolver =>
logInfo(s"Shuffle merge enabled with ${dep.getMergerLocs.size} merger locations " +
s" for stage ${context.stageId()} with shuffle ID ${dep.shuffleId}")
logDebug(s"Starting pushing blocks for the task ${context.taskAttemptId()}")
val dataFile = resolver.getDataFile(dep.shuffleId, mapId)
new ShuffleBlockPusher(SparkEnv.get.conf)
.initiateBlockPush(dataFile, writer.getPartitionLengths(), dep, mapIndex)
case _ =>
}
}
}
mapStatus.get
} catch {
case e: Exception =>
try {
if (writer != null) {
writer.stop(success = false)
}
} catch {
case e: Exception =>
log.debug("Could not stop writer", e)
}
throw e
}
}
}