-
Notifications
You must be signed in to change notification settings - Fork 0
/
SparkHadoopMultiWriter.scala
282 lines (237 loc) · 9.82 KB
/
SparkHadoopMultiWriter.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
package org.apache.spark
import java.util.Date
import java.util.concurrent.ConcurrentHashMap
import org.apache.hadoop.conf.{Configurable, Configuration}
import org.apache.hadoop.mapreduce.task.{TaskAttemptContextImpl => NewTaskAttemptContextImpl}
import org.apache.hadoop.mapreduce.{
TaskType,
JobContext => NewJobContext,
OutputFormat => NewOutputFormat,
RecordWriter => NewRecordWriter,
TaskAttemptContext => NewTaskAttemptContext,
TaskAttemptID => NewTaskAttemptID
}
import org.apache.spark.executor.OutputMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
import org.apache.spark.internal.io.{
FileCommitProtocol,
HadoopMapReduceCommitProtocol,
HadoopWriteConfigUtil,
SparkHadoopWriterUtils
}
import org.apache.spark.rdd.RDD
import org.apache.spark.util.{SerializableConfiguration, Utils}
import scala.reflect.ClassTag
object SparkHadoopMultiWriter extends Logging {
private val RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES = 256
def write[K, V: ClassTag](
rdd: RDD[(String, (K, V))],
config: HadoopMapReduceMultiOutputWriteConfigUtil[K, V]): Map[String, Long] = {
import SparkHadoopWriterUtils._
// Extract context and configuration from RDD.
val sparkContext = rdd.context
val commitJobId = rdd.id
// Set up a job.
val jobTrackerId = createJobTrackerID(new Date())
val jobContext = config.createJobContext(jobTrackerId, commitJobId)
config.initOutputFormat(jobContext)
// Assert the output format/key/value class is set in JobConf.
config.assertConf(jobContext, rdd.conf)
val committer = config.createCommitter(commitJobId)
committer.setupJob(jobContext)
// Try to write all RDD partitions as a Hadoop OutputFormat.
try {
val ret = sparkContext.runJob(
rdd,
(context: TaskContext, iter: Iterator[(String, (K, V))]) => {
// SPARK-24552: Generate a unique "attempt ID" based on the stage and task attempt numbers.
// Assumes that there won't be more than Short.MaxValue attempts, at least not concurrently.
val attemptId = (context.stageAttemptNumber << 16) | context.attemptNumber
executeTask(
context = context,
config = config,
jobTrackerId = jobTrackerId,
commitJobId = commitJobId,
sparkPartitionId = context.partitionId,
sparkAttemptNumber = attemptId,
committer = committer,
iterator = iter
)
}
)
val commitMessageList = ret.map(_._2)
committer.commitJob(jobContext, commitMessageList)
logInfo(s"Job ${jobContext.getJobID} committed.")
ret
.map(_._1)
.reduce((x, y) =>
(x.toList ++ y.toList).groupBy(_._1).map { case (k, v) => k -> v.map(_._2).sum })
} catch {
case cause: Throwable =>
logError(s"Aborting job ${jobContext.getJobID}.", cause)
committer.abortJob(jobContext)
throw new SparkException("Job aborted.", cause)
}
}
private def executeTask[K, V: ClassTag](
context: TaskContext,
config: HadoopMapReduceMultiOutputWriteConfigUtil[K, V],
jobTrackerId: String,
commitJobId: Int,
sparkPartitionId: Int,
sparkAttemptNumber: Int,
committer: FileCommitProtocol,
iterator: Iterator[(String, (K, V))]): (Map[String, Long], TaskCommitMessage) = {
// Set up a task.
val taskContext = config
.createTaskAttemptContext(jobTrackerId, commitJobId, sparkPartitionId, sparkAttemptNumber)
committer.setupTask(taskContext)
val outputMetrics = initHadoopOutputMetrics(context)
// Initiate the writer.
config.initWriter(taskContext, sparkPartitionId)
var recordsWritten = 0L
// Write all rows in RDD partition.
try {
val ret = Utils.tryWithSafeFinallyAndFailureCallbacks {
val partitionCountMap = scala.collection.mutable.Map[String, Long]()
while (iterator.hasNext) {
val pair = iterator.next()
config.write(pair._1, pair._2)
// Update bytes written metric every few records
maybeUpdateOutputMetrics(outputMetrics, recordsWritten)
recordsWritten += 1
partitionCountMap.put(pair._1, partitionCountMap.getOrElse(pair._1, 0L) + 1L)
}
config.closeWriter(taskContext)
(partitionCountMap.toMap, committer.commitTask(taskContext))
}(catchBlock = {
// If there is an error, release resource and then abort the task.
try {
config.closeWriter(taskContext)
} finally {
committer.abortTask(taskContext)
logError(s"Task ${taskContext.getTaskAttemptID} aborted.")
}
})
outputMetrics.setRecordsWritten(recordsWritten)
ret
} catch {
case t: Throwable =>
throw new SparkException("Task failed while writing rows", t)
}
}
private def initHadoopOutputMetrics(context: TaskContext): OutputMetrics = {
context.taskMetrics().outputMetrics
}
private def maybeUpdateOutputMetrics(outputMetrics: OutputMetrics, recordsWritten: Long): Unit = {
if (recordsWritten % RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0) {
outputMetrics.setRecordsWritten(recordsWritten)
}
}
}
class HadoopMapReduceMultiOutputWriteConfigUtil[K, V: ClassTag](conf: SerializableConfiguration)
extends HadoopWriteConfigUtil[K, V]
with Logging {
private var outputFormat: Class[_ <: NewOutputFormat[K, V]] = null
private var recordWriters: ConcurrentHashMap[String, NewRecordWriter[K, V]] = null
private var taskContext: NewTaskAttemptContext = null
private val BASE_OUTPUT_NAME = "mapreduce.output.basename"
private val PART = "part"
private def getConf: Configuration = conf.value
// --------------------------------------------------------------------------
// Create JobContext/TaskAttemptContext
// --------------------------------------------------------------------------
override def createJobContext(jobTrackerId: String, jobId: Int): NewJobContext = {
val jobAttemptId = new NewTaskAttemptID(jobTrackerId, jobId, TaskType.MAP, 0, 0)
new NewTaskAttemptContextImpl(getConf, jobAttemptId)
}
override def createTaskAttemptContext(
jobTrackerId: String,
jobId: Int,
splitId: Int,
taskAttemptId: Int): NewTaskAttemptContext = {
val attemptId =
new NewTaskAttemptID(jobTrackerId, jobId, TaskType.REDUCE, splitId, taskAttemptId)
new NewTaskAttemptContextImpl(getConf, attemptId)
}
// --------------------------------------------------------------------------
// Create committer
// --------------------------------------------------------------------------
override def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol = {
FileCommitProtocol
.instantiate(
className = classOf[HadoopMapReduceCommitProtocol].getName,
jobId = jobId.toString,
outputPath = getConf.get("mapreduce.output.fileoutputformat.outputdir")
)
.asInstanceOf[HadoopMapReduceCommitProtocol]
}
// --------------------------------------------------------------------------
// Create writer
// --------------------------------------------------------------------------
override def initWriter(taskContext: NewTaskAttemptContext, splitId: Int): Unit = {
this.recordWriters = new ConcurrentHashMap[String, NewRecordWriter[K, V]]()
this.taskContext = taskContext
}
def write(baseOutputPath: String, pair: (K, V)): Unit = {
require(recordWriters != null, "Must call createWriter before write.")
getRecordWriter(taskContext, baseOutputPath).write(pair._1, pair._2)
}
override def write(pair: (K, V)): Unit = write("", pair)
private def getRecordWriter(
taskContext: NewTaskAttemptContext,
baseFileName: String): NewRecordWriter[K, V] = { // look for record-writer in the cache
var writer = recordWriters.get(baseFileName)
// If not in cache, create a new one
if (writer == null) { // get the record writer from context output format
taskContext.getConfiguration.set(BASE_OUTPUT_NAME, if (baseFileName.isEmpty) { PART } else {
baseFileName + "/" + PART
})
val taskFormat = getOutputFormat()
// If OutputFormat is Configurable, we should set conf to it.
taskFormat match {
case c: Configurable => c.setConf(getConf)
case _ => ()
}
writer = taskFormat
.getRecordWriter(taskContext)
.asInstanceOf[NewRecordWriter[K, V]]
// add the record-writer to the cache
recordWriters.put(baseFileName, writer)
}
writer
}
override def closeWriter(taskContext: NewTaskAttemptContext): Unit = {
if (recordWriters != null) {
import scala.collection.JavaConversions._
for (writer <- recordWriters.values) {
writer.close(taskContext)
}
recordWriters = null
} else {
logWarning("Writer has been closed.")
}
}
// --------------------------------------------------------------------------
// Create OutputFormat
// --------------------------------------------------------------------------
override def initOutputFormat(jobContext: NewJobContext): Unit = {
if (outputFormat == null) {
outputFormat = jobContext.getOutputFormatClass
.asInstanceOf[Class[_ <: NewOutputFormat[K, V]]]
}
}
private def getOutputFormat(): NewOutputFormat[K, V] = {
require(outputFormat != null, "Must call initOutputFormat first.")
outputFormat.newInstance()
}
// --------------------------------------------------------------------------
// Verify hadoop config
// --------------------------------------------------------------------------
override def assertConf(jobContext: NewJobContext, conf: SparkConf): Unit = {
if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) {
getOutputFormat().checkOutputSpecs(jobContext)
}
}
}