Skip to content

Commit f450cd1

Browse files
startServiceOnPort will use a SparkConf arg
1 parent 29b751b commit f450cd1

File tree

6 files changed

+8
-9
lines changed

6 files changed

+8
-9
lines changed

core/src/main/scala/org/apache/spark/HttpServer.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -57,7 +57,7 @@ private[spark] class HttpServer(
5757
} else {
5858
logInfo("Starting HTTP Server")
5959
val (actualServer, actualPort) =
60-
Utils.startServiceOnPort[Server](requestedPort, doStart, serverName)
60+
Utils.startServiceOnPort[Server](requestedPort, doStart, serverName, new SparkConf())
6161
server = actualServer
6262
port = actualPort
6363
}

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -176,8 +176,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
176176
logInfo(s"Running Spark version $SPARK_VERSION")
177177

178178
private[spark] val conf = config.clone()
179-
conf.getOption("spark.port.maxRetries")
180-
.foreach(portRetriesConf => System.setProperty("spark.port.maxRetries", portRetriesConf))
181179
conf.validateSettings()
182180

183181
/**

core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -174,7 +174,7 @@ private[nio] class ConnectionManager(
174174
serverChannel.socket.bind(new InetSocketAddress(port))
175175
(serverChannel, serverChannel.socket.getLocalPort)
176176
}
177-
Utils.startServiceOnPort[ServerSocketChannel](port, startService, name)
177+
Utils.startServiceOnPort[ServerSocketChannel](port, startService, name, conf)
178178
serverChannel.register(selector, SelectionKey.OP_ACCEPT)
179179

180180
val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort)

core/src/main/scala/org/apache/spark/ui/JettyUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -201,7 +201,7 @@ private[spark] object JettyUtils extends Logging {
201201
}
202202
}
203203

204-
val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, serverName)
204+
val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, serverName, conf)
205205
ServerInfo(server, boundPort, collection)
206206
}
207207

core/src/main/scala/org/apache/spark/util/AkkaUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ private[spark] object AkkaUtils extends Logging {
5353
val startService: Int => (ActorSystem, Int) = { actualPort =>
5454
doCreateActorSystem(name, host, actualPort, conf, securityManager)
5555
}
56-
Utils.startServiceOnPort(port, startService, name)
56+
Utils.startServiceOnPort(port, startService, name, conf)
5757
}
5858

5959
private def doCreateActorSystem(

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1692,12 +1692,12 @@ private[spark] object Utils extends Logging {
16921692
/**
16931693
* Default maximum number of retries when binding to a port before giving up.
16941694
*/
1695-
lazy val portMaxRetries: Int = {
1695+
def portMaxRetries(conf: SparkConf): Int = {
16961696
if (sys.props.contains("spark.testing")) {
16971697
// Set a higher number of retries for tests...
16981698
sys.props.get("spark.port.maxRetries").map(_.toInt).getOrElse(100)
16991699
} else {
1700-
sys.props.get("spark.port.maxRetries").map(_.toInt).getOrElse(16)
1700+
conf.getOption("spark.port.maxRetries").map(_.toInt).getOrElse(16)
17011701
}
17021702
}
17031703

@@ -1715,8 +1715,9 @@ private[spark] object Utils extends Logging {
17151715
startPort: Int,
17161716
startService: Int => (T, Int),
17171717
serviceName: String = "",
1718-
maxRetries: Int = portMaxRetries): (T, Int) = {
1718+
conf: SparkConf): (T, Int) = {
17191719
val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'"
1720+
val maxRetries = portMaxRetries(conf)
17201721
logInfo(s"Starting service$serviceString on port $startPort with maximum $maxRetries retries. ")
17211722
for (offset <- 0 to maxRetries) {
17221723
// Do not increment port if startPort is 0, which is treated as a special port

0 commit comments

Comments
 (0)