Skip to content

[SPARK-52506][CORE] Allow migrating to fallback storage only #51201

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

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -617,7 +617,8 @@ package object config {
"shuffle blocks. Rejecting remote shuffle blocks means that an executor will not receive " +
"any shuffle migrations, and if there are no other executors available for migration " +
"then shuffle blocks will be lost unless " +
s"${STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH.key} is configured.")
s"${STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH.key} is configured. " +
s"Set to 0 to migrate to fallback storage only.")
.version("3.2.0")
.bytesConf(ByteUnit.BYTE)
.createOptional
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import org.apache.spark._
import org.apache.spark.errors.SparkCoreErrors
import org.apache.spark.internal.{config, Logging, MDC}
import org.apache.spark.internal.LogKeys._
import org.apache.spark.internal.config.STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE
import org.apache.spark.shuffle.ShuffleBlockInfo
import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock
import org.apache.spark.util.{ThreadUtils, Utils}
Expand Down Expand Up @@ -319,8 +320,15 @@ private[storage] class BlockManagerDecommissioner(
log"${MDC(TOTAL, localShuffles.size)} local shuffles are added. " +
log"In total, ${MDC(NUM_REMAINED, remainedShuffles)} shuffles are remained.")

// migrate to fallback storage only if
// STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH is set and
// STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE is 0
val fallbackOnly = conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined &&
conf.get(STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE).contains(0)

// Update the threads doing migrations
val livePeerSet = bm.getPeers(false).toSet
val livePeerSet = if (fallbackOnly) Set(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
else bm.getPeers(false).toSet
val currentPeerSet = migrationPeers.keys.toSet
val deadPeers = currentPeerSet.diff(livePeerSet)
// Randomize the orders of the peers to avoid hotspot nodes.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,7 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
val conf = new SparkConf(false)
.set("spark.app.id", "testId")
.set(STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, true)
.set(STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE, 0L) // migrate to fallback storage only
.set(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH,
Files.createTempDirectory("tmp").toFile.getAbsolutePath + "/")

Expand All @@ -200,8 +201,6 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
when(resolver.getDataFile(shuffleId, mapId)).thenReturn(dataFile)
}

when(bm.getPeers(mc.any()))
.thenReturn(Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID))
val bmm = new BlockManagerMaster(new NoopRpcEndpointRef(conf), null, conf, false)
when(bm.master).thenReturn(bmm)
val blockTransferService = mock(classOf[BlockTransferService])
Expand Down