Skip to content
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-33916][CORE] Fix fallback storage offset and improve compression codec test coverage #30934

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -158,7 +158,7 @@ object FallbackStorage extends Logging {
val name = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID).name
val dataFile = new Path(fallbackPath, s"$appId/$shuffleId/$name")
val f = fallbackFileSystem.open(dataFile)
val size = nextOffset - 1 - offset
val size = nextOffset - offset
logDebug(s"To byte array $size")
Copy link
Contributor

@mridulm mridulm Dec 27, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Given this bug, I am wondering if we want to refactor IndexShuffleBlockResolver.read such that we can reuse it here as well ?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe, for Apache Spark 3.2.0, @mridulm ? Currently, this PR is aiming to provide this fix for Apache Spark 3.1.0 RC before next Monday.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good, thanks @dongjoon-hyun !

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you, @mridulm !

val array = new Array[Byte](size.toInt)
val startTimeNs = System.nanoTime()
Expand Down
Expand Up @@ -59,6 +59,7 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
test("fallback storage APIs - copy/exists") {
val conf = new SparkConf(false)
.set("spark.app.id", "testId")
.set(SHUFFLE_COMPRESS, false)
.set(STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, true)
.set(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH,
Files.createTempDirectory("tmp").toFile.getAbsolutePath + "/")
Expand Down Expand Up @@ -227,43 +228,45 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
}
}

test("Newly added executors should access old data from remote storage") {
sc = new SparkContext(getSparkConf(2, 0))
withSpark(sc) { sc =>
TestUtils.waitUntilExecutorsUp(sc, 2, 60000)
val rdd1 = sc.parallelize(1 to 10, 2)
val rdd2 = rdd1.map(x => (x % 2, 1))
val rdd3 = rdd2.reduceByKey(_ + _)
assert(rdd3.collect() === Array((0, 5), (1, 5)))
Seq("lz4", "snappy", "zstd").foreach { codec =>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The doc of IO_COMPRESSION_CODEC says that it supports lz4, lzf, snappy, and zstd. Should we test lzf too?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I'll add that, @MaxGekk .

test(s"$codec - Newly added executors should access old data from remote storage") {
sc = new SparkContext(getSparkConf(2, 0).set(IO_COMPRESSION_CODEC, codec))
withSpark(sc) { sc =>
TestUtils.waitUntilExecutorsUp(sc, 2, 60000)
val rdd1 = sc.parallelize(1 to 10, 2)
val rdd2 = rdd1.map(x => (x % 2, 1))
val rdd3 = rdd2.reduceByKey(_ + _)
assert(rdd3.collect() === Array((0, 5), (1, 5)))

// Decommission all
val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend]
sc.getExecutorIds().foreach {
sched.decommissionExecutor(_, ExecutorDecommissionInfo(""), false)
}

// Decommission all
val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend]
sc.getExecutorIds().foreach {
sched.decommissionExecutor(_, ExecutorDecommissionInfo(""), false)
}
// Make it sure that fallback storage are ready
val fallbackStorage = new FallbackStorage(sc.getConf)
eventually(timeout(10.seconds), interval(1.seconds)) {
Seq(
"shuffle_0_0_0.index", "shuffle_0_0_0.data",
"shuffle_0_1_0.index", "shuffle_0_1_0.data").foreach { file =>
assert(fallbackStorage.exists(0, file))
}
}

// Make it sure that fallback storage are ready
val fallbackStorage = new FallbackStorage(sc.getConf)
eventually(timeout(10.seconds), interval(1.seconds)) {
Seq(
"shuffle_0_0_0.index", "shuffle_0_0_0.data",
"shuffle_0_1_0.index", "shuffle_0_1_0.data").foreach { file =>
assert(fallbackStorage.exists(0, file))
// Since the data is safe, force to shrink down to zero executor
sc.getExecutorIds().foreach { id =>
sched.killExecutor(id)
}
eventually(timeout(20.seconds), interval(1.seconds)) {
assert(sc.getExecutorIds().isEmpty)
}
}

// Since the data is safe, force to shrink down to zero executor
sc.getExecutorIds().foreach { id =>
sched.killExecutor(id)
}
eventually(timeout(20.seconds), interval(1.seconds)) {
assert(sc.getExecutorIds().isEmpty)
// Dynamic allocation will start new executors
assert(rdd3.collect() === Array((0, 5), (1, 5)))
assert(rdd3.sortByKey().count() == 2)
assert(sc.getExecutorIds().nonEmpty)
}

// Dynamic allocation will start new executors
assert(rdd3.collect() === Array((0, 5), (1, 5)))
assert(rdd3.sortByKey().count() == 2)
assert(sc.getExecutorIds().nonEmpty)
}
}
}