/
IndexShuffleBlockResolver.scala
469 lines (432 loc) · 17.6 KB
/
IndexShuffleBlockResolver.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
/*
* 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 java.io._
import java.nio.ByteBuffer
import java.nio.channels.Channels
import java.nio.file.Files
import org.apache.spark.{SparkConf, SparkEnv, SparkException}
import org.apache.spark.internal.{config, Logging}
import org.apache.spark.io.NioBufferedFileInputStream
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.{ExecutorDiskUtils, MergedBlockMeta}
import org.apache.spark.serializer.SerializerManager
import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID
import org.apache.spark.storage._
import org.apache.spark.util.Utils
/**
* Create and maintain the shuffle blocks' mapping between logic block and physical file location.
* Data of shuffle blocks from the same map task are stored in a single consolidated data file.
* The offsets of the data blocks in the data file are stored in a separate index file.
*
* We use the name of the shuffle data's shuffleBlockId with reduce ID set to 0 and add ".data"
* as the filename postfix for data file, and ".index" as the filename postfix for index file.
*
*/
// Note: Changes to the format in this file should be kept in sync with
// org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData().
private[spark] class IndexShuffleBlockResolver(
conf: SparkConf,
// var for testing
var _blockManager: BlockManager = null)
extends ShuffleBlockResolver
with Logging with MigratableResolver {
private lazy val blockManager = Option(_blockManager).getOrElse(SparkEnv.get.blockManager)
private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle")
private val remoteShuffleMaxDisk: Option[Long] =
conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE)
def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None)
/**
* Get the shuffle files that are stored locally. Used for block migrations.
*/
override def getStoredShuffles(): Seq[ShuffleBlockInfo] = {
val allBlocks = blockManager.diskBlockManager.getAllBlocks()
allBlocks.flatMap {
case ShuffleIndexBlockId(shuffleId, mapId, _) =>
Some(ShuffleBlockInfo(shuffleId, mapId))
case _ =>
None
}
}
private def getShuffleBytesStored(): Long = {
val shuffleFiles: Seq[File] = getStoredShuffles().map {
si => getDataFile(si.shuffleId, si.mapId)
}
shuffleFiles.map(_.length()).sum
}
/**
* Get the shuffle data file.
*
* When the dirs parameter is None then use the disk manager's local directories. Otherwise,
* read from the specified directories.
*/
def getDataFile(shuffleId: Int, mapId: Long, dirs: Option[Array[String]]): File = {
val blockId = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)
dirs
.map(ExecutorDiskUtils.getFile(_, blockManager.subDirsPerLocalDir, blockId.name))
.getOrElse(blockManager.diskBlockManager.getFile(blockId))
}
/**
* Get the shuffle index file.
*
* When the dirs parameter is None then use the disk manager's local directories. Otherwise,
* read from the specified directories.
*/
def getIndexFile(
shuffleId: Int,
mapId: Long,
dirs: Option[Array[String]] = None): File = {
val blockId = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)
dirs
.map(ExecutorDiskUtils.getFile(_, blockManager.subDirsPerLocalDir, blockId.name))
.getOrElse(blockManager.diskBlockManager.getFile(blockId))
}
private def getMergedBlockDataFile(
appId: String,
shuffleId: Int,
reduceId: Int,
dirs: Option[Array[String]] = None): File = {
blockManager.diskBlockManager.getMergedShuffleFile(
ShuffleMergedDataBlockId(appId, shuffleId, reduceId), dirs)
}
private def getMergedBlockIndexFile(
appId: String,
shuffleId: Int,
reduceId: Int,
dirs: Option[Array[String]] = None): File = {
blockManager.diskBlockManager.getMergedShuffleFile(
ShuffleMergedIndexBlockId(appId, shuffleId, reduceId), dirs)
}
private def getMergedBlockMetaFile(
appId: String,
shuffleId: Int,
reduceId: Int,
dirs: Option[Array[String]] = None): File = {
blockManager.diskBlockManager.getMergedShuffleFile(
ShuffleMergedMetaBlockId(appId, shuffleId, reduceId), dirs)
}
/**
* Remove data file and index file that contain the output data from one map.
*/
def removeDataByMap(shuffleId: Int, mapId: Long): Unit = {
var file = getDataFile(shuffleId, mapId)
if (file.exists()) {
if (!file.delete()) {
logWarning(s"Error deleting data ${file.getPath()}")
}
}
file = getIndexFile(shuffleId, mapId)
if (file.exists()) {
if (!file.delete()) {
logWarning(s"Error deleting index ${file.getPath()}")
}
}
}
/**
* Check whether the given index and data files match each other.
* If so, return the partition lengths in the data file. Otherwise return null.
*/
private def checkIndexAndDataFile(index: File, data: File, blocks: Int): Array[Long] = {
// the index file should have `block + 1` longs as offset.
if (index.length() != (blocks + 1) * 8L) {
return null
}
val lengths = new Array[Long](blocks)
// Read the lengths of blocks
val in = try {
new DataInputStream(new NioBufferedFileInputStream(index))
} catch {
case e: IOException =>
return null
}
try {
// Convert the offsets into lengths of each block
var offset = in.readLong()
if (offset != 0L) {
return null
}
var i = 0
while (i < blocks) {
val off = in.readLong()
lengths(i) = off - offset
offset = off
i += 1
}
} catch {
case e: IOException =>
return null
} finally {
in.close()
}
// the size of data file should match with index file
if (data.length() == lengths.sum) {
lengths
} else {
null
}
}
/**
* Write a provided shuffle block as a stream. Used for block migrations.
* ShuffleBlockBatchIds must contain the full range represented in the ShuffleIndexBlock.
* Requires the caller to delete any shuffle index blocks where the shuffle block fails to
* put.
*/
override def putShuffleBlockAsStream(blockId: BlockId, serializerManager: SerializerManager):
StreamCallbackWithID = {
// Throw an exception if we have exceeded maximum shuffle files stored
remoteShuffleMaxDisk.foreach { maxBytes =>
val bytesUsed = getShuffleBytesStored()
if (maxBytes < bytesUsed) {
throw new SparkException(s"Not storing remote shuffles $bytesUsed exceeds $maxBytes")
}
}
val file = blockId match {
case ShuffleIndexBlockId(shuffleId, mapId, _) =>
getIndexFile(shuffleId, mapId)
case ShuffleDataBlockId(shuffleId, mapId, _) =>
getDataFile(shuffleId, mapId)
case _ =>
throw new IllegalStateException(s"Unexpected shuffle block transfer ${blockId} as " +
s"${blockId.getClass().getSimpleName()}")
}
val fileTmp = Utils.tempFileWith(file)
val channel = Channels.newChannel(
serializerManager.wrapStream(blockId,
new FileOutputStream(fileTmp)))
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 shuffle block $blockId, now storing on local disk.")
channel.close()
val diskSize = fileTmp.length()
this.synchronized {
if (file.exists()) {
file.delete()
}
if (!fileTmp.renameTo(file)) {
throw new IOException(s"fail to rename file ${fileTmp} to ${file}")
}
}
blockManager.reportBlockStatus(blockId, BlockStatus(StorageLevel.DISK_ONLY, 0, diskSize))
}
override def onFailure(streamId: String, cause: Throwable): Unit = {
// the framework handles the connection itself, we just need to do local cleanup
logWarning(s"Error while uploading $blockId", cause)
channel.close()
fileTmp.delete()
}
}
}
/**
* Get the index & data block for migration.
*/
def getMigrationBlocks(shuffleBlockInfo: ShuffleBlockInfo): List[(BlockId, ManagedBuffer)] = {
try {
val shuffleId = shuffleBlockInfo.shuffleId
val mapId = shuffleBlockInfo.mapId
// Load the index block
val indexFile = getIndexFile(shuffleId, mapId)
val indexBlockId = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)
val indexFileSize = indexFile.length()
val indexBlockData = new FileSegmentManagedBuffer(
transportConf, indexFile, 0, indexFileSize)
// Load the data block
val dataFile = getDataFile(shuffleId, mapId)
val dataBlockId = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)
val dataBlockData = new FileSegmentManagedBuffer(
transportConf, dataFile, 0, dataFile.length())
// Make sure the index exist.
if (!indexFile.exists()) {
throw new FileNotFoundException("Index file is deleted already.")
}
if (dataFile.exists()) {
List((dataBlockId, dataBlockData), (indexBlockId, indexBlockData))
} else {
List((indexBlockId, indexBlockData))
}
} catch {
case _: Exception => // If we can't load the blocks ignore them.
logWarning(s"Failed to resolve shuffle block ${shuffleBlockInfo}. " +
"This is expected to occur if a block is removed after decommissioning has started.")
List.empty[(BlockId, ManagedBuffer)]
}
}
/**
* Write an index file with the offsets of each block, plus a final offset at the end for the
* end of the output file. This will be used by getBlockData to figure out where each block
* begins and ends.
*
* It will commit the data and index file as an atomic operation, use the existing ones, or
* replace them with new ones.
*
* Note: the `lengths` will be updated to match the existing index file if use the existing ones.
*/
def writeIndexFileAndCommit(
shuffleId: Int,
mapId: Long,
lengths: Array[Long],
dataTmp: File): Unit = {
val indexFile = getIndexFile(shuffleId, mapId)
val indexTmp = Utils.tempFileWith(indexFile)
try {
val dataFile = getDataFile(shuffleId, mapId)
// There is only one IndexShuffleBlockResolver per executor, this synchronization make sure
// the following check and rename are atomic.
this.synchronized {
val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length)
if (existingLengths != null) {
// Another attempt for the same task has already written our map outputs successfully,
// so just use the existing partition lengths and delete our temporary map outputs.
System.arraycopy(existingLengths, 0, lengths, 0, lengths.length)
if (dataTmp != null && dataTmp.exists()) {
dataTmp.delete()
}
} else {
// This is the first successful attempt in writing the map outputs for this task,
// so override any existing index and data files with the ones we wrote.
val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp)))
Utils.tryWithSafeFinally {
// We take in lengths of each block, need to convert it to offsets.
var offset = 0L
out.writeLong(offset)
for (length <- lengths) {
offset += length
out.writeLong(offset)
}
} {
out.close()
}
if (indexFile.exists()) {
indexFile.delete()
}
if (dataFile.exists()) {
dataFile.delete()
}
if (!indexTmp.renameTo(indexFile)) {
throw new IOException("fail to rename file " + indexTmp + " to " + indexFile)
}
if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) {
throw new IOException("fail to rename file " + dataTmp + " to " + dataFile)
}
}
}
} finally {
logDebug(s"Shuffle index for mapId $mapId: ${lengths.mkString("[", ",", "]")}")
if (indexTmp.exists() && !indexTmp.delete()) {
logError(s"Failed to delete temporary index file at ${indexTmp.getAbsolutePath}")
}
}
}
/**
* This is only used for reading local merged block data. In such cases, all chunks in the
* merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
* knows how to consume local merged shuffle file as multiple chunks.
*/
override def getMergedBlockData(
blockId: ShuffleBlockId,
dirs: Option[Array[String]]): Seq[ManagedBuffer] = {
val indexFile =
getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
// Load all the indexes in order to identify all chunks in the specified merged shuffle file.
val size = indexFile.length.toInt
val offsets = Utils.tryWithResource {
new DataInputStream(Files.newInputStream(indexFile.toPath))
} { dis =>
val buffer = ByteBuffer.allocate(size)
dis.readFully(buffer.array)
buffer.asLongBuffer
}
// Number of chunks is number of indexes - 1
val numChunks = size / 8 - 1
for (index <- 0 until numChunks) yield {
new FileSegmentManagedBuffer(transportConf, dataFile,
offsets.get(index),
offsets.get(index + 1) - offsets.get(index))
}
}
/**
* This is only used for reading local merged block meta data.
*/
override def getMergedBlockMeta(
blockId: ShuffleBlockId,
dirs: Option[Array[String]]): MergedBlockMeta = {
val indexFile =
getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
val size = indexFile.length.toInt
val numChunks = (size / 8) - 1
val metaFile = getMergedBlockMetaFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
val chunkBitMaps = new FileSegmentManagedBuffer(transportConf, metaFile, 0L, metaFile.length)
new MergedBlockMeta(numChunks, chunkBitMaps)
}
override def getBlockData(
blockId: BlockId,
dirs: Option[Array[String]]): ManagedBuffer = {
val (shuffleId, mapId, startReduceId, endReduceId) = blockId match {
case id: ShuffleBlockId =>
(id.shuffleId, id.mapId, id.reduceId, id.reduceId + 1)
case batchId: ShuffleBlockBatchId =>
(batchId.shuffleId, batchId.mapId, batchId.startReduceId, batchId.endReduceId)
case _ =>
throw new IllegalArgumentException("unexpected shuffle block id format: " + blockId)
}
// The block is actually going to be a range of a single map output file for this map, so
// find out the consolidated file, then the offset within that from our index
val indexFile = getIndexFile(shuffleId, mapId, dirs)
// SPARK-22982: if this FileInputStream's position is seeked forward by another piece of code
// which is incorrectly using our file descriptor then this code will fetch the wrong offsets
// (which may cause a reducer to be sent a different reducer's data). The explicit position
// checks added here were a useful debugging aid during SPARK-22982 and may help prevent this
// class of issue from re-occurring in the future which is why they are left here even though
// SPARK-22982 is fixed.
val channel = Files.newByteChannel(indexFile.toPath)
channel.position(startReduceId * 8L)
val in = new DataInputStream(Channels.newInputStream(channel))
try {
val startOffset = in.readLong()
channel.position(endReduceId * 8L)
val endOffset = in.readLong()
val actualPosition = channel.position()
val expectedPosition = endReduceId * 8L + 8
if (actualPosition != expectedPosition) {
throw new Exception(s"SPARK-22982: Incorrect channel position after index file reads: " +
s"expected $expectedPosition but actual position was $actualPosition.")
}
new FileSegmentManagedBuffer(
transportConf,
getDataFile(shuffleId, mapId, dirs),
startOffset,
endOffset - startOffset)
} finally {
in.close()
}
}
override def stop(): Unit = {}
}
private[spark] object IndexShuffleBlockResolver {
// No-op reduce ID used in interactions with disk store.
// The disk store currently expects puts to relate to a (map, reduce) pair, but in the sort
// shuffle outputs for several reduces are glommed into a single file.
val NOOP_REDUCE_ID = 0
}