-
Notifications
You must be signed in to change notification settings - Fork 28.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[WIP][SPARK-20629][CORE] Copy shuffle data when nodes are being shutdown #28331
Changes from 31 commits
4126c1b
8ee8949
afb1b1a
4071ae2
ff620ba
adb03db
be2a5e7
783114b
dbe2418
a240f98
ef8fcc5
838a346
e85c8ef
2da0f2d
9d31746
38ff8be
13ec43a
a92025c
fe265d7
70c3871
069dd3b
6340f9b
4cb0458
4cfeb8e
e81aa5a
841d443
ba20ec0
17a6a3f
155aeb2
7f93df6
7e32341
a3aa8eb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -57,7 +57,7 @@ import org.apache.spark.resource._ | |
import org.apache.spark.resource.ResourceUtils._ | ||
import org.apache.spark.rpc.RpcEndpointRef | ||
import org.apache.spark.scheduler._ | ||
import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend | ||
import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} | ||
import org.apache.spark.scheduler.local.LocalSchedulerBackend | ||
import org.apache.spark.shuffle.ShuffleDataIOUtils | ||
import org.apache.spark.shuffle.api.ShuffleDriverComponents | ||
|
@@ -1586,7 +1586,12 @@ class SparkContext(config: SparkConf) extends Logging { | |
listenerBus.removeListener(listener) | ||
} | ||
|
||
private[spark] def getExecutorIds(): Seq[String] = { | ||
/** | ||
* :: DeveloperApi :: | ||
* Deregister the listener from Spark's listener bus. | ||
*/ | ||
@DeveloperApi | ||
def getExecutorIds(): Seq[String] = { | ||
schedulerBackend match { | ||
case b: ExecutorAllocationClient => | ||
b.getExecutorIds() | ||
|
@@ -1725,6 +1730,17 @@ class SparkContext(config: SparkConf) extends Logging { | |
} | ||
} | ||
|
||
|
||
@DeveloperApi | ||
def decommissionExecutors(executorIds: Seq[String]): Unit = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think private[spark] as well |
||
schedulerBackend match { | ||
case b: CoarseGrainedSchedulerBackend => | ||
executorIds.foreach(b.decommissionExecutor) | ||
case _ => | ||
logWarning("Decommissioning executors is not supported by current scheduler.") | ||
} | ||
} | ||
|
||
/** The version of Spark on which this application is running. */ | ||
def version: String = SPARK_VERSION | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,15 +18,18 @@ | |
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} | ||
import org.apache.spark.internal.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 | ||
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 | ||
|
@@ -46,7 +49,7 @@ private[spark] class IndexShuffleBlockResolver( | |
conf: SparkConf, | ||
_blockManager: BlockManager = null) | ||
extends ShuffleBlockResolver | ||
with Logging { | ||
with Logging with MigratableResolver { | ||
|
||
private lazy val blockManager = Option(_blockManager).getOrElse(SparkEnv.get.blockManager) | ||
|
||
|
@@ -55,6 +58,25 @@ private[spark] class IndexShuffleBlockResolver( | |
|
||
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(): Set[(Int, Long)] = { | ||
// Matches ShuffleIndexBlockId name | ||
val pattern = "shuffle_(\\d+)_(\\d+)_.+\\.index".r | ||
val rootDirs = blockManager.diskBlockManager.localDirs | ||
// ExecutorDiskUtil puts things inside one level hashed sub directories | ||
val searchDirs = rootDirs.flatMap(_.listFiles()).filter(_.isDirectory()) ++ rootDirs | ||
val filenames = searchDirs.flatMap(_.list()) | ||
logDebug(s"Got block files ${filenames.toList}") | ||
filenames.flatMap{ fname => | ||
pattern.findAllIn(fname).matchData.map { | ||
matched => (matched.group(1).toInt, matched.group(2).toLong) | ||
} | ||
}.toSet | ||
} | ||
|
||
|
||
/** | ||
* Get the shuffle data file. | ||
* | ||
|
@@ -148,6 +170,86 @@ private[spark] class IndexShuffleBlockResolver( | |
} | ||
} | ||
|
||
/** | ||
* 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 = { | ||
val file = blockId match { | ||
case ShuffleIndexBlockId(shuffleId, mapId, _) => | ||
getIndexFile(shuffleId, mapId) | ||
case ShuffleDataBlockId(shuffleId, mapId, _) => | ||
getDataFile(shuffleId, mapId) | ||
case _ => | ||
throw new Exception(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 block $blockId, now putting into local shuffle service") | ||
channel.close() | ||
val diskSize = fileTmp.length() | ||
this.synchronized { | ||
if (file.exists()) { | ||
file.delete() | ||
} | ||
Comment on lines
+210
to
+212
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. When the file exists, does it mean there is index/data file with same shuffle id and map id? When it could happen? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I suppose this should never happen, I'm not sure though let me do some thinking on that. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So this mirrors the logic inside of writeIndexFileAndCommit, the matching check there was introduced in SPARK-17547 |
||
if (!fileTmp.renameTo(file)) { | ||
throw new IOException(s"fail to rename file ${fileTmp} to ${file}") | ||
} | ||
} | ||
blockManager.reportBlockStatus(blockId, BlockStatus( | ||
StorageLevel( | ||
useDisk = true, | ||
useMemory = false, | ||
useOffHeap = false, | ||
deserialized = false, | ||
replication = 0) | ||
, 0, diskSize)) | ||
} | ||
|
||
override def onFailure(streamId: String, cause: Throwable): Unit = { | ||
// the framework handles the connection itself, we just need to do local cleanup | ||
channel.close() | ||
fileTmp.delete() | ||
} | ||
} | ||
} | ||
|
||
/** | ||
* Get the index & data block for migration. | ||
*/ | ||
def getMigrationBlocks(shuffleId: Int, mapId: Long): List[(BlockId, ManagedBuffer)] = { | ||
// 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()) | ||
List((indexBlockId, indexBlockData), (dataBlockId, dataBlockData)) | ||
} | ||
|
||
|
||
/** | ||
* 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 | ||
|
@@ -169,7 +271,7 @@ private[spark] class IndexShuffleBlockResolver( | |
val dataFile = getDataFile(shuffleId, mapId) | ||
// There is only one IndexShuffleBlockResolver per executor, this synchronization make sure | ||
// the following check and rename are atomic. | ||
synchronized { | ||
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, | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'll move this back to private[spark] (also the comment I have seems wrong so revert the comment)