Stanis Shkel created SPARK-26400:
------------------------------------

             Summary: [k8s] Init container silently swallows errors when 
fetching jars from remote url
                 Key: SPARK-26400
                 URL: https://issues.apache.org/jira/browse/SPARK-26400
             Project: Spark
          Issue Type: Bug
          Components: Kubernetes
    Affects Versions: 2.3.2
            Reporter: Stanis Shkel


I run the following command

{code:bash}

spark-2.3.2-bin-hadoop2.7/bin/spark-submit --name client \
--master "k8s://cluster" \
--deploy-mode cluster \
--conf spark.executor.instances=2 \
--conf spark.executor.memory=5G \
--conf spark.driver.memory=8G \
--conf 
spark.kubernetes.container.image=rego.azurecr.io/spark:spark-2.3.2-hadoop2.7 \
--class au.com.random.DoesntMatter \
"https://fake-link.com/jars/my.jar";

{code}

I expect init container to fail to download jar and get a failure in the init 
stage. Instead I get driver failure with the following message.

{code:bash}

++ id -u
+ myuid=0
++ id -g
+ mygid=0
++ getent passwd 0
+ uidentry=root:x:0:0:root:/root:/bin/ash
+ '[' -z root:x:0:0:root:/root:/bin/ash ']'
+ SPARK_K8S_CMD=driver
+ '[' -z driver ']'
+ shift 1
+ SPARK_CLASSPATH=':/opt/spark/jars/*'
+ env
+ grep SPARK_JAVA_OPT_
+ sed 's/[^=]*=\(.*\)/\1/g'
+ sort -t_ -k4 -n
+ readarray -t SPARK_JAVA_OPTS
+ '[' -n /var/spark-data/spark-jars/my.jar:/var/spark-data/spark-jars/my.jar ']'
+ 
SPARK_CLASSPATH=':/opt/spark/jars/*:/var/spark-data/spark-jars/my.jar:/var/spark-data/spark-jars/my.jar'
+ '[' -n /var/spark-data/spark-files ']'
+ cp -R /var/spark-data/spark-files/. .
+ case "$SPARK_K8S_CMD" in
+ CMD=(${JAVA_HOME}/bin/java "${SPARK_JAVA_OPTS[@]}" -cp "$SPARK_CLASSPATH" 
-Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY 
-Dspark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS $SPARK_DRIVER_CLASS 
$SPARK_DRIVER_ARGS)
+ exec /sbin/tini -s -- /usr/lib/jvm/java-1.8-openjdk/bin/java 
-Dspark.master=k8s://kubernetes:443 
-Dspark.app.id=spark-2f340a028a314e9cb0df8165d887bfb7 
-Dspark.kubernetes.container.image=azure.azurecr.io/spark:spark-2.3.2-hadoop2.7 
-Dspark.submit.deployMode=cluster -Dspark.driver.blockManager.port=7079 
-Dspark.executor.memory=5G 
-Dspark.kubernetes.executor.podNamePrefix=client-f20f30e154a13624a728d6f56d45da3e
 
-Dspark.jars=https://fake-link.com/jars/my.jar,https://fake-link.com/jars/my.jar
 -Dspark.driver.memory=8G -Dspark.driver.port=7078 
-Dspark.kubernetes.driver.pod.name=client-f20f30e154a13624a728d6f56d45da3e-driver
 -Dspark.app.name=client 
-Dspark.kubernetes.initContainer.configMapKey=spark-init.properties 
-Dspark.executor.instances=2 
-Dspark.driver.host=client-f20f30e154a13624a728d6f56d45da3e-driver-svc.default.svc
 
-Dspark.kubernetes.initContainer.configMapName=client-f20f30e154a13624a728d6f56d45da3e-init-config
 -cp 
':/opt/spark/jars/*:/var/spark-data/spark-jars/my.jar:/var/spark-data/spark-jars/my.jar'
 -Xms8G -Xmx8G -Dspark.driver.bindAddress=10.1.0.101 au.com.random.DoesntMatter
Error: Could not find or load main class au.com.random.DoesntMatter

{code}

This happens because spark-init container failed to download the dependencies 
but misreports the status. Here is a log snippet from spark-init container

{code:bash}

