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
[SPARK-44635][CORE] Handle shuffle fetch failures in decommissions #42296
Changes from 1 commit
dfdc906
e717582
6a2e34c
6b50554
3d424ab
22fb18e
8188981
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 | ||||
---|---|---|---|---|---|---|
|
@@ -39,7 +39,7 @@ import org.apache.spark.internal.config._ | |||||
import org.apache.spark.io.CompressionCodec | ||||||
import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} | ||||||
import org.apache.spark.scheduler.{MapStatus, MergeStatus, ShuffleOutputStatus} | ||||||
import org.apache.spark.shuffle.MetadataFetchFailedException | ||||||
import org.apache.spark.shuffle.{MetadataFetchFailedException, MetadataUpdateFailedException} | ||||||
import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId, ShuffleMergedBlockId} | ||||||
import org.apache.spark.util._ | ||||||
import org.apache.spark.util.collection.OpenHashMap | ||||||
|
@@ -1288,6 +1288,30 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr | |||||
mapSizesByExecutorId.iter | ||||||
} | ||||||
|
||||||
def getMapOutputLocationWithRefresh( | ||||||
shuffleId: Int, | ||||||
mapId: Long, | ||||||
prevLocation: BlockManagerId): BlockManagerId = { | ||||||
// Try to get the cached location first in case other concurrent tasks | ||||||
// fetched the fresh location already | ||||||
var currentLocationOpt = getMapOutputLocation(shuffleId, mapId) | ||||||
if (currentLocationOpt.isDefined && currentLocationOpt.get == prevLocation) { | ||||||
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:
Suggested change
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. Will change to |
||||||
// Address in the cache unchanged. Try to clean cache and get a fresh location | ||||||
unregisterShuffle(shuffleId) | ||||||
currentLocationOpt = getMapOutputLocation(shuffleId, mapId) | ||||||
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. Note: we end up removing both map and merge status here - for this call second call, pass 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. Good catch. Will do. |
||||||
} | ||||||
if (currentLocationOpt.isEmpty) { | ||||||
throw new MetadataUpdateFailedException(shuffleId, mapId, | ||||||
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. Could you reuse |
||||||
message = s"Failed to get map output location for shuffleId $shuffleId, mapId $mapId") | ||||||
} | ||||||
currentLocationOpt.get | ||||||
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: 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 shuffle fallback storage is enabled, this 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. Let us filter it out here, and add support for fetching from fallback in a separate pr. +CC @dongjoon-hyun as well. |
||||||
} | ||||||
|
||||||
private def getMapOutputLocation(shuffleId: Int, mapId: Long): Option[BlockManagerId] = { | ||||||
val (mapOutputStatuses, _) = getStatuses(shuffleId, conf, false) | ||||||
mapOutputStatuses.filter(_ != null).find(_.mapId == mapId).map(_.location) | ||||||
} | ||||||
|
||||||
override def getPushBasedShuffleMapSizesByExecutorId( | ||||||
shuffleId: Int, | ||||||
startMapIndex: Int, | ||||||
|
Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
@@ -491,6 +491,27 @@ private[spark] object TestUtils { | |||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
file.getPath | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
def withConf[T](confPairs: (String, String)*)(f: => T): T = { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
bozhang2820 marked this conversation as resolved.
Show resolved
Hide resolved
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
val conf = SparkEnv.get.conf | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
val (keys, values) = confPairs.unzip | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
val currentValues = keys.map { key => | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
if (conf.contains(key)) { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
Some(conf.get(key)) | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} else { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
None | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
(keys, values).zipped.foreach { (key, value) => | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
conf.set(key, value) | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
try f finally { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
keys.zip(currentValues).foreach { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
case (key, Some(value)) => conf.set(key, value) | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
case (key, None) => conf.remove(key) | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
Comment on lines
+497
to
+515
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:
Suggested change
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -528,6 +528,15 @@ package object config { | |
.bytesConf(ByteUnit.BYTE) | ||
.createOptional | ||
|
||
private[spark] val STORAGE_DECOMMISSION_SHUFFLE_REFRESH = | ||
ConfigBuilder("spark.storage.decommission.shuffleBlocks.refreshLocationsEnabled") | ||
.doc("If true, executors will try to refresh the cached locations for the shuffle blocks" + | ||
"when fetch failures happens (and decommission shuffle block migration is enabled), " + | ||
"and retry fetching when the location changes.") | ||
.version("3.5.0") | ||
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. Change to |
||
.booleanConf | ||
.createWithDefault(false) | ||
|
||
private[spark] val STORAGE_REPLICATION_TOPOLOGY_FILE = | ||
ConfigBuilder("spark.storage.replication.topologyFile") | ||
.version("2.1.0") | ||
|
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -33,10 +33,10 @@ import io.netty.util.internal.OutOfDirectMemoryError | |||||
import org.apache.commons.io.IOUtils | ||||||
import org.roaringbitmap.RoaringBitmap | ||||||
|
||||||
import org.apache.spark.{MapOutputTracker, SparkException, TaskContext} | ||||||
import org.apache.spark.{MapOutputTracker, MapOutputTrackerWorker, SparkEnv, SparkException, TaskContext} | ||||||
import org.apache.spark.MapOutputTracker.SHUFFLE_PUSH_MAP_ID | ||||||
import org.apache.spark.errors.SparkCoreErrors | ||||||
import org.apache.spark.internal.Logging | ||||||
import org.apache.spark.internal.{config, Logging} | ||||||
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} | ||||||
import org.apache.spark.network.shuffle._ | ||||||
import org.apache.spark.network.shuffle.checksum.{Cause, ShuffleChecksumHelper} | ||||||
|
@@ -111,6 +111,14 @@ final class ShuffleBlockFetcherIterator( | |||||
// nodes, rather than blocking on reading output from one node. | ||||||
private val targetRemoteRequestSize = math.max(maxBytesInFlight / 5, 1L) | ||||||
|
||||||
private val isShuffleMigrationEnabled = | ||||||
SparkEnv.get.conf.get(config.DECOMMISSION_ENABLED) && | ||||||
SparkEnv.get.conf.get(config.STORAGE_DECOMMISSION_ENABLED) && | ||||||
SparkEnv.get.conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED) | ||||||
|
||||||
private val shouldPerformShuffleLocationRefresh = | ||||||
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. what about make this one of the constructor argument? One of the benefit is that you don't need to write tests with |
||||||
isShuffleMigrationEnabled && SparkEnv.get.conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_REFRESH) | ||||||
|
||||||
/** | ||||||
* Total number of blocks to fetch. | ||||||
*/ | ||||||
|
@@ -264,18 +272,22 @@ final class ShuffleBlockFetcherIterator( | |||||
case FetchBlockInfo(blockId, size, mapIndex) => (blockId.toString, (size, mapIndex)) | ||||||
}.toMap | ||||||
val remainingBlocks = new HashSet[String]() ++= infoMap.keys | ||||||
val deferredBlocks = new ArrayBuffer[String]() | ||||||
val deferredBlocks = new HashMap[BlockManagerId, Queue[String]]() | ||||||
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:
Suggested change
|
||||||
val blockIds = req.blocks.map(_.blockId.toString) | ||||||
val address = req.address | ||||||
val requestStartTime = clock.nanoTime() | ||||||
|
||||||
@inline def enqueueDeferredFetchRequestIfNecessary(): Unit = { | ||||||
if (remainingBlocks.isEmpty && deferredBlocks.nonEmpty) { | ||||||
val blocks = deferredBlocks.map { blockId => | ||||||
val (size, mapIndex) = infoMap(blockId) | ||||||
FetchBlockInfo(BlockId(blockId), size, mapIndex) | ||||||
val newAddressToBlocks = new HashMap[BlockManagerId, Queue[FetchBlockInfo]]() | ||||||
deferredBlocks.foreach { case (blockManagerId, blockIds) => | ||||||
val blocks = blockIds.map { blockId => | ||||||
val (size, mapIndex) = infoMap(blockId) | ||||||
FetchBlockInfo(BlockId(blockId), size, mapIndex) | ||||||
} | ||||||
newAddressToBlocks.put(blockManagerId, blocks) | ||||||
} | ||||||
mridulm marked this conversation as resolved.
Show resolved
Hide resolved
|
||||||
results.put(DeferFetchRequestResult(FetchRequest(address, blocks))) | ||||||
results.put(DeferFetchRequestResult(address, newAddressToBlocks)) | ||||||
deferredBlocks.clear() | ||||||
} | ||||||
} | ||||||
|
@@ -344,7 +356,8 @@ final class ShuffleBlockFetcherIterator( | |||||
s"due to Netty OOM, will retry") | ||||||
} | ||||||
remainingBlocks -= blockId | ||||||
deferredBlocks += blockId | ||||||
deferredBlocks.getOrElseUpdate(address, new Queue[String]()) | ||||||
.enqueue(blockId) | ||||||
enqueueDeferredFetchRequestIfNecessary() | ||||||
} | ||||||
|
||||||
|
@@ -355,8 +368,23 @@ final class ShuffleBlockFetcherIterator( | |||||
updateMergedReqsDuration(wasReqForMergedChunks = true) | ||||||
results.put(FallbackOnPushMergedFailureResult( | ||||||
block, address, infoMap(blockId)._1, remainingBlocks.isEmpty)) | ||||||
} else { | ||||||
} else if (!shouldPerformShuffleLocationRefresh) { | ||||||
results.put(FailureFetchResult(block, infoMap(blockId)._2, address, e)) | ||||||
} else { | ||||||
val (shuffleId, mapId) = BlockId.getShuffleIdAndMapId(block) | ||||||
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. Should we move the Something like:
|
||||||
val mapOutputTrackerWorker = mapOutputTracker.asInstanceOf[MapOutputTrackerWorker] | ||||||
val currentAddress = mapOutputTrackerWorker | ||||||
.getMapOutputLocationWithRefresh(shuffleId, mapId, address) | ||||||
if (currentAddress != address) { | ||||||
logInfo(s"Map status location for block $blockId changed from $address " + | ||||||
s"to $currentAddress") | ||||||
remainingBlocks -= blockId | ||||||
deferredBlocks.getOrElseUpdate(currentAddress, new Queue[String]()) | ||||||
.enqueue(blockId) | ||||||
enqueueDeferredFetchRequestIfNecessary() | ||||||
} else { | ||||||
results.put(FailureFetchResult(block, infoMap(blockId)._2, address, e)) | ||||||
} | ||||||
} | ||||||
} | ||||||
} | ||||||
|
@@ -970,15 +998,16 @@ final class ShuffleBlockFetcherIterator( | |||||
} | ||||||
throwFetchFailedException(blockId, mapIndex, address, e, Some(errorMsg)) | ||||||
|
||||||
case DeferFetchRequestResult(request) => | ||||||
val address = request.address | ||||||
numBlocksInFlightPerAddress(address) -= request.blocks.size | ||||||
bytesInFlight -= request.size | ||||||
case DeferFetchRequestResult(failedAddress, newAddressToBlocks) => | ||||||
numBlocksInFlightPerAddress(failedAddress) -= newAddressToBlocks.values.map(_.size).sum | ||||||
bytesInFlight -= newAddressToBlocks.values.map(_.map(_.size).sum).sum | ||||||
reqsInFlight -= 1 | ||||||
logDebug("Number of requests in flight " + reqsInFlight) | ||||||
val defReqQueue = | ||||||
deferredFetchRequests.getOrElseUpdate(address, new Queue[FetchRequest]()) | ||||||
defReqQueue.enqueue(request) | ||||||
newAddressToBlocks.foreach { case (newAddress, blocks) => | ||||||
val defReqQueue = | ||||||
deferredFetchRequests.getOrElseUpdate(newAddress, new Queue[FetchRequest]()) | ||||||
defReqQueue.enqueue(FetchRequest(newAddress, blocks)) | ||||||
} | ||||||
result = null | ||||||
|
||||||
case FallbackOnPushMergedFailureResult(blockId, address, size, isNetworkReqDone) => | ||||||
|
@@ -1580,7 +1609,9 @@ object ShuffleBlockFetcherIterator { | |||||
* Result of a fetch request that should be deferred for some reasons, e.g., Netty OOM | ||||||
bozhang2820 marked this conversation as resolved.
Show resolved
Hide resolved
|
||||||
*/ | ||||||
private[storage] | ||||||
case class DeferFetchRequestResult(fetchRequest: FetchRequest) extends FetchResult | ||||||
case class DeferFetchRequestResult( | ||||||
failedAddress: BlockManagerId, | ||||||
newAddressToBlocks: HashMap[BlockManagerId, Queue[FetchBlockInfo]]) extends FetchResult | ||||||
|
||||||
/** | ||||||
* Result of an un-successful fetch of either of these: | ||||||
|
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.
Maybe return
Option[BlockManagerId]
?: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.
We still want to throw a
MetadataFetchFailedException
when failing to get a refreshed location here. So I would prefer returning aBlockManagerId
and make it specific.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.
We can do the following with
Option
:It is also consistent with other function signatures with
getMapOutputLocation
.