[
https://issues.apache.org/jira/browse/SPARK-6476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15068572#comment-15068572
]
Kyle Sutton commented on SPARK-6476:
The issue of the file server using the default IP instead of the IP address
configured through {{spark.driver.host}} still exists in _Spark 1.5.2_
The problem is that, while the file server is listening on all ports on the
file server host, the _Spark_ service attempts to call back to the default port
of the host, to which it may or may not have connectivity.
For instance, the following setup causes a {{java.net.SocketTimeoutException}}
when the _Spark_ service tries to contact the _Spark_ driver host for a JAR:
* Driver host has a default IP of {{192.168.1.2}} and a secondary LAN
connection IP of {{172.30.0.2}}
* _Spark_ service is on the LAN with an IP of {{172.30.0.3}}
* A connection is made from the driver host to the _Spark_ service
** {{spark.driver.host}} is set to the IP of the driver host on the LAN
{{172.30.0.2}}
** {{spark.driver.port}} is set to {{50003}}
** {{spark.fileserver.port}} is set to {{50005}}
* Locally (on the driver host), the following listeners are active:
** {{0.0.0.0:50005}}
** {{172.30.0.2:50003}}
* The _Spark_ service calls back to the file server host for a JAR file using
the driver host's default IP: {{http://192.168.1.2:50005/jars/code.jar}}
* The _Spark_ service, being on a different network than the driver host,
cannot see the {{192.168.1.0/24}} address space, and fails to connect to the
file server
** A {{netstat}} on the _Spark_ service host will show the connection to the
file server host as being in {{SYN_SENT}} state until the process gives up
trying to connect
{code:title=Driver|borderStyle=solid}
SparkConf conf = new SparkConf()
.setMaster("spark://172.30.0.3:7077")
.setAppName("TestApp")
.set("spark.driver.host", "172.30.0.2")
.set("spark.driver.port", "50003")
.set("spark.fileserver.port", "50005");
JavaSparkContext sc = new JavaSparkContext(conf);
sc.addJar("target/code.jar");
{code}
{code:title=Stacktrace|borderStyle=solid}
15/12/22 12:48:33 WARN TaskSetManager: Lost task 0.0 in stage 0.0 (TID 0,
172.30.0.3): java.net.SocketTimeoutException: connect timed out
at java.net.PlainSocketImpl.socketConnect(Native Method)
at
java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:350)
at
java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:206)
at
java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188)
at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392)
at java.net.Socket.connect(Socket.java:589)
at sun.net.NetworkClient.doConnect(NetworkClient.java:175)
at sun.net.www.http.HttpClient.openServer(HttpClient.java:432)
at sun.net.www.http.HttpClient.openServer(HttpClient.java:527)
at sun.net.www.http.HttpClient.(HttpClient.java:211)
at sun.net.www.http.HttpClient.New(HttpClient.java:308)
at sun.net.www.http.HttpClient.New(HttpClient.java:326)
at
sun.net.www.protocol.http.HttpURLConnection.getNewHttpClient(HttpURLConnection.java:1169)
at
sun.net.www.protocol.http.HttpURLConnection.plainConnect0(HttpURLConnection.java:1105)
at
sun.net.www.protocol.http.HttpURLConnection.plainConnect(HttpURLConnection.java:999)
at
sun.net.www.protocol.http.HttpURLConnection.connect(HttpURLConnection.java:933)
at org.apache.spark.util.Utils$.doFetchFile(Utils.scala:555)
at org.apache.spark.util.Utils$.fetchFile(Utils.scala:356)
at
org.apache.spark.executor.Executor$$anonfun$org$apache$spark$executor$Executor$$updateDependencies$5.apply(Executor.scala:405)
at
org.apache.spark.executor.Executor$$anonfun$org$apache$spark$executor$Executor$$updateDependencies$5.apply(Executor.scala:397)
at
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772)
at
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
at
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
at
scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
at
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
at
org.apache.spark.executor.Executor.org$apache$spark$executor$Executor$$updateDependencies(Executor.scala:397)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:193)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.