Hi Prashant,

I have the problem only on K8S, it's working fine when spark is executed on
top of yarn.
I'm asking myself if the delegation gets saved, any idea how to check that?
Could it be because kms is in HA and spark request 2 delegation token?

For the testing,  just running spark3 on top of any k8s cluster reading
data to any hadoop3 with kms should be fine. I'm using a HDP3 cluster, but
there is probably a more easy way to test.

Michel

Le mer. 19 août 2020 à 09:50, Prashant Sharma <scrapco...@gmail.com> a
écrit :

> -dev
> Hi,
>
> I have used Spark with HDFS encrypted with Hadoop KMS, and it worked well.
> Somehow, I could not recall, if I had the kubernetes in the mix. Somehow,
> seeing the error, it is not clear what caused the failure. Can I reproduce
> this somehow?
>
> Thanks,
>
> On Sat, Aug 15, 2020 at 7:18 PM Michel Sumbul <michelsum...@gmail.com>
> wrote:
>
>> Hi guys,
>>
>> Does anyone have an idea on this issue? even some tips to troubleshoot it?
>> I got the impression that after the creation of the delegation for the
>> KMS, the token is not sent to the executor or maybe not saved?
>>
>> I'm sure I'm not the only one using Spark with HDFS encrypted with KMS :-)
>>
>> Thanks,
>> Michel
>>
>> Le jeu. 13 août 2020 à 14:32, Michel Sumbul <michelsum...@gmail.com> a
>> écrit :
>>
>>> Hi guys,
>>>
>>> Does anyone try Spark3 on k8s reading data from HDFS encrypted with KMS
>>> in HA mode (with kerberos)?
>>>
>>> I have a wordcount job running with Spark3 reading data on HDFS (hadoop
>>> 3.1) everything secure with kerberos. Everything works fine if the data
>>> folder is not encrypted (spark on k8s). If the data is on an encrypted
>>> folder, Spark3 on yarn is working fine but it doesn't work when Spark3 is
>>> running on K8S.
>>> I submit the job with spark-submit command and I provide the keytab and
>>> the principal to use.
>>> I got the kerberos error saying that there is no TGT to authenticate to
>>> the KMS (ranger kms, full stack trace of the error at the end of the mail)
>>> servers but in the log I can see that Spark get 2 delegation token, one for
>>> each KMS servers:
>>>
>>> ------------------------------ ------------------------------
>>>
>>> 20/08/13 10:50:50 INFO HadoopDelegationTokenManager: Attempting to login
>>> to KDC using principal: mytestu...@paf.com
>>>
>>> 20/08/13 10:50:50 INFO HadoopDelegationTokenManager: Successfully logged
>>> into KDC.
>>>
>>> 20/08/13 10:50:52 WARN DomainSocketFactory: The short-circuit local
>>> reads feature cannot be used because libhadoop cannot be loaded.
>>>
>>> 20/08/13 10:50:52 INFO HadoopFSDelegationTokenProvider: getting token
>>> for: DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_-237056190_16, ugi=
>>> mytestu...@paf.com (auth:KERBEROS)]] with renewer testuser
>>>
>>> 20/08/13 10:50:52 INFO DFSClient: Created token for testuser:
>>> HDFS_DELEGATION_TOKEN owner= mytestu...@paf.com, renewer=testuser,
>>> realUser=, issueDate=1597315852353, maxDate=1597920652353,
>>> sequenceNumber=55185062, masterKeyId=1964 on ha-hdfs:cluster2
>>>
>>> 20/08/13 10:50:52 INFO KMSClientProvider: New token created: (Kind:
>>> kms-dt, Service: kms://ht...@server2.paf.com:9393/kms, Ident: (kms-dt
>>> owner=testuser, renewer=testuser, realUser=, issueDate=1597315852642,
>>> maxDate=1597920652642, sequenceNumber=3929883, masterKeyId=623))
>>>
>>> 20/08/13 10:50:52 INFO HadoopFSDelegationTokenProvider: getting token
>>> for: DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_-237056190_16, ugi=
>>> testu...@paf.com (auth:KERBEROS)]] with renewer testu...@paf.com
>>>
>>> 20/08/13 10:50:52 INFO DFSClient: Created token for testuser:
>>> HDFS_DELEGATION_TOKEN owner=testu...@paf.com, renewer=testuser,
>>> realUser=, issueDate=1597315852744, maxDate=1597920652744,
>>> sequenceNumber=55185063, masterKeyId=1964 on ha-hdfs:cluster2
>>>
>>> 20/08/13 10:50:52 INFO KMSClientProvider: New token created: (Kind:
>>> kms-dt, Service: kms://ht...@server.paf.com:9393/kms, Ident: (kms-dt
>>> owner=testuser, renewer=testuser, realUser=, issueDate=1597315852839,
>>> maxDate=1597920652839, sequenceNumber=3929884, masterKeyId=624))
>>>
>>> 20/08/13 10:50:52 INFO HadoopFSDelegationTokenProvider: Renewal interval
>>> is 86400104 for token HDFS_DELEGATION_TOKEN
>>>
>>> 20/08/13 10:50:52 INFO HadoopFSDelegationTokenProvider: Renewal interval
>>> is 86400108 for token kms-dt
>>>
>>> 20/08/13 10:50:54 INFO HiveConf: Found configuration file null
>>>
>>> 20/08/13 10:50:54 INFO HadoopDelegationTokenManager: Scheduling renewal
>>> in 18.0 h.
>>>
>>> 20/08/13 10:50:54 INFO HadoopDelegationTokenManager: Updating delegation
>>> tokens.
>>>
>>> 20/08/13 10:50:54 INFO SparkHadoopUtil: Updating delegation tokens for
>>> current user.
>>>
>>> 20/08/13 10:50:55 INFO SparkHadoopUtil: Updating delegation tokens for
>>> current user.
>>> ------------------- ------------------------------
>>>
>>> In the core-site.xml, I have the following property for the 2 kms server
>>>
>>> ----------
>>>
>>> <name>hadoop.security.key.provider.path</name>
>>>
>>> <value>kms://ht...@server.paf.com;server2.paf.com:9393/kms</value>
>>>
>>> ---------
>>>
>>>
>>> Does anyone have an idea how to make it work? Or at least anyone has
>>> been able to make it work?
>>> Does anyone know where the delegation tokens are saved during the
>>> execution of jobs on k8s and how it is shared between the executors?
>>>
>>>
>>> Thanks,
>>> Michel
>>>
>>> PS: The full stack trace of the error:
>>>
>>> ----------------------------
>>>
>>> Caused by: org.apache.spark.SparkException: Job aborted due to stage
>>> failure: Task 22 in stage 0.0 failed 4 times, most recent failure: Lost
>>> task 22.3 in stage 0.0 (TID 23, 10.5.5.5, executor 1): java.io.IOException:
>>> org.apache.hadoop.security.authentication.client.AuthenticationException:
>>> Error while authenticating with endpoint:
>>> https://server.paf.com:9393/kms/v1/keyversion/dir_tmp_key%400/_eek?eek_op=decrypt
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.KMSClientProvider.createConnection(KMSClientProvider.java:525)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.KMSClientProvider.decryptEncryptedKey(KMSClientProvider.java:826)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:351)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:347)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.doOp(LoadBalancingKMSClientProvider.java:172)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:347)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:532)
>>>
>>>         at
>>> org.apache.hadoop.hdfs.HdfsKMSUtil.decryptEncryptedDataEncryptionKey(HdfsKMSUtil.java:206)
>>>
>>>         at
>>> org.apache.hadoop.hdfs.HdfsKMSUtil.createWrappedInputStream(HdfsKMSUtil.java:186)
>>>
>>>         at
>>> org.apache.hadoop.hdfs.DFSClient.createWrappedInputStream(DFSClient.java:930)
>>>
>>>         at
>>> org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:326)
>>>
>>>         at
>>> org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:321)
>>>
>>>         at
>>> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>>>
>>>         at
>>> org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:333)
>>>
>>>         at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:899)
>>>
>>>         at
>>> org.apache.hadoop.mapred.LineRecordReader.<init>(LineRecordReader.java:109)
>>>
>>>         at
>>> org.apache.hadoop.mapred.TextInputFormat.getRecordReader(TextInputFormat.java:67)
>>>
>>>         at
>>> org.apache.spark.rdd.HadoopRDD$$anon$1.liftedTree1$1(HadoopRDD.scala:282)
>>>
>>>         at
>>> org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:281)
>>>
>>>         at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:239)
>>>
>>>         at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:96)
>>>
>>>         at
>>> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
>>>
>>>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
>>>
>>>         at
>>> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>>>
>>>         at
>>> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
>>>
>>>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
>>>
>>>         at
>>> org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65)
>>>
>>>         at
>>> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
>>>
>>>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
>>>
>>>         at
>>> org.apache.spark.api.python.PairwiseRDD.compute(PythonRDD.scala:103)
>>>
>>>         at
>>> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
>>>
>>>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
>>>
>>>         at
>>> org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
>>>
>>>         at
>>> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
>>>
>>>         at
>>> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
>>>
>>>         at org.apache.spark.scheduler.Task.run(Task.scala:127)
>>>
>>>         at
>>> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:444)
>>>
>>>         at
>>> org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
>>>
>>>         at
>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:447)
>>>
>>>         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.security.authentication.client.AuthenticationException:
>>> Error while authenticating with endpoint:
>>> https://server.paf.com:9393/kms/v1/keyversion/dir_tmp_key%400/_eek?eek_op=decrypt
>>>
>>>         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.security.authentication.client.KerberosAuthenticator.wrapExceptionWithMessage(KerberosAuthenticator.java:232)
>>>
>>>         at
>>> org.apache.hadoop.security.authentication.client.KerberosAuthenticator.authenticate(KerberosAuthenticator.java:219)
>>>
>>>         at
>>> org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator.authenticate(DelegationTokenAuthenticator.java:143)
>>>
>>>         at
>>> org.apache.hadoop.security.authentication.client.AuthenticatedURL.openConnection(AuthenticatedURL.java:348)
>>>
>>>         at
>>> org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.openConnection(DelegationTokenAuthenticatedURL.java:329)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.KMSClientProvider$1.run(KMSClientProvider.java:512)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.KMSClientProvider$1.run(KMSClientProvider.java:507)
>>>
>>>         at java.security.AccessController.doPrivileged(Native Method)
>>>
>>>         at javax.security.auth.Subject.doAs(Subject.java:422)
>>>
>>>         at
>>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1730)
>>>
>>>         at
>>> org.apache.hadoop.crypto.key.kms.KMSClientProvider.createConnection(KMSClientProvider.java:506)
>>>
>>>         ... 41 more
>>>
>>> Caused by:
>>> org.apache.hadoop.security.authentication.client.AuthenticationException:
>>> GSSException: No valid credentials provided (Mechanism level: Failed to
>>> find any Kerberos tgt)
>>>
>>>         at
>>> org.apache.hadoop.security.authentication.client.KerberosAuthenticator.doSpnegoSequence(KerberosAuthenticator.java:360)
>>>
>>>         at
>>> org.apache.hadoop.security.authentication.client.KerberosAuthenticator.authenticate(KerberosAuthenticator.java:204)
>>>
>>>         ... 50 more
>>>
>>> Caused by: GSSException: No valid credentials provided (Mechanism level:
>>> Failed to find any Kerberos tgt)
>>>
>>>         at
>>> sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:162)
>>>
>>>         at
>>> sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:122)
>>>
>>>         at
>>> sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:189)
>>>
>>>         at
>>> sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:224)
>>>
>>>         at
>>> sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212)
>>>
>>>         at
>>> sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179)
>>>
>>>         at
>>> org.apache.hadoop.security.authentication.client.KerberosAuthenticator$1.run(KerberosAuthenticator.java:336)
>>>
>>>         at
>>> org.apache.hadoop.security.authentication.client.KerberosAuthenticator$1.run(KerberosAuthenticator.java:310)
>>>
>>>         at java.security.AccessController.doPrivileged(Native Method)
>>>
>>>         at javax.security.auth.Subject.doAs(Subject.java:422)
>>>
>>>         at
>>> org.apache.hadoop.security.authentication.client.KerberosAuthenticator.doSpnegoSequence(KerberosAuthenticator.java:310)
>>>
>>>         ... 51 more
>>>
>>>
>>>
>>>
>>>

Reply via email to