Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

K8S Spark Init Container does not work with Secure HDFS #619

Closed
@rvesse

Description

@rvesse

When trying to run a job that requires the use of the --files flag to pre-load files into the container it seems that the init container does not include the Kerberos login logic which results in failure to download the dependencies thus failing the entire job.

Looking at the PR that added Secure HDFS support (#540) I don't see any sign that the init container logic was modified so it appears that this was not included.

Submission Line

spark-submit --deploy-mode cluster --master k8s://https://192.168.0.7:6443 --kubernetes-namespace rvesse --conf spark.executor.instances=5 --conf spark.app.name=spark-test --conf spark.kubernetes.authenticate.driver.serviceAccountName=spark --conf spark.kubernetes.kerberos.principal=rvesse@local --conf spark.kubernetes.kerberos.keytab=/security/secrets/rvesse.keytab --conf spark.kubernetes.kerberos.enabled=true --files hdfs://192.168.0.1:8020/user/rvesse/test2.py local:///var/spark-data/spark-files/test2.py

test2.py is just a toy Spark job, the contents are irrelevant here because the job fails before they are ever consumed but I would note that the same job runs fine on an unsecured HDFS cluster.

Resulting Logs

Job eventually fails, kubectl describe pods shows that the init container failed, and the following are the logs from that container:

kubectl logs spark-test-1519903287101-driver -c spark-init
++ 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 ']'
+ /sbin/tini -s -- /opt/spark/bin/spark-class org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer /etc/spark-init/spark-init.properties
2018-03-01 11:21:32 INFO  KubernetesSparkDependencyDownloadInitContainer:54 - Starting init-container to download Spark application dependencies.
2018-03-01 11:21:33 WARN  NativeCodeLoader:62 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
2018-03-01 11:21:33 INFO  SecurityManager:54 - Changing view acls to: root
2018-03-01 11:21:33 INFO  SecurityManager:54 - Changing modify acls to: root
2018-03-01 11:21:33 INFO  SecurityManager:54 - Changing view acls groups to: 
2018-03-01 11:21:33 INFO  SecurityManager:54 - Changing modify acls groups to: 
2018-03-01 11:21:33 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-03-01 11:21:33 INFO  SecurityManager:54 - Changing view acls to: root
2018-03-01 11:21:33 INFO  SecurityManager:54 - Changing modify acls to: root
2018-03-01 11:21:33 INFO  SecurityManager:54 - Changing view acls groups to: 
2018-03-01 11:21:33 INFO  SecurityManager:54 - Changing modify acls groups to: 
2018-03-01 11:21:33 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()
Exception in thread "main" org.apache.spark.SparkException: Exception thrown in awaitResult: 
	at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:205)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer$$anonfun$waitForFutures$1.apply(KubernetesSparkDependencyDownloadInitContainer.scala:187)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer$$anonfun$waitForFutures$1.apply(KubernetesSparkDependencyDownloadInitContainer.scala:187)
	at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
	at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:35)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer.waitForFutures(KubernetesSparkDependencyDownloadInitContainer.scala:186)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer.run(KubernetesSparkDependencyDownloadInitContainer.scala:140)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer$.main(KubernetesSparkDependencyDownloadInitContainer.scala:222)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer.main(KubernetesSparkDependencyDownloadInitContainer.scala)
Caused by: org.apache.hadoop.security.AccessControlException: SIMPLE authentication is not enabled.  Available:[TOKEN, KERBEROS]
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
	at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
	at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2110)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1305)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
	at org.apache.hadoop.fs.FileSystem.isFile(FileSystem.java:1452)
	at org.apache.spark.util.Utils$.fetchHcfsFile(Utils.scala:707)
	at org.apache.spark.util.Utils$.doFetchFile(Utils.scala:685)
	at org.apache.spark.util.Utils$.fetchFile(Utils.scala:480)
	at org.apache.spark.deploy.rest.k8s.FileFetcherImpl.fetchFile(KubernetesSparkDependencyDownloadInitContainer.scala:195)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer$$anonfun$org$apache$spark$deploy$rest$k8s$KubernetesSparkDependencyDownloadInitContainer$$downloadFiles$4.apply(KubernetesSparkDependencyDownloadInitContainer.scala:181)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer$$anonfun$org$apache$spark$deploy$rest$k8s$KubernetesSparkDependencyDownloadInitContainer$$downloadFiles$4.apply(KubernetesSparkDependencyDownloadInitContainer.scala:180)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer.org$apache$spark$deploy$rest$k8s$KubernetesSparkDependencyDownloadInitContainer$$downloadFiles(KubernetesSparkDependencyDownloadInitContainer.scala:180)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer$$anonfun$4.apply$mcV$sp(KubernetesSparkDependencyDownloadInitContainer.scala:135)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer$$anonfun$4.apply(KubernetesSparkDependencyDownloadInitContainer.scala:135)
	at org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer$$anonfun$4.apply(KubernetesSparkDependencyDownloadInitContainer.scala:135)
	at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
	at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.AccessControlException): SIMPLE authentication is not enabled.  Available:[TOKEN, KERBEROS]
	at org.apache.hadoop.ipc.Client.call(Client.java:1475)
	at org.apache.hadoop.ipc.Client.call(Client.java:1412)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
	at com.sun.proxy.$Proxy11.getFileInfo(Unknown Source)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:771)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191)
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
	at com.sun.proxy.$Proxy12.getFileInfo(Unknown Source)
	at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2108)
	... 21 more

So it looks like the init container isn't recognising that it should be useful Kerberos login for HDFS

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions