-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-32919][SHUFFLE][test-maven][test-hadoop2.7] Driver side changes for coordinating push based shuffle by selecting external shuffle services for merging partitions #30164
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
Changes from 12 commits
68fc364
2688df2
0423970
eb93fe1
f2f61e2
3a6219f
ca44d03
2172f65
1e76824
3e03109
cbb66eb
a1c8831
047ad0c
2d6d266
2b0c073
9ba4dfb
5127d8b
e320ac0
a2d85ef
affa8a0
46f5670
2467a61
050a5ae
1714829
1ba7668
5ce2934
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 |
|---|---|---|
|
|
@@ -74,6 +74,14 @@ class BlockManagerMasterEndpoint( | |
| // Mapping from block id to the set of block managers that have the block. | ||
| private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] | ||
|
|
||
| // Mapping from host name to shuffle (mergers) services where the current app | ||
| // registered an executor in the past. Older hosts are removed when the | ||
| // maxRetainedMergerLocations size is reached in favor of newer locations. | ||
| private val shuffleMergerLocations = new mutable.LinkedHashMap[String, BlockManagerId]() | ||
venkata91 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| // Maximum number of merger locations to cache | ||
| private val maxRetainedMergerLocations = conf.get(config.SHUFFLE_MERGERS_MAX_RETAINED_LOCATIONS) | ||
|
|
||
| private val askThreadPool = | ||
| ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool", 100) | ||
| private implicit val askExecutionContext = ExecutionContext.fromExecutorService(askThreadPool) | ||
|
|
@@ -139,6 +147,9 @@ class BlockManagerMasterEndpoint( | |
| case GetBlockStatus(blockId, askStorageEndpoints) => | ||
| context.reply(blockStatus(blockId, askStorageEndpoints)) | ||
|
|
||
| case GetShufflePushMergerLocations(numMergersNeeded, hostsToFilter) => | ||
| context.reply(getShufflePushMergerLocations(numMergersNeeded, hostsToFilter)) | ||
|
|
||
| case IsExecutorAlive(executorId) => | ||
| context.reply(blockManagerIdByExecutor.contains(executorId)) | ||
|
|
||
|
|
@@ -360,6 +371,17 @@ class BlockManagerMasterEndpoint( | |
|
|
||
| } | ||
|
|
||
| private def addMergerLocation(blockManagerId: BlockManagerId): Unit = { | ||
| if (!blockManagerId.isDriver && !shuffleMergerLocations.contains(blockManagerId.host)) { | ||
| val shuffleServerId = BlockManagerId(blockManagerId.executorId, blockManagerId.host, | ||
| StorageUtils.externalShuffleServicePort(conf)) | ||
| if (shuffleMergerLocations.size >= maxRetainedMergerLocations) { | ||
| shuffleMergerLocations -= shuffleMergerLocations.head._1 | ||
venkata91 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
| shuffleMergerLocations(shuffleServerId.host) = shuffleServerId | ||
venkata91 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
| } | ||
|
|
||
| private def removeExecutor(execId: String): Unit = { | ||
| logInfo("Trying to remove executor " + execId + " from BlockManagerMaster.") | ||
| blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) | ||
|
|
@@ -526,6 +548,8 @@ class BlockManagerMasterEndpoint( | |
|
|
||
| blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), | ||
| maxOnHeapMemSize, maxOffHeapMemSize, storageEndpoint, externalShuffleServiceBlockStatus) | ||
|
|
||
| addMergerLocation(id) | ||
|
||
| } | ||
| listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxOnHeapMemSize + maxOffHeapMemSize, | ||
| Some(maxOnHeapMemSize), Some(maxOffHeapMemSize))) | ||
|
|
@@ -657,6 +681,31 @@ class BlockManagerMasterEndpoint( | |
| } | ||
| } | ||
|
|
||
| private def getShufflePushMergerLocations( | ||
| numMergersNeeded: Int, | ||
| hostsToFilter: Set[String]): Seq[BlockManagerId] = { | ||
| val blockManagersWithExecutors = blockManagerIdByExecutor.groupBy(_._2.host) | ||
| .mapValues(_.head).values.map(_._2).toSet | ||
| val filteredBlockManagersWithExecutors = blockManagersWithExecutors | ||
| .filterNot(x => hostsToFilter.contains(x.host)) | ||
| val filteredMergersWithExecutors = filteredBlockManagersWithExecutors.map( | ||
| x => BlockManagerId(x.executorId, x.host, StorageUtils.externalShuffleServicePort(conf))) | ||
venkata91 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| // Enough mergers are available as part of active executors list | ||
| if (filteredMergersWithExecutors.size >= numMergersNeeded) { | ||
| filteredMergersWithExecutors.toSeq | ||
| } else { | ||
| // Delta mergers added from inactive mergers list to the active mergers list | ||
| val filteredMergersWithExecutorsHosts = filteredMergersWithExecutors.map(_.host) | ||
venkata91 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| val filteredMergersWithoutExecutors = shuffleMergerLocations.values | ||
| .filterNot( | ||
| mergerHost => filteredMergersWithExecutorsHosts.contains(mergerHost.host)) | ||
venkata91 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| filteredMergersWithExecutors.toSeq ++ | ||
| filteredMergersWithoutExecutors.toSeq | ||
| .take(numMergersNeeded - filteredMergersWithExecutors.size) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Returns an [[RpcEndpointRef]] of the [[BlockManagerReplicaEndpoint]] for sending RPC messages. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,7 +28,7 @@ import java.nio.channels.{Channels, FileChannel, WritableByteChannel} | |
| import java.nio.charset.StandardCharsets | ||
| import java.nio.file.Files | ||
| import java.security.SecureRandom | ||
| import java.util.{Arrays, Locale, Properties, Random, UUID} | ||
| import java.util.{Locale, Properties, Random, UUID} | ||
| import java.util.concurrent._ | ||
| import java.util.concurrent.TimeUnit.NANOSECONDS | ||
| import java.util.zip.GZIPInputStream | ||
|
|
@@ -2541,6 +2541,15 @@ private[spark] object Utils extends Logging { | |
| master == "local" || master.startsWith("local[") | ||
| } | ||
|
|
||
| /** | ||
| * Push based shuffle can only be enabled when external shuffle service is enabled. | ||
| * In the initial version, we cannot support pushed based shuffle and adaptive execution | ||
| * at the same time. Will improve this in a later version. | ||
venkata91 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| */ | ||
| def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = { | ||
| conf.get(PUSH_BASED_SHUFFLE_ENABLED) && conf.get(SHUFFLE_SERVICE_ENABLED) | ||
|
||
| } | ||
|
|
||
| /** | ||
| * Return whether dynamic allocation is enabled in the given conf. | ||
| */ | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.