Skip to content
Closed
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
[SPARK-37509][CORE] Improve Fallback Storage upload speed by avoiding…
… S3 rate limiter
  • Loading branch information
dongjoon-hyun committed Nov 30, 2021
commit 7cdcb447c4078825552a92e0e7833961f68eb9d7
16 changes: 11 additions & 5 deletions core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.{STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP, STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH}
import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcTimeout}
import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleBlockInfo}
import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID
Expand Down Expand Up @@ -60,15 +61,17 @@ private[storage] class FallbackStorage(conf: SparkConf) extends Logging {
val indexFile = r.getIndexFile(shuffleId, mapId)

if (indexFile.exists()) {
val hash = JavaUtils.nonNegativeHash(indexFile.getName)
fallbackFileSystem.copyFromLocalFile(
new Path(indexFile.getAbsolutePath),
new Path(fallbackPath, s"$appId/$shuffleId/${indexFile.getName}"))
new Path(fallbackPath, s"$appId/$shuffleId/$hash/${indexFile.getName}"))

val dataFile = r.getDataFile(shuffleId, mapId)
if (dataFile.exists()) {
val hash = JavaUtils.nonNegativeHash(dataFile.getName)
fallbackFileSystem.copyFromLocalFile(
new Path(dataFile.getAbsolutePath),
new Path(fallbackPath, s"$appId/$shuffleId/${dataFile.getName}"))
new Path(fallbackPath, s"$appId/$shuffleId/$hash/${dataFile.getName}"))
}

// Report block statuses
Expand All @@ -86,7 +89,8 @@ private[storage] class FallbackStorage(conf: SparkConf) extends Logging {
}

def exists(shuffleId: Int, filename: String): Boolean = {
fallbackFileSystem.exists(new Path(fallbackPath, s"$appId/$shuffleId/$filename"))
val hash = JavaUtils.nonNegativeHash(filename)
fallbackFileSystem.exists(new Path(fallbackPath, s"$appId/$shuffleId/$hash/$filename"))
}
}

Expand Down Expand Up @@ -168,7 +172,8 @@ private[spark] object FallbackStorage extends Logging {
}

val name = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID).name
val indexFile = new Path(fallbackPath, s"$appId/$shuffleId/$name")
val hash = JavaUtils.nonNegativeHash(name)
val indexFile = new Path(fallbackPath, s"$appId/$shuffleId/$hash/$name")
val start = startReduceId * 8L
val end = endReduceId * 8L
Utils.tryWithResource(fallbackFileSystem.open(indexFile)) { inputStream =>
Expand All @@ -178,7 +183,8 @@ private[spark] object FallbackStorage extends Logging {
index.skip(end - (start + 8L))
val nextOffset = index.readLong()
val name = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID).name
val dataFile = new Path(fallbackPath, s"$appId/$shuffleId/$name")
val hash = JavaUtils.nonNegativeHash(name)
val dataFile = new Path(fallbackPath, s"$appId/$shuffleId/$hash/$name")
val f = fallbackFileSystem.open(dataFile)
val size = nextOffset - offset
logDebug(s"To byte array $size")
Expand Down