[ https://issues.apache.org/jira/browse/SPARK-39399?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17581495#comment-17581495 ]
Shrikant Prasad commented on SPARK-39399: ----------------------------------------- [~dongjoon] [~hyukjin.kwon] Can you please have a look at this issue and let me know if I need to add any more details in order to take this forward. > proxy-user not working for Spark on k8s in cluster deploy mode > -------------------------------------------------------------- > > Key: SPARK-39399 > URL: https://issues.apache.org/jira/browse/SPARK-39399 > Project: Spark > Issue Type: Bug > Components: Kubernetes, Spark Core > Affects Versions: 3.2.0 > Reporter: Shrikant Prasad > Priority: Major > > As part of https://issues.apache.org/jira/browse/SPARK-25355 Proxy user > support was added for Spark on K8s. But the PR only added proxy user argument > on the spark-submit command. The actual functionality of authentication using > the proxy user is not working in case of cluster deploy mode. > We get AccessControlException when trying to access the kerberized HDFS > through a proxy user. > Spark-Submit: > $SPARK_HOME/bin/spark-submit \ > --master <K8S_APISERVER> \ > --deploy-mode cluster \ > --name with_proxy_user_di \ > --proxy-user <username> \ > --class org.apache.spark.examples.SparkPi \ > --conf spark.kubernetes.container.image=<SPARK3.2_with_hadoop3.1_image> \ > --conf spark.kubernetes.driver.limit.cores=1 \ > --conf spark.executor.instances=1 \ > --conf spark.kubernetes.authenticate.driver.serviceAccountName=spark \ > --conf spark.kubernetes.namespace=<namespace_name> \ > --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ > --conf spark.eventLog.enabled=true \ > --conf spark.eventLog.dir=hdfs://<hdfs_cluster>/scaas/shs_logs \ > --conf spark.kubernetes.file.upload.path=hdfs://<hdfs_cluster>/tmp \ > --conf spark.kubernetes.container.image.pullPolicy=Always \ > $SPARK_HOME/examples/jars/spark-examples_2.12-3.2.0-1.jar > Driver Logs: > {code:java} > ++ id -u > + myuid=185 > ++ id -g > + mygid=0 > + set +e > ++ getent passwd 185 > + uidentry= > + set -e > + '[' -z '' ']' > + '[' -w /etc/passwd ']' > + echo '185:x:185:0:anonymous uid:/opt/spark:/bin/false' > + SPARK_CLASSPATH=':/opt/spark/jars/*' > + env > + grep SPARK_JAVA_OPT_ > + sort -t_ -k4 -n > + sed 's/[^=]*=\(.*\)/\1/g' > + readarray -t SPARK_EXECUTOR_JAVA_OPTS > + '[' -n '' ']' > + '[' -z ']' > + '[' -z ']' > + '[' -n '' ']' > + '[' -z x ']' > + SPARK_CLASSPATH='/opt/hadoop/conf::/opt/spark/jars/*' > + '[' -z x ']' > + SPARK_CLASSPATH='/opt/spark/conf:/opt/hadoop/conf::/opt/spark/jars/*' > + case "$1" in > + shift 1 > + CMD=("$SPARK_HOME/bin/spark-submit" --conf > "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" --deploy-mode client > "$@") > + exec /usr/bin/tini -s -- /opt/spark/bin/spark-submit --conf > spark.driver.bindAddress=<addr> --deploy-mode client --proxy-user proxy_user > --properties-file /opt/spark/conf/spark.properties --class > org.apache.spark.examples.SparkPi spark-internal > WARNING: An illegal reflective access operation has occurred > WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform > (file:/opt/spark/jars/spark-unsafe_2.12-3.2.0-1.jar) to constructor > java.nio.DirectByteBuffer(long,int) > WARNING: Please consider reporting this to the maintainers of > org.apache.spark.unsafe.Platform > WARNING: Use --illegal-access=warn to enable warnings of further illegal > reflective access operations > WARNING: All illegal access operations will be denied in a future release > 22/04/26 08:54:38 DEBUG MutableMetricsFactory: field > org.apache.hadoop.metrics2.lib.MutableRate > org.apache.hadoop.security.UserGroupInformation$UgiMetrics.loginSuccess with > annotation @org.apache.hadoop.metrics2.annotation.Metric(about="", > sampleName="Ops", always=false, type=DEFAULT, value={"Rate of successful > kerberos logins and latency (milliseconds)"}, valueName="Time") > 22/04/26 08:54:38 DEBUG MutableMetricsFactory: field > org.apache.hadoop.metrics2.lib.MutableRate > org.apache.hadoop.security.UserGroupInformation$UgiMetrics.loginFailure with > annotation @org.apache.hadoop.metrics2.annotation.Metric(about="", > sampleName="Ops", always=false, type=DEFAULT, value={"Rate of failed kerberos > logins and latency (milliseconds)"}, valueName="Time") > 22/04/26 08:54:38 DEBUG MutableMetricsFactory: field > org.apache.hadoop.metrics2.lib.MutableRate > org.apache.hadoop.security.UserGroupInformation$UgiMetrics.getGroups with > annotation @org.apache.hadoop.metrics2.annotation.Metric(about="", > sampleName="Ops", always=false, type=DEFAULT, value={"GetGroups"}, > valueName="Time") > 22/04/26 08:54:38 DEBUG MutableMetricsFactory: field private > org.apache.hadoop.metrics2.lib.MutableGaugeLong > org.apache.hadoop.security.UserGroupInformation$UgiMetrics.renewalFailuresTotal > with annotation @org.apache.hadoop.metrics2.annotation.Metric(about="", > sampleName="Ops", always=false, type=DEFAULT, value={"Renewal failures since > startup"}, valueName="Time") > 22/04/26 08:54:38 DEBUG MutableMetricsFactory: field private > org.apache.hadoop.metrics2.lib.MutableGaugeInt > org.apache.hadoop.security.UserGroupInformation$UgiMetrics.renewalFailures > with annotation @org.apache.hadoop.metrics2.annotation.Metric(about="", > sampleName="Ops", always=false, type=DEFAULT, value={"Renewal failures since > last successful login"}, valueName="Time") > 22/04/26 08:54:38 DEBUG MetricsSystemImpl: UgiMetrics, User and group related > metrics > 22/04/26 08:54:38 DEBUG SecurityUtil: Setting > hadoop.security.token.service.use_ip to true > 22/04/26 08:54:38 DEBUG Shell: Failed to detect a valid hadoop home directory > java.io.FileNotFoundException: HADOOP_HOME and hadoop.home.dir are unset. > at org.apache.hadoop.util.Shell.checkHadoopHomeInner(Shell.java:469) > at org.apache.hadoop.util.Shell.checkHadoopHome(Shell.java:440) > at org.apache.hadoop.util.Shell.<clinit>(Shell.java:517) > at org.apache.hadoop.util.StringUtils.<clinit>(StringUtils.java:78) > at > org.apache.hadoop.conf.Configuration.getBoolean(Configuration.java:1665) > at > org.apache.hadoop.security.SecurityUtil.setConfigurationInternal(SecurityUtil.java:102) > at org.apache.hadoop.security.SecurityUtil.<clinit>(SecurityUtil.java:86) > at > org.apache.hadoop.security.UserGroupInformation.initialize(UserGroupInformation.java:315) > at > org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:303) > at > org.apache.hadoop.security.UserGroupInformation.doSubjectLogin(UserGroupInformation.java:1827) > at > org.apache.hadoop.security.UserGroupInformation.createLoginUser(UserGroupInformation.java:709) > at > org.apache.hadoop.security.UserGroupInformation.getLoginUser(UserGroupInformation.java:659) > at > org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:570) > at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:161) > at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) > at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) > at > org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1043) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1052) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > 22/04/26 08:54:38 DEBUG Shell: setsid exited with exit code 0 > 22/04/26 08:54:38 DEBUG Groups: Creating new Groups object > 22/04/26 08:54:38 DEBUG AbstractJavaKeyStoreProvider: backing jks path > initialized to file:/etc/security/bind.jceks > 22/04/26 08:54:38 DEBUG AbstractJavaKeyStoreProvider: initialized local file > as '/etc/security/bind.jceks'. > 22/04/26 08:54:38 DEBUG AbstractJavaKeyStoreProvider: the local file does not > exist. > 22/04/26 08:54:38 DEBUG LdapGroupsMapping: Usersearch baseDN: dc=<dc> > 22/04/26 08:54:38 DEBUG LdapGroupsMapping: Groupsearch baseDN: dc=<dc> > 22/04/26 08:54:38 DEBUG Groups: Group mapping > impl=org.apache.hadoop.security.LdapGroupsMapping; cacheTimeout=300000; > warningDeltaMs=5000 > 22/04/26 08:54:38 DEBUG UserGroupInformation: hadoop login > 22/04/26 08:54:38 DEBUG UserGroupInformation: hadoop login commit > 22/04/26 08:54:38 DEBUG UserGroupInformation: using local user:UnixPrincipal: > 185 > 22/04/26 08:54:38 DEBUG UserGroupInformation: Using user: "UnixPrincipal: > 185" with name 185 > 22/04/26 08:54:38 DEBUG UserGroupInformation: User entry: "185" > 22/04/26 08:54:38 DEBUG UserGroupInformation: Reading credentials from > location set in HADOOP_TOKEN_FILE_LOCATION: > /mnt/secrets/hadoop-credentials/..2022_04_26_08_54_34.1262645511/hadoop-tokens > 22/04/26 08:54:39 DEBUG UserGroupInformation: Loaded 3 tokens > 22/04/26 08:54:39 DEBUG UserGroupInformation: UGI loginUser:185 (auth:SIMPLE) > 22/04/26 08:54:39 DEBUG UserGroupInformation: PrivilegedAction as:proxy_user > (auth:PROXY) via 185 (auth:SIMPLE) > from:org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:163) > 22/04/26 08:54:39 DEBUG FileSystem: Loading filesystems > 22/04/26 08:54:39 DEBUG FileSystem: file:// = class > org.apache.hadoop.fs.LocalFileSystem from > /opt/spark/jars/hadoop-client-api-3.1.1.jar > 22/04/26 08:54:39 DEBUG FileSystem: viewfs:// = class > org.apache.hadoop.fs.viewfs.ViewFileSystem from > /opt/spark/jars/hadoop-client-api-3.1.1.jar > 22/04/26 08:54:39 DEBUG FileSystem: har:// = class > org.apache.hadoop.fs.HarFileSystem from > /opt/spark/jars/hadoop-client-api-3.1.1.jar > 22/04/26 08:54:39 DEBUG FileSystem: http:// = class > org.apache.hadoop.fs.http.HttpFileSystem from > /opt/spark/jars/hadoop-client-api-3.1.1.jar > 22/04/26 08:54:39 DEBUG FileSystem: https:// = class > org.apache.hadoop.fs.http.HttpsFileSystem from > /opt/spark/jars/hadoop-client-api-3.1.1.jar > 22/04/26 08:54:39 DEBUG FileSystem: hdfs:// = class > org.apache.hadoop.hdfs.DistributedFileSystem from > /opt/spark/jars/hadoop-client-api-3.1.1.jar > 22/04/26 08:54:39 DEBUG FileSystem: webhdfs:// = class > org.apache.hadoop.hdfs.web.WebHdfsFileSystem from > /opt/spark/jars/hadoop-client-api-3.1.1.jar > 22/04/26 08:54:39 DEBUG FileSystem: swebhdfs:// = class > org.apache.hadoop.hdfs.web.SWebHdfsFileSystem from > /opt/spark/jars/hadoop-client-api-3.1.1.jar > 22/04/26 08:54:39 DEBUG FileSystem: nullscan:// = class > org.apache.hadoop.hive.ql.io.NullScanFileSystem from > /opt/spark/jars/hive-exec-2.3.9-core.jar > 22/04/26 08:54:39 DEBUG FileSystem: file:// = class > org.apache.hadoop.hive.ql.io.ProxyLocalFileSystem from > /opt/spark/jars/hive-exec-2.3.9-core.jar > 22/04/26 08:54:39 DEBUG FileSystem: Looking for FS supporting hdfs > 22/04/26 08:54:39 DEBUG FileSystem: looking for configuration option > fs.hdfs.impl > 22/04/26 08:54:39 DEBUG FileSystem: Looking in service filesystems for > implementation class > 22/04/26 08:54:39 DEBUG FileSystem: FS for hdfs is class > org.apache.hadoop.hdfs.DistributedFileSystem > 22/04/26 08:54:39 DEBUG DfsClientConf: > dfs.client.use.legacy.blockreader.local = false > 22/04/26 08:54:39 DEBUG DfsClientConf: dfs.client.read.shortcircuit = true > 22/04/26 08:54:39 DEBUG DfsClientConf: dfs.client.domain.socket.data.traffic > = false > 22/04/26 08:54:39 DEBUG DfsClientConf: dfs.domain.socket.path = > /var/lib/hadoop-hdfs/dn_socket > 22/04/26 08:54:39 DEBUG DFSClient: Sets > dfs.client.block.write.replace-datanode-on-failure.min-replication to 0 > 22/04/26 08:54:39 DEBUG HAUtilClient: No HA service delegation token found > for logical URI > hdfs://<hdfs>/tmp/spark-upload-bf713a0c-166b-43fc-a5e6-24957e75b224/spark-examples_2.12-3.0.1.jar > 22/04/26 08:54:39 DEBUG DfsClientConf: > dfs.client.use.legacy.blockreader.local = false > 22/04/26 08:54:39 DEBUG DfsClientConf: dfs.client.read.shortcircuit = true > 22/04/26 08:54:39 DEBUG DfsClientConf: dfs.client.domain.socket.data.traffic > = false > 22/04/26 08:54:39 DEBUG DfsClientConf: dfs.domain.socket.path = > /var/lib/hadoop-hdfs/dn_socket > 22/04/26 08:54:39 DEBUG RetryUtils: multipleLinearRandomRetry = null > 22/04/26 08:54:39 DEBUG Server: rpcKind=RPC_PROTOCOL_BUFFER, > rpcRequestWrapperClass=class > org.apache.hadoop.ipc.ProtobufRpcEngine$RpcProtobufRequest, > rpcInvoker=org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker@4a325eb9 > 22/04/26 08:54:39 DEBUG Client: getting client out of cache: > org.apache.hadoop.ipc.Client@2577d6c8 > 22/04/26 08:54:40 DEBUG NativeCodeLoader: Trying to load the custom-built > native-hadoop library... > 22/04/26 08:54:40 DEBUG NativeCodeLoader: Failed to load native-hadoop with > error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: > [/usr/java/packages/lib, /usr/lib64, /lib64, /lib, /usr/lib] > 22/04/26 08:54:40 DEBUG NativeCodeLoader: > java.library.path=/usr/java/packages/lib:/usr/lib64:/lib64:/lib:/usr/lib > 22/04/26 08:54:40 WARN NativeCodeLoader: Unable to load native-hadoop library > for your platform... using builtin-java classes where applicable > 22/04/26 08:54:40 WARN DomainSocketFactory: The short-circuit local reads > feature cannot be used because libhadoop cannot be loaded. > 22/04/26 08:54:40 DEBUG DataTransferSaslUtil: DataTransferProtocol using > SaslPropertiesResolver, configured QOP dfs.data.transfer.protection = > authentication,privacy, configured class > dfs.data.transfer.saslproperties.resolver.class = class > org.apache.hadoop.security.SaslPropertiesResolver > 22/04/26 08:54:40 DEBUG Client: The ping interval is 60000 ms. > 22/04/26 08:54:40 DEBUG Client: Connecting to <server>/<ip>:8020 > 22/04/26 08:54:40 DEBUG UserGroupInformation: PrivilegedAction as:185 > (auth:SIMPLE) > from:org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:796) > 22/04/26 08:54:40 DEBUG SaslRpcClient: Sending sasl message state: > NEGOTIATE22/04/26 08:54:40 DEBUG SaslRpcClient: Get token info > proto:interface org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB > info:@org.apache.hadoop.security.token.TokenInfo(value=org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector.class) > 22/04/26 08:54:40 DEBUG SaslRpcClient: tokens aren't supported for this > protocol or user doesn't have one > 22/04/26 08:54:40 DEBUG SaslRpcClient: client isn't using kerberos > 22/04/26 08:54:40 DEBUG UserGroupInformation: PrivilegedActionException > as:185 (auth:SIMPLE) cause:org.apache.hadoop.security.AccessControlException: > Client cannot authenticate via:[TOKEN, KERBEROS] > 22/04/26 08:54:40 DEBUG UserGroupInformation: PrivilegedAction as:185 > (auth:SIMPLE) > from:org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:720) > 22/04/26 08:54:40 WARN Client: Exception encountered while connecting to the > server : org.apache.hadoop.security.AccessControlException: Client cannot > authenticate via:[TOKEN, KERBEROS] > 22/04/26 08:54:40 DEBUG UserGroupInformation: PrivilegedActionException > as:185 (auth:SIMPLE) cause:java.io.IOException: > org.apache.hadoop.security.AccessControlException: Client cannot authenticate > via:[TOKEN, KERBEROS] > 22/04/26 08:54:40 DEBUG Client: closing ipc connection to <server>/<ip>:8020: > org.apache.hadoop.security.AccessControlException: Client cannot authenticate > via:[TOKEN, KERBEROS] > java.io.IOException: org.apache.hadoop.security.AccessControlException: > Client cannot authenticate via:[TOKEN, KERBEROS] > at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:757) > at java.base/java.security.AccessController.doPrivileged(Native Method) > at java.base/javax.security.auth.Subject.doAs(Unknown Source) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1729) > at > org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:720) > at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:813) > at org.apache.hadoop.ipc.Client$Connection.access$3600(Client.java:410) > at org.apache.hadoop.ipc.Client.getConnection(Client.java:1558) > at org.apache.hadoop.ipc.Client.call(Client.java:1389) > at org.apache.hadoop.ipc.Client.call(Client.java:1353) > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:228) > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:116) > at com.sun.proxy.$Proxy14.getFileInfo(Unknown Source) > at > org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:900) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(Unknown > Source) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(Unknown > Source) > at java.base/java.lang.reflect.Method.invoke(Unknown Source) > at > org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:422) > at > org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165) > at > org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157) > at > org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95) > at > org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359) > at com.sun.proxy.$Proxy15.getFileInfo(Unknown Source) > at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1654) > at > org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1579) > at > org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1576) > at > org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) > at > org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1591) > at org.apache.hadoop.fs.Globber.getFileStatus(Globber.java:65) > at org.apache.hadoop.fs.Globber.doGlob(Globber.java:270) > at org.apache.hadoop.fs.Globber.glob(Globber.java:149) > at org.apache.hadoop.fs.FileSystem.globStatus(FileSystem.java:2067) > at > org.apache.spark.util.DependencyUtils$.resolveGlobPath(DependencyUtils.scala:318) > at > org.apache.spark.util.DependencyUtils$.$anonfun$resolveGlobPaths$2(DependencyUtils.scala:273) > at > org.apache.spark.util.DependencyUtils$.$anonfun$resolveGlobPaths$2$adapted(DependencyUtils.scala:271) > at > scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:293) > at > scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36) > at > scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33) > at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:38) > at scala.collection.TraversableLike.flatMap(TraversableLike.scala:293) > at scala.collection.TraversableLike.flatMap$(TraversableLike.scala:290) > at scala.collection.AbstractTraversable.flatMap(Traversable.scala:108) > at > org.apache.spark.util.DependencyUtils$.resolveGlobPaths(DependencyUtils.scala:271) > at > org.apache.spark.deploy.SparkSubmit.$anonfun$prepareSubmitEnvironment$4(SparkSubmit.scala:364) > at scala.Option.map(Option.scala:230) > at > org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment(SparkSubmit.scala:364) > at > org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:898) > at org.apache.spark.deploy.SparkSubmit$$anon$1.run(SparkSubmit.scala:165) > at org.apache.spark.deploy.SparkSubmit$$anon$1.run(SparkSubmit.scala:163) > at java.base/java.security.AccessController.doPrivileged(Native Method) > at java.base/javax.security.auth.Subject.doAs(Unknown Source) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1729) > at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:163) > at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) > at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) > at > org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1043) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1052) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > Caused by: org.apache.hadoop.security.AccessControlException: Client cannot > authenticate via:[TOKEN, KERBEROS] > at > org.apache.hadoop.security.SaslRpcClient.selectSaslClient(SaslRpcClient.java:173) > at > org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:390) > at > org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:614) > at org.apache.hadoop.ipc.Client$Connection.access$2300(Client.java:410) > at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:800) > at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:796) > at java.base/java.security.AccessController.doPrivileged(Native Method) > at java.base/javax.security.auth.Subject.doAs(Unknown Source) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1729) > at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:796) > ... 53 more {code} > > The reason for no delegation token found is that the proxy user UGI doesn't > have any credentials/tokens ( tokenSize:: 0 ) > {code:java} > 22/04/28 16:59:37 DEBUG UserGroupInformation: loginUser-token::Kind: > HDFS_DELEGATION_TOKEN, Service: ha-hdfs:<hdfs>, Ident: (token for proxyUser: > HDFS_DELEGATION_TOKEN owner=proxyUser, renewer=proxyUser, > realUser=superuser/t...@test.com, issueDate=1651165129518, > maxDate=1651769929518, sequenceNumber=180516, masterKeyId=601) > 22/04/28 16:59:37 DEBUG Token: Cannot find class for token kind > HIVE_DELEGATION_TOKEN > 22/04/28 16:59:37 DEBUG UserGroupInformation: loginUser-token::Kind: > HIVE_DELEGATION_TOKEN, Service: , Ident: 00 08 73 68 72 70 72 61 73 61 04 68 > 69 76 65 1e 6c 69 76 79 2f 6c 69 76 79 2d 69 6e 74 40 43 4f 52 50 44 45 56 2e > 56 49 53 41 2e 43 4f 4d 8a 01 80 71 1c 71 b5 8a 01 80 b9 35 79 b5 8e 15 cd 8e > 03 6e > 22/04/28 16:59:37 DEBUG UserGroupInformation: loginUser-token::Kind: kms-dt, > Service: <ip>:9292, Ident: (kms-dt owner=proxyUser, renewer=proxyUser, > realUser=superuser, issueDate=1651165129566, maxDate=1651769929566, > sequenceNumber=181197, masterKeyId=1152) > 22/04/28 16:59:37 DEBUG UserGroupInformation: UGI loginUser:185 (auth:SIMPLE) > 22/04/28 16:59:37 DEBUG UserGroupInformation: createProxyUser: > from:org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) > 22/04/28 16:59:37 DEBUG UserGroupInformation: proxy user created, > ugi::proxyUser (auth:PROXY) via 185 (auth:SIMPLE) subject::Subject: > Principal: proxyUser > Principal: 185 (auth:SIMPLE) > tokenSize:: 0 {code} > {code:java} > 22/04/28 16:59:38 DEBUG AbstractNNFailoverProxyProvider: ugi::proxyUser > (auth:PROXY) via 185 (auth:SIMPLE) tokensize:: 0 > 22/04/28 16:59:38 DEBUG HAUtilClient: ugi::proxyUser (auth:PROXY) via 185 > (auth:SIMPLE) tokenSize::0 > 22/04/28 16:59:38 DEBUG AbstractDelegationTokenSelector: kindName:: > HDFS_DELEGATION_TOKEN service:: ha-hdfs:<hdfs> tokens size:: 0 > 22/04/28 16:59:38 DEBUG HAUtilClient: No HA service delegation token found > for logical URI > hdfs://<hdfs>:8020/tmp/spark-upload-10582dde-f07c-4bf7-a611-5afbdd12ff6c/spark-examples_2.12-3.0.1.jar > {code} > > Please refer to the last 4 comments on > https://issues.apache.org/jira/browse/SPARK-25355. -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org