-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-6521][Core]executors in the same node read local shuffle file #5178
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
Closed
Closed
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -52,12 +52,16 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { | |
| ShuffleBlockId(shuffleId, mapId, 0) | ||
| } | ||
|
|
||
| def getDataFile(shuffleId: Int, mapId: Int): File = { | ||
| blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, 0)) | ||
| def getDataFile(shuffleId: Int, | ||
| mapId: Int, | ||
| blockManagerId: BlockManagerId = blockManager.blockManagerId): File = { | ||
|
Contributor
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. hey @viper-kun the style here and other places should be: |
||
| blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, 0), blockManagerId) | ||
| } | ||
|
|
||
| private def getIndexFile(shuffleId: Int, mapId: Int): File = { | ||
| blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, 0)) | ||
| private def getIndexFile(shuffleId: Int, | ||
| mapId: Int, | ||
| blockManagerId: BlockManagerId = blockManager.blockManagerId): File = { | ||
| blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, 0), blockManagerId) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -101,10 +105,11 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { | |
| Some(getBlockData(blockId).nioByteBuffer()) | ||
| } | ||
|
|
||
| override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { | ||
| override def getBlockData(blockId: ShuffleBlockId, | ||
| blockManagerId: BlockManagerId = blockManager.blockManagerId): ManagedBuffer = { | ||
| // 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(blockId.shuffleId, blockId.mapId) | ||
| val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId, blockManagerId) | ||
|
|
||
| val in = new DataInputStream(new FileInputStream(indexFile)) | ||
| try { | ||
|
|
@@ -113,7 +118,7 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { | |
| val nextOffset = in.readLong() | ||
| new FileSegmentManagedBuffer( | ||
| transportConf, | ||
| getDataFile(blockId.shuffleId, blockId.mapId), | ||
| getDataFile(blockId.shuffleId, blockId.mapId, blockManagerId), | ||
| offset, | ||
| nextOffset - offset) | ||
| } finally { | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -53,8 +53,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus | |
| private val akkaTimeout = AkkaUtils.askTimeout(conf) | ||
|
|
||
| override def receiveWithLogging: PartialFunction[Any, Unit] = { | ||
| case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => | ||
| register(blockManagerId, maxMemSize, slaveActor) | ||
| case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor, localDirsPath) => | ||
| register(blockManagerId, maxMemSize, slaveActor, localDirsPath) | ||
| sender ! true | ||
|
|
||
| case UpdateBlockInfo( | ||
|
|
@@ -77,6 +77,9 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus | |
| case GetMemoryStatus => | ||
| sender ! memoryStatus | ||
|
|
||
| case GetLocalDirsPath(blockManagerId) => | ||
| sender ! getLocalDirsPath(blockManagerId) | ||
|
|
||
| case GetStorageStatus => | ||
| sender ! storageStatus | ||
|
|
||
|
|
@@ -223,6 +226,15 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus | |
| } | ||
| } | ||
|
|
||
| // Return local dirs of other blockmanager on the same machine as blockManagerId | ||
| private def getLocalDirsPath( | ||
| blockManagerId: BlockManagerId): Map[BlockManagerId, Array[String]] = { | ||
| blockManagerInfo | ||
| .filter { case(id, _) => (id != blockManagerId && id.host == blockManagerId.host)} | ||
|
Member
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. Nit: Unnecessary parentheses. |
||
| .mapValues { info => info.localDirsPath } | ||
| .toMap | ||
| } | ||
|
|
||
| // Return a map from the block manager id to max memory and remaining memory. | ||
| private def memoryStatus: Map[BlockManagerId, (Long, Long)] = { | ||
| blockManagerInfo.map { case(blockManagerId, info) => | ||
|
|
@@ -291,7 +303,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus | |
| ).map(_.flatten.toSeq) | ||
| } | ||
|
|
||
| private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { | ||
| private def register( | ||
| id: BlockManagerId, | ||
| maxMemSize: Long, | ||
| slaveActor: ActorRef, localDirsPath: Array[String]) { | ||
| val time = System.currentTimeMillis() | ||
| if (!blockManagerInfo.contains(id)) { | ||
| blockManagerIdByExecutor.get(id.executorId) match { | ||
|
|
@@ -308,7 +323,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus | |
| blockManagerIdByExecutor(id.executorId) = id | ||
|
|
||
| blockManagerInfo(id) = new BlockManagerInfo( | ||
| id, System.currentTimeMillis(), maxMemSize, slaveActor) | ||
| id, System.currentTimeMillis(), maxMemSize, slaveActor, localDirsPath) | ||
| } | ||
| listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxMemSize)) | ||
| } | ||
|
|
@@ -320,7 +335,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus | |
| memSize: Long, | ||
| diskSize: Long, | ||
| tachyonSize: Long): Boolean = { | ||
|
|
||
| if (!blockManagerInfo.contains(blockManagerId)) { | ||
| if (blockManagerId.isDriver && !isLocal) { | ||
| // We intentionally do not register the master (except in local mode), | ||
|
|
@@ -412,7 +426,8 @@ private[spark] class BlockManagerInfo( | |
| val blockManagerId: BlockManagerId, | ||
| timeMs: Long, | ||
| val maxMem: Long, | ||
| val slaveActor: ActorRef) | ||
| val slaveActor: ActorRef, | ||
| val localDirsPath: Array[String]) | ||
| extends Logging { | ||
|
|
||
| private var _lastSeenMs: Long = timeMs | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Why this patch only supports SortShuffleManager?
ISTM the same logic could be straightforwardly implemented in HashShuffleManager.
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.
It not supports HashShuffle with consolidateShuffleFiles. For not confuse,it only support SortShuffleManager.