Skip to content

Commit 6fe1ce6

Browse files
zsxwingAndrew Or
authored and
Andrew Or
committed
[SPARK-11831][CORE][TESTS] Use port 0 to avoid port conflicts in tests (backport to branch 1.5)
backport #9841 to branch 1.5 Author: Shixiong Zhu <shixiong@databricks.com> Closes #9850 from zsxwing/SPARK-11831-branch-1.5.
1 parent 001c446 commit 6fe1ce6

File tree

2 files changed

+10
-10
lines changed

2 files changed

+10
-10
lines changed

core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
3838

3939
override def beforeAll(): Unit = {
4040
val conf = new SparkConf()
41-
env = createRpcEnv(conf, "local", 12345)
41+
env = createRpcEnv(conf, "local", 0)
4242
}
4343

4444
override def afterAll(): Unit = {
@@ -75,7 +75,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
7575
}
7676
})
7777

78-
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345)
78+
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0)
7979
// Use anotherEnv to find out the RpcEndpointRef
8080
val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "send-remotely")
8181
try {
@@ -130,7 +130,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
130130
}
131131
})
132132

133-
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345)
133+
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0)
134134
// Use anotherEnv to find out the RpcEndpointRef
135135
val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-remotely")
136136
try {
@@ -158,7 +158,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
158158
val shortProp = "spark.rpc.short.timeout"
159159
conf.set("spark.rpc.retry.wait", "0")
160160
conf.set("spark.rpc.numRetries", "1")
161-
val anotherEnv = createRpcEnv(conf, "remote", 13345)
161+
val anotherEnv = createRpcEnv(conf, "remote", 0)
162162
// Use anotherEnv to find out the RpcEndpointRef
163163
val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-timeout")
164164
try {
@@ -420,7 +420,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
420420
}
421421
})
422422

423-
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345)
423+
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0)
424424
// Use anotherEnv to find out the RpcEndpointRef
425425
val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "sendWithReply-remotely")
426426
try {
@@ -460,7 +460,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
460460
}
461461
})
462462

463-
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345)
463+
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0)
464464
// Use anotherEnv to find out the RpcEndpointRef
465465
val rpcEndpointRef = anotherEnv.setupEndpointRef(
466466
"local", env.address, "sendWithReply-remotely-error")
@@ -500,7 +500,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
500500

501501
})
502502

503-
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345)
503+
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0)
504504
// Use anotherEnv to find out the RpcEndpointRef
505505
val rpcEndpointRef = anotherEnv.setupEndpointRef(
506506
"local", env.address, "network-events")
@@ -529,7 +529,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
529529
}
530530
})
531531

532-
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345)
532+
val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0)
533533
// Use anotherEnv to find out the RpcEndpointRef
534534
val rpcEndpointRef = anotherEnv.setupEndpointRef(
535535
"local", env.address, "sendWithReply-unserializable-error")

core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ class AkkaRpcEnvSuite extends RpcEnvSuite {
3737
})
3838
val conf = new SparkConf()
3939
val newRpcEnv = new AkkaRpcEnvFactory().create(
40-
RpcEnvConfig(conf, "test", "localhost", 12346, new SecurityManager(conf)))
40+
RpcEnvConfig(conf, "test", "localhost", 0, new SecurityManager(conf)))
4141
try {
4242
val newRef = newRpcEnv.setupEndpointRef("local", ref.address, "test_endpoint")
4343
assert(s"akka.tcp://local@${env.address}/user/test_endpoint" ===
@@ -56,7 +56,7 @@ class AkkaRpcEnvSuite extends RpcEnvSuite {
5656
val conf = SSLSampleConfigs.sparkSSLConfig()
5757
val securityManager = new SecurityManager(conf)
5858
val rpcEnv = new AkkaRpcEnvFactory().create(
59-
RpcEnvConfig(conf, "test", "localhost", 12346, securityManager))
59+
RpcEnvConfig(conf, "test", "localhost", 0, securityManager))
6060
try {
6161
val uri = rpcEnv.uriOf("local", RpcAddress("1.2.3.4", 12345), "test_endpoint")
6262
assert("akka.ssl.tcp://local@1.2.3.4:12345/user/test_endpoint" === uri)

0 commit comments

Comments
 (0)