++ id -u
+ myuid=0
++ id -g
+ mygid=0
++ getent passwd 0
+ uidentry=root:x:0:0:root:/root:/bin/ash
+ '[' -z root:x:0:0:root:/root:/bin/ash ']'
+ SPARK_K8S_CMD=init
+ '[' -z init ']'
+ shift 1
+ SPARK_CLASSPATH=':/opt/spark/jars/*'
+ env
+ grep SPARK_JAVA_OPT_
+ sed 's/[^=]*=\(.*\)/\1/g'
+ sort -t_ -k4 -n
+ readarray -t SPARK_JAVA_OPTS
+ '[' -n '' ']'
+ '[' -n '' ']'
+ case "$SPARK_K8S_CMD" in
+ CMD=("$SPARK_HOME/bin/spark-class" 
"org.apache.spark.deploy.k8s.SparkPodInitContainer" "$@")
+ exec /sbin/tini -s -- /opt/spark/bin/spark-class 
org.apache.spark.deploy.k8s.SparkPodInitContainer 
/etc/spark-init/spark-init.properties
2018-12-18 21:15:41 INFO SparkPodInitContainer:54 - Starting init-container to 
download Spark application dependencies.
2018-12-18 21:15:41 WARN NativeCodeLoader:62 - Unable to load native-hadoop 
library for your platform... using builtin-java classes where applicable
2018-12-18 21:15:41 INFO SecurityManager:54 - Changing view acls to: root
2018-12-18 21:15:41 INFO SecurityManager:54 - Changing modify acls to: root
2018-12-18 21:15:41 INFO SecurityManager:54 - Changing view acls groups to:
2018-12-18 21:15:41 INFO SecurityManager:54 - Changing modify acls groups to:
2018-12-18 21:15:41 INFO SecurityManager:54 - SecurityManager: authentication 
disabled; ui acls disabled; users with view permissions: Set(root); groups with 
view permissions: Set(); users with modify permissions: Set(root); groups with 
modify permissions: Set()
2018-12-18 21:15:41 INFO SparkPodInitContainer:54 - Downloading remote jars: 
Some(https://fake-link.com/jars/my.jar,https://fake-link.com/jars/my.jar)
2018-12-18 21:15:41 INFO SparkPodInitContainer:54 - Downloading remote files: 
None
2018-12-18 21:15:42 INFO SparkPodInitContainer:54 - Finished downloading 
application dependencies.

{code}

I think the problem resides somewhere around here(I have little experience with 
scala, so might be wrong)

 
[https://github.com/apache/spark/blob/branch-2.3/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainer.scala#L80]

If I remove Future wrapper around fileFetcher I get expected behaviour. My 
driver pod will fail with Init:Error and I get the error thrown properly. PS 
the error below is from a different command, where I was trying to pull from a 
blob, but similar issue.

{code:bash}

Exception in thread "main" org.apache.hadoop.fs.azure.AzureException: 
org.apache.hadoop.fs.azure.AzureException: Container qrefinery in account 
test.blob.core.windows.net not found, and we can't create it using anoynomous 
credentials.
 at 
org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.createAzureStorageSession(AzureNativeFileSystemStore.java:938)
 at 
org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.initialize(AzureNativeFileSystemStore.java:438)
 at 
org.apache.hadoop.fs.azure.NativeAzureFileSystem.initialize(NativeAzureFileSystem.java:1048)
 at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2669)
 at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94)
 at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703)
 at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685)
 at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373)
 at org.apache.spark.util.Utils$.getHadoopFileSystem(Utils.scala:1910)
 at org.apache.spark.util.Utils$.doFetchFile(Utils.scala:700)
 at org.apache.spark.util.Utils$.fetchFile(Utils.scala:492)
 at 
org.apache.spark.deploy.k8s.FileFetcher.fetchFile(SparkPodInitContainer.scala:91)
 at 
org.apache.spark.deploy.k8s.SparkPodInitContainer$$anonfun$downloadFiles$1$$anonfun$apply$2.apply(SparkPodInitContainer.scala:81)
 at 
org.apache.spark.deploy.k8s.SparkPodInitContainer$$anonfun$downloadFiles$1$$anonfun$apply$2.apply(SparkPodInitContainer.scala:79)
 at 
scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:35)
 at 
org.apache.spark.deploy.k8s.SparkPodInitContainer$$anonfun$downloadFiles$1.apply(SparkPodInitContainer.scala:79)
 at 
org.apache.spark.deploy.k8s.SparkPodInitContainer$$anonfun$downloadFiles$1.apply(SparkPodInitContainer.scala:77)
 at scala.Option.foreach(Option.scala:257)
 at 
org.apache.spark.deploy.k8s.SparkPodInitContainer.downloadFiles(SparkPodInitContainer.scala:77)
 at 
org.apache.spark.deploy.k8s.SparkPodInitContainer.run(SparkPodInitContainer.scala:56)
 at 
org.apache.spark.deploy.k8s.SparkPodInitContainer$.main(SparkPodInitContainer.scala:113)
 at 
org.apache.spark.deploy.k8s.SparkPodInitContainer.main(SparkPodInitContainer.scala)
Caused by: org.apache.hadoop.fs.azure.AzureException: Container qrefinery in 
account jr3e3d.blob.core.windows.net not found, and we can't create it using 
anoynomous credentials.
 at 
org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.connectUsingAnonymousCredentials(AzureNativeFileSystemStore.java:730)
 at 
org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.createAzureStorageSession(AzureNativeFileSystemStore.java:933)
 ... 22 more

{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to