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-30199][DSTREAM] Recover spark.(ui|blockManager).port from checkpoint #28320

Closed
wants to merge 1 commit into from
Closed
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 @@ -54,6 +54,8 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time)
"spark.driver.bindAddress",
"spark.driver.port",
"spark.master",
"spark.ui.port",
"spark.blockManager.port",
"spark.kubernetes.driver.pod.name",
"spark.kubernetes.executor.podNamePrefix",
"spark.yarn.jars",
Expand All @@ -66,6 +68,8 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time)
.remove("spark.driver.host")
.remove("spark.driver.bindAddress")
.remove("spark.driver.port")
.remove("spark.ui.port")
.remove("spark.blockManager.port")
.remove("spark.kubernetes.driver.pod.name")
.remove("spark.kubernetes.executor.podNamePrefix")
val newReloadConf = new SparkConf(loadDefaults = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -410,6 +410,33 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester
assert(restoredConf1.get("spark.driver.port") !== "9999")
}

test("SPARK-30199 get ui port and blockmanager port") {
val conf = Map("spark.ui.port" -> "30001", "spark.blockManager.port" -> "30002")
conf.foreach { case (k, v) => System.setProperty(k, v) }
ssc = new StreamingContext(master, framework, batchDuration)
conf.foreach { case (k, v) => assert(ssc.conf.get(k) === v) }

val cp = new Checkpoint(ssc, Time(1000))
ssc.stop()

// Serialize/deserialize to simulate write to storage and reading it back
val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp))

val newCpConf = newCp.createSparkConf()
conf.foreach { case (k, v) => assert(newCpConf.contains(k) && newCpConf.get(k) === v) }

// Check if all the parameters have been restored
ssc = new StreamingContext(null, newCp, null)
conf.foreach { case (k, v) => assert(ssc.conf.get(k) === v) }
ssc.stop()

// If port numbers are not set in system property, these parameters should not be presented
// in the newly recovered conf.
conf.foreach(kv => System.clearProperty(kv._1))
val newCpConf1 = newCp.createSparkConf()
conf.foreach { case (k, _) => assert(!newCpConf1.contains(k)) }
}

// This tests whether the system can recover from a master failure with simple
// non-stateful operations. This assumes as reliable, replayable input
// source - TestInputDStream.
Expand Down